From 7f46b6a48b0b987e12f393558ee29359b82b9c02 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Mon, 8 Aug 2022 14:10:50 +0000 Subject: [PATCH 001/151] Add Hudi engine template --- src/Storages/StorageHudi.cpp | 35 +++++++++++++++++++++++++++++++ src/Storages/StorageHudi.h | 21 +++++++++++++++++++ src/Storages/registerStorages.cpp | 3 +++ 3 files changed, 59 insertions(+) create mode 100644 src/Storages/StorageHudi.cpp create mode 100644 src/Storages/StorageHudi.h diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp new file mode 100644 index 00000000000..9c7e5485f3c --- /dev/null +++ b/src/Storages/StorageHudi.cpp @@ -0,0 +1,35 @@ +#include +#include + +namespace DB { + +namespace ErrorCodes { + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +StorageHudi::StorageHudi( + const StorageID & table_id_, + ColumnsDescription /*columns_description_*/, + ConstraintsDescription /*constraints_description_*/, + const String & /*comment*/ +) : IStorage(table_id_) {} + + + +void registerStorageHudi(StorageFactory & factory) +{ + factory.registerStorage("Hudi", [](const StorageFactory::Arguments & args) + { + if (!args.engine_args.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Engine {} doesn't support any arguments ({} given)", + args.engine_name, + args.engine_args.size()); + + return std::make_shared(args.table_id, args.columns, args.constraints, args.comment); + }); +} + +} + diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h new file mode 100644 index 00000000000..e4fb52c6d39 --- /dev/null +++ b/src/Storages/StorageHudi.h @@ -0,0 +1,21 @@ +#pragma once + +#include "config_core.h" + +#include + +namespace DB +{ + +class StorageHudi final : public IStorage { +public: + StorageHudi( + const StorageID & table_id_, + ColumnsDescription columns_description_, + ConstraintsDescription constraints_description_, + const String & comment); + + String getName() const override { return "Hudi"; } +}; + +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 575b3de7ae2..b5561243e56 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -88,6 +88,7 @@ void registerStorageFileLog(StorageFactory & factory); void registerStorageSQLite(StorageFactory & factory); #endif +void registerStorageHudi(StorageFactory & factory); void registerStorages() { @@ -171,6 +172,8 @@ void registerStorages() #if USE_SQLITE registerStorageSQLite(factory); #endif + + registerStorageHudi(factory); } } From c5ae7126517d7326ad02981a97c9989be571dd99 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Mon, 22 Aug 2022 09:37:20 +0000 Subject: [PATCH 002/151] Add list files on s3 --- src/Storages/StorageHudi.cpp | 136 ++++++++++++++++++++++++++++++++--- src/Storages/StorageHudi.h | 24 ++++++- 2 files changed, 150 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 9c7e5485f3c..8c920808e5a 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -1,18 +1,122 @@ + +#include + #include #include +#include + +#include +#include +#include +#include +#include + namespace DB { namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int S3_ERROR; } StorageHudi::StorageHudi( + const S3::URI & uri_, + const String & access_key_, + const String & secret_access_key, const StorageID & table_id_, ColumnsDescription /*columns_description_*/, ConstraintsDescription /*constraints_description_*/, - const String & /*comment*/ -) : IStorage(table_id_) {} + const String & /*comment*/, + ContextPtr context_ +) : IStorage(table_id_) + , s3_configuration({uri_, access_key_, secret_access_key, {}, {}, {}}) + , log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")")) +{ + context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri_.uri); + updateS3Configuration(context_, s3_configuration); + + const auto & client = s3_configuration.client; + { + Aws::S3::Model::ListObjectsV2Request request; + Aws::S3::Model::ListObjectsV2Outcome outcome; + + const auto key = "hudi"; + bool is_finished{false}; + const auto bucket{s3_configuration.uri.bucket}; + + 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) + { + const auto& filename = obj.GetKey(); + LOG_DEBUG(log, "Found file: {}", filename); + } + + request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + is_finished = !outcome.GetResult().GetIsTruncated(); + } + } + +} + +void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) +{ + auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString()); + + bool need_update_configuration = settings != S3Settings{}; + if (need_update_configuration) + { + if (upd.rw_settings != settings.rw_settings) + upd.rw_settings = settings.rw_settings; + } + + upd.rw_settings.updateFromSettingsIfEmpty(ctx->getSettings()); + + if (upd.client && (!upd.access_key_id.empty() || settings.auth_settings == upd.auth_settings)) + return; + + Aws::Auth::AWSCredentials credentials(upd.access_key_id, upd.secret_access_key); + HeaderCollection headers; + if (upd.access_key_id.empty()) + { + credentials = Aws::Auth::AWSCredentials(settings.auth_settings.access_key_id, settings.auth_settings.secret_access_key); + headers = settings.auth_settings.headers; + } + + S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( + settings.auth_settings.region, + ctx->getRemoteHostFilter(), ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, + ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging); + + client_configuration.endpointOverride = upd.uri.endpoint; + client_configuration.maxConnections = upd.rw_settings.max_connections; + + upd.client = S3::ClientFactory::instance().create( + client_configuration, + upd.uri.is_virtual_hosted_style, + credentials.GetAWSAccessKeyId(), + credentials.GetAWSSecretKey(), + settings.auth_settings.server_side_encryption_customer_key_base64, + std::move(headers), + settings.auth_settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)), + settings.auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false))); + + upd.auth_settings = std::move(settings.auth_settings); +} @@ -20,14 +124,28 @@ void registerStorageHudi(StorageFactory & factory) { factory.registerStorage("Hudi", [](const StorageFactory::Arguments & args) { - if (!args.engine_args.empty()) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Engine {} doesn't support any arguments ({} given)", - args.engine_name, - args.engine_args.size()); + auto & engine_args = args.engine_args; + if (engine_args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); - return std::make_shared(args.table_id, args.columns, args.constraints, args.comment); + auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); + + configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); + configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + + + S3::URI s3_uri(Poco::URI(configuration.url)); + + return std::make_shared( + s3_uri, + configuration.auth_settings.access_key_id, + configuration.auth_settings.secret_access_key, + args.table_id, + args.columns, + args.constraints, + args.comment, + args.getContext()); }); } diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index e4fb52c6d39..d64214d6c4b 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -3,6 +3,16 @@ #include "config_core.h" #include +#include + +namespace Poco { + class Logger; +} + +namespace Aws::S3 +{ + class S3Client; +} namespace DB { @@ -10,12 +20,24 @@ namespace DB class StorageHudi final : public IStorage { public: StorageHudi( + const S3::URI& uri_, + const String& access_key_, + const String& secret_access_key_, const StorageID & table_id_, ColumnsDescription columns_description_, ConstraintsDescription constraints_description_, - const String & comment); + const String & comment, + ContextPtr context_ + ); String getName() const override { return "Hudi"; } + +private: + static void updateS3Configuration(ContextPtr, StorageS3::S3Configuration &); + +private: + StorageS3::S3Configuration s3_configuration; + Poco::Logger * log; }; } From 842c636354e2abba87cd6d3b89db4718365ab780 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 24 Aug 2022 15:07:37 +0000 Subject: [PATCH 003/151] Undone hudi --- src/Storages/StorageHudi.cpp | 188 +++++++++++++++++++++++++++-------- src/Storages/StorageHudi.h | 19 +++- 2 files changed, 166 insertions(+), 41 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 8c920808e5a..afad299241c 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -22,57 +22,75 @@ namespace ErrorCodes { StorageHudi::StorageHudi( const S3::URI & uri_, const String & access_key_, - const String & secret_access_key, + const String & secret_access_key_, const StorageID & table_id_, - ColumnsDescription /*columns_description_*/, - ConstraintsDescription /*constraints_description_*/, - const String & /*comment*/, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, ContextPtr context_ ) : IStorage(table_id_) - , s3_configuration({uri_, access_key_, secret_access_key, {}, {}, {}}) + , s3_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) , log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")")) + , query("") + , engine(nullptr) { context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri_.uri); + StorageInMemoryMetadata storage_metadata; + updateS3Configuration(context_, s3_configuration); - - const auto & client = s3_configuration.client; + + if (columns_.empty()) { - Aws::S3::Model::ListObjectsV2Request request; - Aws::S3::Model::ListObjectsV2Outcome outcome; - - const auto key = "hudi"; - bool is_finished{false}; - const auto bucket{s3_configuration.uri.bucket}; - - 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) - { - const auto& filename = obj.GetKey(); - LOG_DEBUG(log, "Found file: {}", filename); - } - - request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); - is_finished = !outcome.GetResult().GetIsTruncated(); - } + auto columns = StorageS3::getTableStructureFromData( + "Parquet", + uri_, + access_key_, + secret_access_key_, + "", + false, + std::nullopt, + context_ + ); + storage_metadata.setColumns(columns); + } + else + { + storage_metadata.setColumns(columns_); } + storage_metadata.setConstraints(constraints_); + storage_metadata.setComment(comment); + setInMemoryMetadata(storage_metadata); } +Pipe StorageHudi::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) +{ + auto keys = getKeysFromS3(); + + auto new_query = generateQueryFromKeys(std::forward(keys)); + // construct new engine + if (new_query != query) { + query = new_query; + auto new_query_uri = s3_configuration.uri.toString() + "/" + query; + engine = std::make_shared( + S3::URI(Poco::URI(new_query_uri)), + + ); + } + + return engine->read(column_names, storage_snapshot, + query_info, context, processed_stage, + max_block_size, num_streams) + + } + void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) { auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString()); @@ -118,6 +136,98 @@ void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configurati upd.auth_settings = std::move(settings.auth_settings); } +std::vector StorageHudi::getKeysFromS3() { + std::vector keys; + + const auto & client = s3_configuration.client; + + Aws::S3::Model::ListObjectsV2Request request; + Aws::S3::Model::ListObjectsV2Outcome outcome; + + bool is_finished{false}; + const auto bucket{s3_configuration.uri.bucket}; + const std::string key = ""; + + request.SetBucket(bucket); + + 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) + { + const auto& filename = obj.GetKey(); + keys.push_back(filename); + LOG_DEBUG(log, "Found file: {}", filename); + } + + request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + is_finished = !outcome.GetResult().GetIsTruncated(); + } + + return keys; +} + +std::string StorageHudi::generateQueryFromKeys(std::vector&& keys) { + // filter only .parquet files + std::erase_if(keys, [](const std::string& s) { + if (s.size() >= 8) { + return s.substr(s.size() - 8) != ".parquet"; + } + return true; + }); + + // for each partition path take only latest parquet file + + std::unordered_map> latest_parquets; + + for (const auto& key : keys) { + + auto slash = key.find_last_of("/"); + std::string path; + if (slash == std::string::npos) { + path = ""; + } else { + path = key.substr(0, slash); + } + + uint64_t timestamp = std::stoul(key.substr(key.find_last_of("_")+1)); + + auto it = latest_parquets.find(path); + + if (it != latest_parquets.end()) { + if (it->second.second < timestamp) { + it->second = {key, timestamp}; + } + } else { + latest_parquets[path] = {key, timestamp}; + } + } + + std::vector filtered_keys; + std::transform(latest_parquets.begin(), latest_parquets.end(), std::back_inserter(filtered_keys), [](const auto& kv){return kv.second.first;}); + + std::string new_query; + + for (auto&& key : filtered_keys) { + if (!new_query.empty()) { + new_query += ","; + } + new_query += key; + } + new_query = "{" + new_query + "}"; + + return new_query; +} void registerStorageHudi(StorageFactory & factory) diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index d64214d6c4b..b02de7bf130 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -24,20 +24,35 @@ public: const String& access_key_, const String& secret_access_key_, const StorageID & table_id_, - ColumnsDescription columns_description_, - ConstraintsDescription constraints_description_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, const String & comment, ContextPtr context_ ); String getName() const override { return "Hudi"; } + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + private: static void updateS3Configuration(ContextPtr, StorageS3::S3Configuration &); +private: + std::vector getKeysFromS3(); + std::string generateQueryFromKeys(std::vector&& keys); + private: StorageS3::S3Configuration s3_configuration; Poco::Logger * log; + std::string query; + std::shared_ptr engine; }; } From 66c9305668f9ce515a86be04cd01cf376c80da73 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Thu, 25 Aug 2022 08:48:49 +0000 Subject: [PATCH 004/151] Add Hudi engine --- src/Storages/StorageHudi.cpp | 84 +++++++++++++++++++++--------------- src/Storages/StorageHudi.h | 7 ++- 2 files changed, 53 insertions(+), 38 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index afad299241c..21086a7e085 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -1,16 +1,16 @@ - +#include +#include #include -#include +#include #include #include #include #include #include -#include -#include +#include namespace DB { @@ -29,38 +29,55 @@ StorageHudi::StorageHudi( const String & comment, ContextPtr context_ ) : IStorage(table_id_) - , s3_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) + , base_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) , log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")")) - , query("") - , engine(nullptr) { - context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri_.uri); StorageInMemoryMetadata storage_metadata; + updateS3Configuration(context_, base_configuration); - updateS3Configuration(context_, s3_configuration); + auto keys = getKeysFromS3(); + + auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); + LOG_DEBUG(log, "New uri: {}", new_uri); + + auto s3_uri = S3::URI(Poco::URI(new_uri)); +// StorageS3::S3Configuration s3_configuration{s3_uri, access_key_, secret_access_key_, {}, {}, {}}; + if (columns_.empty()) { auto columns = StorageS3::getTableStructureFromData( - "Parquet", - uri_, + String("Parquet"), + s3_uri, access_key_, secret_access_key_, "", false, - std::nullopt, + {}, context_ - ); + ); storage_metadata.setColumns(columns); } - else - { + else storage_metadata.setColumns(columns_); - } storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); + + s3engine = std::make_shared( + s3_uri, + access_key_, + secret_access_key_, + table_id_, + String("Parquet"), // format name + base_configuration.rw_settings, + columns_, + constraints_, + comment, + context_, + std::nullopt + ); } Pipe StorageHudi::read( @@ -72,24 +89,18 @@ Pipe StorageHudi::read( size_t max_block_size, unsigned num_streams) { - auto keys = getKeysFromS3(); + updateS3Configuration(context, base_configuration); - auto new_query = generateQueryFromKeys(std::forward(keys)); - // construct new engine - if (new_query != query) { - query = new_query; - auto new_query_uri = s3_configuration.uri.toString() + "/" + query; - engine = std::make_shared( - S3::URI(Poco::URI(new_query_uri)), - - ); - } + //auto keys = getKeysFromS3(); - return engine->read(column_names, storage_snapshot, + //auto new_uri = base_configuration.uri.uri.toString() + "/" + generateQueryFromKeys(std::forward(keys)); + //s3_configuration.uri = S3::URI(Poco::URI(new_uri)); + + return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, - max_block_size, num_streams) + max_block_size, num_streams); - } +} void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) { @@ -139,13 +150,13 @@ void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configurati std::vector StorageHudi::getKeysFromS3() { std::vector keys; - const auto & client = s3_configuration.client; + const auto & client = base_configuration.client; Aws::S3::Model::ListObjectsV2Request request; Aws::S3::Model::ListObjectsV2Outcome outcome; bool is_finished{false}; - const auto bucket{s3_configuration.uri.bucket}; + const auto bucket{base_configuration.uri.bucket}; const std::string key = ""; request.SetBucket(bucket); @@ -167,7 +178,7 @@ std::vector StorageHudi::getKeysFromS3() { { const auto& filename = obj.GetKey(); keys.push_back(filename); - LOG_DEBUG(log, "Found file: {}", filename); + //LOG_DEBUG(log, "Found file: {}", filename); } request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); @@ -256,7 +267,12 @@ void registerStorageHudi(StorageFactory & factory) args.constraints, args.comment, args.getContext()); - }); + }, + { + .supports_settings = true, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); } } diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index b02de7bf130..61e2af1340d 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -17,7 +17,7 @@ namespace Aws::S3 namespace DB { -class StorageHudi final : public IStorage { +class StorageHudi : public IStorage { public: StorageHudi( const S3::URI& uri_, @@ -49,10 +49,9 @@ private: std::string generateQueryFromKeys(std::vector&& keys); private: - StorageS3::S3Configuration s3_configuration; + StorageS3::S3Configuration base_configuration; + std::shared_ptr s3engine; Poco::Logger * log; - std::string query; - std::shared_ptr engine; }; } From e82c4800f282bb750cbd3fbec066d264c63b7d4a Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Fri, 26 Aug 2022 18:17:32 +0000 Subject: [PATCH 005/151] Better --- src/Storages/StorageHudi.cpp | 134 ++++++++++++++++++----------------- src/Storages/StorageHudi.h | 23 +++--- 2 files changed, 80 insertions(+), 77 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 21086a7e085..efba2d3f85f 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -12,9 +12,11 @@ #include -namespace DB { +namespace DB +{ -namespace ErrorCodes { +namespace ErrorCodes +{ extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int S3_ERROR; } @@ -24,11 +26,11 @@ StorageHudi::StorageHudi( const String & access_key_, const String & secret_access_key_, const StorageID & table_id_, - const ColumnsDescription & columns_, + ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_ -) : IStorage(table_id_) + ContextPtr context_) + : IStorage(table_id_) , base_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) , log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")")) { @@ -38,25 +40,16 @@ StorageHudi::StorageHudi( auto keys = getKeysFromS3(); auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); - + LOG_DEBUG(log, "New uri: {}", new_uri); auto s3_uri = S3::URI(Poco::URI(new_uri)); -// StorageS3::S3Configuration s3_configuration{s3_uri, access_key_, secret_access_key_, {}, {}, {}}; if (columns_.empty()) { - auto columns = StorageS3::getTableStructureFromData( - String("Parquet"), - s3_uri, - access_key_, - secret_access_key_, - "", - false, - {}, - context_ - ); - storage_metadata.setColumns(columns); + columns_ + = StorageS3::getTableStructureFromData(String("Parquet"), s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + storage_metadata.setColumns(columns_); } else storage_metadata.setColumns(columns_); @@ -76,30 +69,21 @@ StorageHudi::StorageHudi( constraints_, comment, context_, - std::nullopt - ); + std::nullopt); } Pipe StorageHudi::read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - unsigned num_streams) + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) { updateS3Configuration(context, base_configuration); - //auto keys = getKeysFromS3(); - - //auto new_uri = base_configuration.uri.uri.toString() + "/" + generateQueryFromKeys(std::forward(keys)); - //s3_configuration.uri = S3::URI(Poco::URI(new_uri)); - - return s3engine->read(column_names, storage_snapshot, - query_info, context, processed_stage, - max_block_size, num_streams); - + return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) @@ -128,7 +112,8 @@ void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configurati S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( settings.auth_settings.region, - ctx->getRemoteHostFilter(), ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, + ctx->getRemoteHostFilter(), + ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging); client_configuration.endpointOverride = upd.uri.endpoint; @@ -147,11 +132,12 @@ void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configurati upd.auth_settings = std::move(settings.auth_settings); } -std::vector StorageHudi::getKeysFromS3() { +std::vector StorageHudi::getKeysFromS3() +{ std::vector keys; const auto & client = base_configuration.client; - + Aws::S3::Model::ListObjectsV2Request request; Aws::S3::Model::ListObjectsV2Outcome outcome; @@ -176,7 +162,7 @@ std::vector StorageHudi::getKeysFromS3() { const auto & result_batch = outcome.GetResult().GetContents(); for (const auto & obj : result_batch) { - const auto& filename = obj.GetKey(); + const auto & filename = obj.GetKey(); keys.push_back(filename); //LOG_DEBUG(log, "Found file: {}", filename); } @@ -188,49 +174,64 @@ std::vector StorageHudi::getKeysFromS3() { return keys; } -std::string StorageHudi::generateQueryFromKeys(std::vector&& keys) { +std::string StorageHudi::generateQueryFromKeys(std::vector && keys) +{ // filter only .parquet files - std::erase_if(keys, [](const std::string& s) { - if (s.size() >= 8) { - return s.substr(s.size() - 8) != ".parquet"; - } - return true; - }); + std::erase_if( + keys, + [](const std::string & s) + { + if (s.size() >= 8) + { + return s.substr(s.size() - 8) != ".parquet"; + } + return true; + }); // for each partition path take only latest parquet file std::unordered_map> latest_parquets; - - for (const auto& key : keys) { + for (const auto & key : keys) + { auto slash = key.find_last_of("/"); std::string path; - if (slash == std::string::npos) { + if (slash == std::string::npos) + { path = ""; - } else { + } + else + { path = key.substr(0, slash); } - uint64_t timestamp = std::stoul(key.substr(key.find_last_of("_")+1)); + uint64_t timestamp = std::stoul(key.substr(key.find_last_of("_") + 1)); auto it = latest_parquets.find(path); - - if (it != latest_parquets.end()) { - if (it->second.second < timestamp) { + + if (it != latest_parquets.end()) + { + if (it->second.second < timestamp) + { it->second = {key, timestamp}; } - } else { + } + else + { latest_parquets[path] = {key, timestamp}; } } std::vector filtered_keys; - std::transform(latest_parquets.begin(), latest_parquets.end(), std::back_inserter(filtered_keys), [](const auto& kv){return kv.second.first;}); + std::transform( + latest_parquets.begin(), latest_parquets.end(), std::back_inserter(filtered_keys), [](const auto & kv) { return kv.second.first; }); std::string new_query; - - for (auto&& key : filtered_keys) { - if (!new_query.empty()) { + + for (auto && key : filtered_keys) + { + if (!new_query.empty()) + { new_query += ","; } new_query += key; @@ -243,7 +244,9 @@ std::string StorageHudi::generateQueryFromKeys(std::vector&& keys) void registerStorageHudi(StorageFactory & factory) { - factory.registerStorage("Hudi", [](const StorageFactory::Arguments & args) + factory.registerStorage( + "Hudi", + [](const StorageFactory::Arguments & args) { auto & engine_args = args.engine_args; if (engine_args.empty()) @@ -269,11 +272,10 @@ void registerStorageHudi(StorageFactory & factory) args.getContext()); }, { - .supports_settings = true, - .supports_schema_inference = true, - .source_access_type = AccessType::S3, - }); + .supports_settings = true, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); } } - diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index 61e2af1340d..dd5cc18495e 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -5,30 +5,31 @@ #include #include -namespace Poco { - class Logger; +namespace Poco +{ +class Logger; } namespace Aws::S3 { - class S3Client; +class S3Client; } namespace DB { -class StorageHudi : public IStorage { +class StorageHudi : public IStorage +{ public: StorageHudi( - const S3::URI& uri_, - const String& access_key_, - const String& secret_access_key_, + const S3::URI & uri_, + const String & access_key_, + const String & secret_access_key_, const StorageID & table_id_, - const ColumnsDescription & columns_, + ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_ - ); + ContextPtr context_); String getName() const override { return "Hudi"; } @@ -46,7 +47,7 @@ private: private: std::vector getKeysFromS3(); - std::string generateQueryFromKeys(std::vector&& keys); + std::string generateQueryFromKeys(std::vector && keys); private: StorageS3::S3Configuration base_configuration; From 3e87fd0ee46236a159bab37412199ea927402db3 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 30 Aug 2022 09:14:05 +0000 Subject: [PATCH 006/151] Add test template --- .../integration/test_storage_hudi/__init__.py | 0 tests/integration/test_storage_hudi/test.py | 107 ++++++++++++ .../.hoodie/.20220830083647456.commit.crc | Bin 0 -> 60 bytes .../.20220830083647456.commit.requested.crc | Bin 0 -> 8 bytes .../.hoodie/.20220830083647456.inflight.crc | Bin 0 -> 44 bytes .../test_table/.hoodie/.hoodie.properties.crc | Bin 0 -> 16 bytes .../.hoodie/20220830083647456.commit | 165 ++++++++++++++++++ .../20220830083647456.commit.requested | 0 .../.hoodie/20220830083647456.inflight | 161 +++++++++++++++++ .../test_table/.hoodie/hoodie.properties | 21 +++ .../.hoodie/.00000000000000.deltacommit.crc | Bin 0 -> 72 bytes .../.00000000000000.deltacommit.inflight.crc | Bin 0 -> 32 bytes .../.00000000000000.deltacommit.requested.crc | Bin 0 -> 8 bytes .../.20220830083647456.deltacommit.crc | Bin 0 -> 76 bytes ...20220830083647456.deltacommit.inflight.crc | Bin 0 -> 32 bytes ...0220830083647456.deltacommit.requested.crc | Bin 0 -> 8 bytes .../metadata/.hoodie/.hoodie.properties.crc | Bin 0 -> 16 bytes .../.hoodie/00000000000000.deltacommit | 97 ++++++++++ .../00000000000000.deltacommit.inflight | 116 ++++++++++++ .../00000000000000.deltacommit.requested | 0 .../.hoodie/20220830083647456.deltacommit | 97 ++++++++++ .../20220830083647456.deltacommit.inflight | 116 ++++++++++++ .../20220830083647456.deltacommit.requested | 0 .../metadata/.hoodie/hoodie.properties | 14 ++ ....files-0000_00000000000000.log.1_0-0-0.crc | Bin 0 -> 12 bytes ...iles-0000_00000000000000.log.1_0-52-57.crc | Bin 0 -> 96 bytes ...iles-0000_00000000000000.log.2_0-83-93.crc | Bin 0 -> 96 bytes .../files/..hoodie_partition_metadata.crc | Bin 0 -> 12 bytes .../.files-0000_00000000000000.log.1_0-0-0 | Bin 0 -> 124 bytes .../.files-0000_00000000000000.log.1_0-52-57 | Bin 0 -> 10928 bytes .../.files-0000_00000000000000.log.2_0-83-93 | Bin 0 -> 11180 bytes .../metadata/files/.hoodie_partition_metadata | 4 + .../sao_paulo/..hoodie_partition_metadata.crc | Bin 0 -> 12 bytes ...ad-0_0-73-83_20220830083647456.parquet.crc | Bin 0 -> 3432 bytes .../sao_paulo/.hoodie_partition_metadata | 4 + ...aebbad-0_0-73-83_20220830083647456.parquet | Bin 0 -> 437831 bytes .../..hoodie_partition_metadata.crc | Bin 0 -> 12 bytes ...55-0_1-73-84_20220830083647456.parquet.crc | Bin 0 -> 3432 bytes .../san_francisco/.hoodie_partition_metadata | 4 + ...268e55-0_1-73-84_20220830083647456.parquet | Bin 0 -> 438186 bytes .../chennai/..hoodie_partition_metadata.crc | Bin 0 -> 12 bytes ...e1-0_2-73-85_20220830083647456.parquet.crc | Bin 0 -> 3428 bytes .../india/chennai/.hoodie_partition_metadata | 4 + ...e5d6e1-0_2-73-85_20220830083647456.parquet | Bin 0 -> 437623 bytes 44 files changed, 910 insertions(+) create mode 100644 tests/integration/test_storage_hudi/__init__.py create mode 100644 tests/integration/test_storage_hudi/test.py create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.requested.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.inflight.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/.hoodie.properties.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit.requested create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.inflight create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/hoodie.properties create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.inflight.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.requested.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.inflight.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.requested.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.hoodie.properties.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.inflight create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.requested create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.inflight create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.requested create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/hoodie.properties create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-0-0.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-52-57.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.2_0-83-93.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..hoodie_partition_metadata.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-0-0 create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-52-57 create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.2_0-83-93 create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.hoodie_partition_metadata create mode 100644 tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/..hoodie_partition_metadata.crc create mode 100644 tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet.crc create mode 100644 tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.hoodie_partition_metadata create mode 100644 tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet create mode 100644 tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/..hoodie_partition_metadata.crc create mode 100644 tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet.crc create mode 100644 tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.hoodie_partition_metadata create mode 100644 tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet create mode 100644 tests/integration/test_storage_hudi/test_table/asia/india/chennai/..hoodie_partition_metadata.crc create mode 100644 tests/integration/test_storage_hudi/test_table/asia/india/chennai/.92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet.crc create mode 100644 tests/integration/test_storage_hudi/test_table/asia/india/chennai/.hoodie_partition_metadata create mode 100644 tests/integration/test_storage_hudi/test_table/asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet diff --git a/tests/integration/test_storage_hudi/__init__.py b/tests/integration/test_storage_hudi/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py new file mode 100644 index 00000000000..716ec15524e --- /dev/null +++ b/tests/integration/test_storage_hudi/test.py @@ -0,0 +1,107 @@ +import logging +import os + +import helpers.client +import pytest +from helpers.cluster import ClickHouseCluster + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +def prepare_s3_bucket(started_cluster): + bucket_read_write_policy = { + "Version": "2012-10-17", + "Statement": [ + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetBucketLocation", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:ListBucket", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetObject", + "Resource": "arn:aws:s3:::root/*", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:PutObject", + "Resource": "arn:aws:s3:::root/*", + }, + ], + } + + minio_client = started_cluster.minio_client + minio_client.set_bucket_policy( + started_cluster.minio_bucket, json.dumps(bucket_read_write_policy) + ) + +def upload_test_table(started_cluster): + bucket = started_cluster.minio_bucket + + for address, dirs, files in os.walk(f"{SCRIPT_DIR}/test_table"): + for name in files: + started_cluster.minio_client.fput_object(bucket, os.path.join(SCRIPT_DIR, address, name), os.path.join(address, name)) + + for obj in list( + minio.list_objects( + bucket, + recursive=True, + ) + ): + logging.info(obj.name) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "dummy", + with_minio=True + ) + + logging.info("Starting cluster...") + cluster.start() + + prepare_s3_bucket(cluster) + logging.info("S3 bucket created") + + upload_test_table(cluster) + logging.info("Test table uploaded") + + yield cluster + + finally: + cluster.shutdown() + +def run_query(instance, query, stdin=None, settings=None): + # type: (ClickHouseInstance, str, object, dict) -> str + + logging.info("Running query '{}'...".format(query)) + result = instance.query(query, stdin=stdin, settings=settings) + logging.info("Query finished") + + return result + + +def test_create_query(started_cluster): + instance = started_cluster.instances["dummy"] + bucket = started_cluster.minio_bucket + + create_query = f"""CREATE TABLE hudi ENGINE=Hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/hudi', 'minio', 'minio123')""" + + run_query(instance, create_query) + +def test_select_query(): + pass \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.crc new file mode 100644 index 0000000000000000000000000000000000000000..4bba97b9515ddcd9dd09f226e1cd81e89f1e9026 GIT binary patch literal 60 zcmV-C0K@-da$^7h00IEYSJ`4^`+RY$Y`hZ?T{ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.requested.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.requested.crc new file mode 100644 index 0000000000000000000000000000000000000000..3b7b044936a890cd8d651d349a752d819d71d22c GIT binary patch literal 8 PcmYc;N@ieSU}69O2$TUk literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.inflight.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.inflight.crc new file mode 100644 index 0000000000000000000000000000000000000000..21984c840bc3f9329c55d1d5824515745f9d0468 GIT binary patch literal 44 zcmYc;N@ieSU}Cskw&P5j)zKi6c-hD;M%E!O)z;;Tsn#4f+2CBxmvH@r-orPu0BZCS AO#lD@ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/.hoodie.properties.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/.hoodie.properties.crc new file mode 100644 index 0000000000000000000000000000000000000000..f67f76b7101c0cf7b1aba9818ac05738d10a419c GIT binary patch literal 16 XcmYc;N@ieSU}6a3SsyCGs3rmc8^!|c literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit b/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit new file mode 100644 index 00000000000..f8d6c248f49 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit @@ -0,0 +1,165 @@ +{ + "partitionToWriteStats" : { + "americas/brazil/sao_paulo" : [ { + "fileId" : "8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0", + "path" : "americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 3, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 437831, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "americas/brazil/sao_paulo", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 437831, + "minEventTime" : null, + "maxEventTime" : null + } ], + "americas/united_states/san_francisco" : [ { + "fileId" : "34b1b177-f0af-467b-9214-473ead268e55-0", + "path" : "americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 5, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 5, + "totalWriteBytes" : 438186, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "americas/united_states/san_francisco", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 438186, + "minEventTime" : null, + "maxEventTime" : null + } ], + "asia/india/chennai" : [ { + "fileId" : "92aa634e-d83f-4057-a385-ea3b22e5d6e1-0", + "path" : "asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 2, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 2, + "totalWriteBytes" : 437623, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "asia/india/chennai", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 437623, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"test_table_record\",\"namespace\":\"hoodie.test_table\",\"fields\":[{\"name\":\"begin_lat\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"begin_lon\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"driver\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"end_lat\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"end_lon\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"fare\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"partitionpath\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"rider\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"ts\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"uuid\",\"type\":[\"null\",\"string\"],\"default\":null}]}" + }, + "operationType" : "UPSERT", + "writeStats" : [ { + "fileId" : "8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0", + "path" : "americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 3, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 437831, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "americas/brazil/sao_paulo", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 437831, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "34b1b177-f0af-467b-9214-473ead268e55-0", + "path" : "americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 5, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 5, + "totalWriteBytes" : 438186, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "americas/united_states/san_francisco", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 438186, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "92aa634e-d83f-4057-a385-ea3b22e5d6e1-0", + "path" : "asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 2, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 2, + "totalWriteBytes" : 437623, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "asia/india/chennai", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 437623, + "minEventTime" : null, + "maxEventTime" : null + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 563, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "americas/brazil/sao_paulo", "americas/united_states/san_francisco", "asia/india/chennai" ], + "fileIdAndRelativePaths" : { + "92aa634e-d83f-4057-a385-ea3b22e5d6e1-0" : "asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet", + "34b1b177-f0af-467b-9214-473ead268e55-0" : "americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet", + "8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0" : "americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet" + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit.requested b/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit.requested new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.inflight b/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.inflight new file mode 100644 index 00000000000..f5ef5c92c2b --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.inflight @@ -0,0 +1,161 @@ +{ + "partitionToWriteStats" : { + "americas/brazil/sao_paulo" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "americas/united_states/san_francisco" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 5, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "asia/india/chennai" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 2, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT", + "writeStats" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 5, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 2, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "americas/brazil/sao_paulo", "americas/united_states/san_francisco", "asia/india/chennai" ], + "fileIdAndRelativePaths" : { + "" : null + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/hoodie.properties b/tests/integration/test_storage_hudi/test_table/.hoodie/hoodie.properties new file mode 100644 index 00000000000..9ae364baf33 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/hoodie.properties @@ -0,0 +1,21 @@ +#Updated at 2022-08-30T08:36:49.089844Z +#Tue Aug 30 08:36:49 UTC 2022 +hoodie.table.type=COPY_ON_WRITE +hoodie.table.metadata.partitions=files +hoodie.table.precombine.field=ts +hoodie.table.partition.fields=partitionpath +hoodie.archivelog.folder=archived +hoodie.timeline.layout.version=1 +hoodie.table.checksum=2702201862 +hoodie.datasource.write.drop.partition.columns=false +hoodie.table.timeline.timezone=LOCAL +hoodie.table.recordkey.fields=uuid +hoodie.table.name=test_table +hoodie.partition.metafile.use.base.format=false +hoodie.datasource.write.hive_style_partitioning=false +hoodie.populate.meta.fields=true +hoodie.table.keygenerator.class=org.apache.hudi.keygen.SimpleKeyGenerator +hoodie.table.base.file.format=PARQUET +hoodie.database.name= +hoodie.datasource.write.partitionpath.urlencode=false +hoodie.table.version=5 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.crc new file mode 100644 index 0000000000000000000000000000000000000000..352b882ec5a697e6d4d0754866a1cdec0be1d3be GIT binary patch literal 72 zcmV-O0Jr~Ra$^7h00ID+B#3JXBN@*inEmG*kvdVpwWAbT>N@Qjh0Y0p<^OR6NMYYe e)p}4$Xm-%78APNu2!RHTD1CAabbD4@y#KTXRUT6S literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.inflight.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.inflight.crc new file mode 100644 index 0000000000000000000000000000000000000000..b6b8f7fc1a3a439cc0c8cf1a3c06491fcf54223f GIT binary patch literal 32 ncmYc;N@ieSU}8AynOb;@FLqIaqtjZc*wA7%;rSPOY^;g^r9}$) literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.requested.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.requested.crc new file mode 100644 index 0000000000000000000000000000000000000000..3b7b044936a890cd8d651d349a752d819d71d22c GIT binary patch literal 8 PcmYc;N@ieSU}69O2$TUk literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.crc new file mode 100644 index 0000000000000000000000000000000000000000..54abc5e9b72e87753d458af67e967cccaa242c8e GIT binary patch literal 76 zcmV-S0JHyNa$^7h00IEQeq^6A>X*!Te*(sYWI5Hv06BTKu=gO04+eJ-1Ey1Rqt#G) ismNQFuvqj*Kj+@fF0)E(wf0GSx`G_6G3KKp3jiwT&ms^2 literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.inflight.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.inflight.crc new file mode 100644 index 0000000000000000000000000000000000000000..ec7cb5faf561c70caf37bd118b025f9e36947c07 GIT binary patch literal 32 ocmYc;N@ieSU}8AynOf-f;LElIN2j$?v7yDY|IOy^v9T%w0J~ufc>n+a literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.requested.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.requested.crc new file mode 100644 index 0000000000000000000000000000000000000000..3b7b044936a890cd8d651d349a752d819d71d22c GIT binary patch literal 8 PcmYc;N@ieSU}69O2$TUk literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.hoodie.properties.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.hoodie.properties.crc new file mode 100644 index 0000000000000000000000000000000000000000..509ae4501ee2922036326030efe3e5c74d1db059 GIT binary patch literal 16 XcmYc;N@ieSU}EsgOFNnOa!LyTBu)jD literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit new file mode 100644 index 00000000000..6d22af6dd2e --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit @@ -0,0 +1,97 @@ +{ + "partitionToWriteStats" : { + "files" : [ { + "fileId" : "files-0000", + "path" : "files/.files-0000_00000000000000.log.1_0-52-57", + "prevCommit" : "00000000000000", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 10928, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 10928, + "minEventTime" : null, + "maxEventTime" : null, + "logVersion" : 1, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".files-0000_00000000000000.log.1_0-52-57" ], + "recordsStats" : { + "val" : null, + "present" : false + }, + "columnStats" : { + "val" : null, + "present" : false + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"HoodieMetadataRecord\",\"namespace\":\"org.apache.hudi.avro.model\",\"doc\":\"A record saved within the Metadata Table\",\"fields\":[{\"name\":\"key\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"type\",\"type\":\"int\",\"doc\":\"Type of the metadata record\"},{\"name\":\"filesystemMetadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":{\"type\":\"record\",\"name\":\"HoodieMetadataFileInfo\",\"fields\":[{\"name\":\"size\",\"type\":\"long\",\"doc\":\"Size of the file\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"True if this file has been deleted\"}]},\"avro.java.string\":\"String\"}],\"doc\":\"Contains information about partitions and files within the dataset\"},{\"name\":\"BloomFilterMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataBloomFilter\",\"doc\":\"Data file bloom filter details\",\"fields\":[{\"name\":\"type\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Bloom filter type code\"},{\"name\":\"timestamp\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Instant timestamp when this metadata was created/updated\"},{\"name\":\"bloomFilter\",\"type\":\"bytes\",\"doc\":\"Bloom filter binary byte array\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Bloom filter entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of bloom filters for all data files in the user table\",\"default\":null},{\"name\":\"ColumnStatsMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataColumnStats\",\"doc\":\"Data file column statistics\",\"fields\":[{\"name\":\"fileName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"File name for which this column statistics applies\",\"default\":null},{\"name\":\"columnName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Column name for which this column statistics applies\",\"default\":null},{\"name\":\"minValue\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"BooleanWrapper\",\"doc\":\"A record wrapping boolean type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"boolean\"}]},{\"type\":\"record\",\"name\":\"IntWrapper\",\"doc\":\"A record wrapping int type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"LongWrapper\",\"doc\":\"A record wrapping long type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"FloatWrapper\",\"doc\":\"A record wrapping float type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"float\"}]},{\"type\":\"record\",\"name\":\"DoubleWrapper\",\"doc\":\"A record wrapping double type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"double\"}]},{\"type\":\"record\",\"name\":\"BytesWrapper\",\"doc\":\"A record wrapping bytes type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"bytes\"}]},{\"type\":\"record\",\"name\":\"StringWrapper\",\"doc\":\"A record wrapping string type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"DateWrapper\",\"doc\":\"A record wrapping Date logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"DecimalWrapper\",\"doc\":\"A record wrapping Decimal logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":30,\"scale\":15}}]},{\"type\":\"record\",\"name\":\"TimeMicrosWrapper\",\"doc\":\"A record wrapping Time-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"long\",\"logicalType\":\"time-micros\"}}]},{\"type\":\"record\",\"name\":\"TimestampMicrosWrapper\",\"doc\":\"A record wrapping Timestamp-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]}],\"doc\":\"Minimum value in the range. Based on user data table schema, we can convert this to appropriate type\",\"default\":null},{\"name\":\"maxValue\",\"type\":[\"null\",\"BooleanWrapper\",\"IntWrapper\",\"LongWrapper\",\"FloatWrapper\",\"DoubleWrapper\",\"BytesWrapper\",\"StringWrapper\",\"DateWrapper\",\"DecimalWrapper\",\"TimeMicrosWrapper\",\"TimestampMicrosWrapper\"],\"doc\":\"Maximum value in the range. Based on user data table schema, we can convert it to appropriate type\",\"default\":null},{\"name\":\"valueCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of values\",\"default\":null},{\"name\":\"nullCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of null values\",\"default\":null},{\"name\":\"totalSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total storage size on disk\",\"default\":null},{\"name\":\"totalUncompressedSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total uncompressed storage size on disk\",\"default\":null},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Column range entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of column statistics for all data files in the user table\",\"default\":null}]}" + }, + "operationType" : "UPSERT_PREPPED", + "writeStats" : [ { + "fileId" : "files-0000", + "path" : "files/.files-0000_00000000000000.log.1_0-52-57", + "prevCommit" : "00000000000000", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 10928, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 10928, + "minEventTime" : null, + "maxEventTime" : null, + "logVersion" : 1, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".files-0000_00000000000000.log.1_0-52-57" ], + "recordsStats" : { + "val" : null, + "present" : false + }, + "columnStats" : { + "val" : null, + "present" : false + } + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 67, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "files" ], + "fileIdAndRelativePaths" : { + "files-0000" : "files/.files-0000_00000000000000.log.1_0-52-57" + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.inflight b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.inflight new file mode 100644 index 00000000000..bb2542e0186 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.inflight @@ -0,0 +1,116 @@ +{ + "partitionToWriteStats" : { + "files" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "files-0000", + "path" : null, + "prevCommit" : "00000000000000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT_PREPPED", + "writeStats" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "files-0000", + "path" : null, + "prevCommit" : "00000000000000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "files" ], + "fileIdAndRelativePaths" : { + "" : null, + "files-0000" : null + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.requested b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.requested new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit new file mode 100644 index 00000000000..210201f7135 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit @@ -0,0 +1,97 @@ +{ + "partitionToWriteStats" : { + "files" : [ { + "fileId" : "files-0000", + "path" : "files/.files-0000_00000000000000.log.2_0-83-93", + "prevCommit" : "00000000000000", + "numWrites" : 4, + "numDeletes" : 0, + "numUpdateWrites" : 4, + "numInserts" : 0, + "totalWriteBytes" : 11180, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 11180, + "minEventTime" : null, + "maxEventTime" : null, + "logVersion" : 2, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".files-0000_00000000000000.log.1_0-52-57", ".files-0000_00000000000000.log.1_0-0-0", ".files-0000_00000000000000.log.2_0-83-93" ], + "recordsStats" : { + "val" : null, + "present" : false + }, + "columnStats" : { + "val" : null, + "present" : false + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"HoodieMetadataRecord\",\"namespace\":\"org.apache.hudi.avro.model\",\"doc\":\"A record saved within the Metadata Table\",\"fields\":[{\"name\":\"key\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"type\",\"type\":\"int\",\"doc\":\"Type of the metadata record\"},{\"name\":\"filesystemMetadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":{\"type\":\"record\",\"name\":\"HoodieMetadataFileInfo\",\"fields\":[{\"name\":\"size\",\"type\":\"long\",\"doc\":\"Size of the file\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"True if this file has been deleted\"}]},\"avro.java.string\":\"String\"}],\"doc\":\"Contains information about partitions and files within the dataset\"},{\"name\":\"BloomFilterMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataBloomFilter\",\"doc\":\"Data file bloom filter details\",\"fields\":[{\"name\":\"type\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Bloom filter type code\"},{\"name\":\"timestamp\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Instant timestamp when this metadata was created/updated\"},{\"name\":\"bloomFilter\",\"type\":\"bytes\",\"doc\":\"Bloom filter binary byte array\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Bloom filter entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of bloom filters for all data files in the user table\",\"default\":null},{\"name\":\"ColumnStatsMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataColumnStats\",\"doc\":\"Data file column statistics\",\"fields\":[{\"name\":\"fileName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"File name for which this column statistics applies\",\"default\":null},{\"name\":\"columnName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Column name for which this column statistics applies\",\"default\":null},{\"name\":\"minValue\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"BooleanWrapper\",\"doc\":\"A record wrapping boolean type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"boolean\"}]},{\"type\":\"record\",\"name\":\"IntWrapper\",\"doc\":\"A record wrapping int type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"LongWrapper\",\"doc\":\"A record wrapping long type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"FloatWrapper\",\"doc\":\"A record wrapping float type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"float\"}]},{\"type\":\"record\",\"name\":\"DoubleWrapper\",\"doc\":\"A record wrapping double type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"double\"}]},{\"type\":\"record\",\"name\":\"BytesWrapper\",\"doc\":\"A record wrapping bytes type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"bytes\"}]},{\"type\":\"record\",\"name\":\"StringWrapper\",\"doc\":\"A record wrapping string type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"DateWrapper\",\"doc\":\"A record wrapping Date logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"DecimalWrapper\",\"doc\":\"A record wrapping Decimal logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":30,\"scale\":15}}]},{\"type\":\"record\",\"name\":\"TimeMicrosWrapper\",\"doc\":\"A record wrapping Time-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"long\",\"logicalType\":\"time-micros\"}}]},{\"type\":\"record\",\"name\":\"TimestampMicrosWrapper\",\"doc\":\"A record wrapping Timestamp-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]}],\"doc\":\"Minimum value in the range. Based on user data table schema, we can convert this to appropriate type\",\"default\":null},{\"name\":\"maxValue\",\"type\":[\"null\",\"BooleanWrapper\",\"IntWrapper\",\"LongWrapper\",\"FloatWrapper\",\"DoubleWrapper\",\"BytesWrapper\",\"StringWrapper\",\"DateWrapper\",\"DecimalWrapper\",\"TimeMicrosWrapper\",\"TimestampMicrosWrapper\"],\"doc\":\"Maximum value in the range. Based on user data table schema, we can convert it to appropriate type\",\"default\":null},{\"name\":\"valueCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of values\",\"default\":null},{\"name\":\"nullCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of null values\",\"default\":null},{\"name\":\"totalSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total storage size on disk\",\"default\":null},{\"name\":\"totalUncompressedSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total uncompressed storage size on disk\",\"default\":null},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Column range entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of column statistics for all data files in the user table\",\"default\":null}]}" + }, + "operationType" : "UPSERT_PREPPED", + "writeStats" : [ { + "fileId" : "files-0000", + "path" : "files/.files-0000_00000000000000.log.2_0-83-93", + "prevCommit" : "00000000000000", + "numWrites" : 4, + "numDeletes" : 0, + "numUpdateWrites" : 4, + "numInserts" : 0, + "totalWriteBytes" : 11180, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 11180, + "minEventTime" : null, + "maxEventTime" : null, + "logVersion" : 2, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".files-0000_00000000000000.log.1_0-52-57", ".files-0000_00000000000000.log.1_0-0-0", ".files-0000_00000000000000.log.2_0-83-93" ], + "recordsStats" : { + "val" : null, + "present" : false + }, + "columnStats" : { + "val" : null, + "present" : false + } + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 39, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "files" ], + "fileIdAndRelativePaths" : { + "files-0000" : "files/.files-0000_00000000000000.log.2_0-83-93" + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.inflight b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.inflight new file mode 100644 index 00000000000..ea1b6a10c13 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.inflight @@ -0,0 +1,116 @@ +{ + "partitionToWriteStats" : { + "files" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "files-0000", + "path" : null, + "prevCommit" : "00000000000000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 4, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT_PREPPED", + "writeStats" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "files-0000", + "path" : null, + "prevCommit" : "00000000000000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 4, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "files" ], + "fileIdAndRelativePaths" : { + "" : null, + "files-0000" : null + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.requested b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.requested new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/hoodie.properties b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/hoodie.properties new file mode 100644 index 00000000000..845df718f6d --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/hoodie.properties @@ -0,0 +1,14 @@ +#Properties saved on 2022-08-30T08:36:47.657528Z +#Tue Aug 30 08:36:47 UTC 2022 +hoodie.compaction.payload.class=org.apache.hudi.metadata.HoodieMetadataPayload +hoodie.table.type=MERGE_ON_READ +hoodie.archivelog.folder=archived +hoodie.timeline.layout.version=1 +hoodie.table.checksum=1983687495 +hoodie.datasource.write.drop.partition.columns=false +hoodie.table.recordkey.fields=key +hoodie.table.name=test_table_metadata +hoodie.populate.meta.fields=false +hoodie.table.keygenerator.class=org.apache.hudi.metadata.HoodieTableMetadataKeyGenerator +hoodie.table.base.file.format=HFILE +hoodie.table.version=5 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-0-0.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-0-0.crc new file mode 100644 index 0000000000000000000000000000000000000000..e016a7f52627a5b0d3862ad6a1f662fc3ad8d94a GIT binary patch literal 12 TcmYc;N@ieSU}9ifcZC}O5mf@t literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-52-57.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-52-57.crc new file mode 100644 index 0000000000000000000000000000000000000000..c1136be0c0e257f2aa3aeed6a5c32dd7e097379c GIT binary patch literal 96 zcmYc;N@ieSU}De?RQxLt_>_I4%7)XcTl}8oT=)}k%-!ySA(_>TZG7lbpTeoRdX$3CKW0JTK069!9Qvd(} literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.2_0-83-93.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.2_0-83-93.crc new file mode 100644 index 0000000000000000000000000000000000000000..0aaf9e50256c454e3ecb14ec9a5c5fae645fa37d GIT binary patch literal 96 zcmYc;N@ieSU}ES={`d8LX<`T0sRhSg8n$*$xud%G6-VxaBWxdUt-gH5P1~*4QmFm& gwYUbcDec;eKAvHn%Cvog%j`|7N-J<-`{&K~0Q~|n`~Uy| literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..hoodie_partition_metadata.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..hoodie_partition_metadata.crc new file mode 100644 index 0000000000000000000000000000000000000000..1b5364eed343b390fb97133b3c42ae8d9e1c6af5 GIT binary patch literal 12 TcmYc;N@ieSU}9K*=PV-t6l(*N literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-0-0 b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-0-0 new file mode 100644 index 0000000000000000000000000000000000000000..9bf687c1a4b5fe36c79ae9759b132a53717475c4 GIT binary patch literal 124 zcmY%P2zBvPW`KY^AjOD;L85#HC;%h`1QtNd1jGi64ADOMMd^Bp1&PTSsd^cuDVci7 j`MJ6Id3w3|DXBSnE~z=GC8PcPz0)n2wJ|VGo!@#RMsVc`%MB zneYAtA;HizP6+rcN1fy<3Fd^vkc5+xke~`+MnZ_ekj0SN zE<_b)BmLVY$r9i<1DTXQ}6zuVKq85Tk&SDUW38fOFMKlJ0@ichf zhH;gGS%;!Y441(X+7+3lQ7NKQ(69$HB#GCTJqlfe6EDP|dXJ*jX#5K?7mbiKMq>mn zibiPp3o$9h6sU;x?0-X`{swiz(Kvtyv+iGo#2`+ZpFH3YwR4j6oWROW3j)S)nsFGb z6(JBJL`Ye6C^fp%$OqOlz-)zzr)X5El~6B+bl8+gG`JaqC{!p=mhR{NweAZR5a3cq z8of11VF_I#gcSsko)&)yxTVFX&BSA@C+}8o_W1!oWUACBldaLZS#24e7*1K?p8k7ozmf zC3D&-l-38&R}h2}utKBcz?ApTt)3~}#(ltw>J3LR3o#$TN?_Aq3knJh>+OYZBGS2a zFiO-qiv^-xiZKU?LEzHgH4|PUW${(_#`u zfIys8(ZB^jD0-4-c~Lt+2L_Icr-7)Sxt}NuN62`H837?AiNxz08-x_Qk%Qtium#Q# zCUMpX5t1qp84T2@6P#8TkN}sUam*>u_a)FXK|+W^0Ws>*Xik#{R21+vt;qp>MQ(3G zp;F=@G(uC_m2^aCGl9-K{VPz7Nmb?d+dav-y?7$=!Ae{bVW@dhGR;Vk|`XSzDC{jlL#(DoghBw zFeDIYN!a3#tqIu*5Td}d1SP_QVnI2M;GiVOV^9)=d!{7Z`Yu#gm=45pp!DlIz`Ry4#?qU^*dtt1C zTIla_%DU2Qk0Grd`z#hN<_9cw1m72d*W6Cc1)fvB#*7J{y3v$*2bVZ9 zG6TG>O2=+U52~s9$l0IgSjDTVspW1-m*sbkG2)GmEG;vJjb1MLcHCSr!7*!}y})Nh za8Qtt`8prGxIOI)_3bCWXS`rUPDe-C-n|_uWw~2xw$}W0@_I+y2ah_(mJZ{jq4hE@ zF!Cc6R;XupBKS#rMQU-+>#{psdy{j)E8fai5U@nB%ohMMzQy5K`Sbu}GNNl@`HD9Q z+j(OxE|}#kSyEnM2}O1C-e*&hf6VGEx&@7Yz`v}WC^h^)?s2ksZ#Z%uCzox0Op9(uVL7qxkO zDYEf8SikDtf^5&);&Y;tc=6>s-#G>ccNG6mTd z@Q6tt>|L^+oJ8Cu&EJu`$CYdn7kZr&A1zwjcmm#Bmwv%)f^EA)ds^YejJ(X4Npf=C zNsFMb8dHB6_37px5-X*N>$$f_G}0#~_*nv=guqWxM%+;(>3$$r(P?nAKF&U#R|`_D03avoH#PrchiRW4nehPWlQ+3 zCwBWcjGj2&?o8X2#$UqnzRAB(1PrrZ#7U+u}nYp>ai~WA@UT?wT;on`pC^CynKG!ZqB#m_{f7xGqE$Nhsc~iW(qF_O?)5D-rtnGJF2!|^vg$f+>zz&BEu?N zuBJG+q`aK!V3yS~qugY+gKOmJSFWEHIQY!nG^1MSbRu_8tWPQ4YPo&2S;7vp_1vm_ z?vLSl$AyT|5zokorUf6rX}xh@-b$~~_4fHz=6{}ZT+hb}UE9OXjtLYUT9xNHbHD!- zx5+7W4uL;rJkF`F9PXC7rE=VHRJ6^pSeE{q;=8sA=jQ9yk6YUwtbR-NbT22aKxz3| zx#fgpmnw-Oy=X_Br`c`C$L9P|rVWyWVb!boS4A^O`%lLe@>(V9#cTM7hkt2*XPXex5XC^6AjmBSMV?=m06!x+$sn-w2K z*m8ShVi1Nk6ynF@D$s4knU&(~HDmn#^7Fch-|aMU_`!9Xj}?d0d^;ph7|cX|*e<}d z)o5(wL@so3=?tUU9CxGFjQlxoa8}Ok`G*olfWH#htmvfvJIbG_9G5Ql2Dug?5pXM# z`J2md85l4yU|_(&fPn!60|o{R3>X+NFkoQ7z<_}P0|Wmb41i~%U5`aMj`{xqt$WqU literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.2_0-83-93 b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.2_0-83-93 new file mode 100644 index 0000000000000000000000000000000000000000..13a41e55cb2603f59eaf78103707eed6f56d0e61 GIT binary patch literal 11180 zcmeHMdpuNWA0HB;RW?Zp$56SBYZ7(~lO+r?x?wqH4o+syndY2vNp^cn%XV2xS1YZq z>+Rm!WV=zQRVZb*(2X`JZJV}Ky1dUhGls}zd-HzYKg{PdA7jq*{GRXg{e7R`n6O#z2bqG?7Vo}V{_7$n6kw{n~Aem0okR*$q>=DR94F4zwrF#sEK@vWS zIT0)-Q4+BDz=8DI@dtuO?f5B>ok61n8xbHRLLD&Kfs4R#0Rm(4 z>0Ge{flzAuC;?IPp=g+ZA`k?Fz=@*gCWR*TO0)bBxZ@ZJqZk39U>{;3Oo<7CBX9`` z$;CxLFvh1BB08T->a-IG*%`c>0LO)Z6%rBmO?mI!b}}X3I2)`e-Eai85cLtP1U3z} zARxf7?p}CDL~^(GM~UsuqJgw8MVW(mAaHfwH5G0M5*AVllDh1J7V^S?5sZZ7=AgJ} z;8NPla+SvcPxHhG3&cw&5P6laxlz@1QLk)z59tkF<6`cQ6nH&EQS+0 z8|#M@-ysLVNMH+`A(U?=6CxBTfTNf|jdH=Y>jLt^_((i;3Of4|=ouG7us{G&>XKJM=t~cmyswzqP+LZ zk2;+kDowU4qZ@|)NKsB6@GJsR!I`EhE*j-UOZCwW4Fros0#xeu9+I4N&PPNiCG4Z& z6QbB`DjxUDU046PNdx5^G1xpJmOD*eTEtNgKn(!NA}T~5XA%do6U00~t@I-h8XrPQ zD9#Ef&lQv^lL#mX1A!4(>>gq0oFT`^Y!B8{1nyC2UKrUoNRaUc0w9%0dj;nMQdQsR zsDw5U@aRO@D?Sea4)=q;NNU(XAfyfVP9YnY06FzdBA*^N5Fn+Ndq?R;<%PaM(&=L$ zFiP`MufCR+g?(d`M$rKPDL=yfiPODq1*DFCD1sUV3GgVC2MY#ZJ9HV{J4}R!3SmLt zl(VG+2N0~Bd#O+?m@;0t(ms?g1pq>?2qXfOpaQg$oi%9m0C%A4w3(dL>rT0#Z1Y8V zV!R(`fEsHlqzC^mHZ1EF-rX-%7s@a&``#H{kp7eH(Gv!KkHCqb69SHr7d5^ph6*J@ zs&J(G8fnunhOsDw1-XF^gAak0gf9N*nvkvlAp+bb2w@8-4wU0C21;@)77>GRPqjoK zi-SBb#zkV3I!$y*4yxZC${-;e|Nb(A*{z&-CvE$@lP0?)rLGB}OQ`J{*4~W`vXj+u z1mCAr>bv*&xPw(V{-d!9YN3C|DeX#kTtfHk(w3;b@Rj;s4wH+MAkgusp;RwGPo#U5 z9=t_e)O?6xYF^*gNeT_soy+UFUSSg?E{3BJh-&pIk@8U@2F&b9X|(kq3|vxyfJFdd z_kSfxr&*xq_(#k3qm7eQ?)0I9KCq4J-hzBkD;Jt1y@}NRo9)VVb@TBy1DBL`;DU0} zTiFX-RZa(E6#NZVs!GQ)R$p}H46zxh9RAY^$LUU)+??_CPZRONvC(@U8=bDIU;cDg z&TZw?^Zpk_Y86GJ3u{bF{h5!w8zM_vrc6xUt7`Z!i+v?0^V1wT=uCIkZ)-PhP15Fb zE{E7IPUW@Cb-;=nui$DLPb>Bts1|(oEOx4|%~jjmTHboU#yUql%YE*2q;#z)L;U0J zQ)yFq%#@Jde=(ZVXz;>lzI}P^@*<7gg&M2k^x1#wLF$Ub%?w;T8m$>WYuT%uvE05Y z;0$UxAus*@u0LPpt-Smo#l`Vn^W3*FIf2HkkO%Yk!QKni-B1<1n4zlWM#{T<)*W73 zv)Q;JOC!w&lP5&=- zaokJ0Nq{sgX6#)r_S{*1eq39?nT`3 ze}?wOfO)!!fseMF&^`WON6w1pQLhYcB&2K(tzB4E>{X-GGIHvTYxt?nFJfaIrXAT| z@%Gi}7w2BwN_l2=L-M5l-w#9ctrEA)u}aIXUhs2Vmfx1f>vt=^%)kEdX;|oy6JJ#h zxxBq~!8Mh}Z`}0{Eqe0Od3+12Cic&$inA3@UZfALRQXzx`Xoos<%D2L8>6m?xoFsn z;)I*?L+lfiLjKOE(G1GU7HJmd8`v6}9C-aW?X?T>=BwMs_uP{Bq`g_R@X`II)Phf@ zz$bkaTd$6`FTHuwd#va;BvYAt`FzWd;{ptP`QqT+e~mx9Xp{1B&e?DGp0y3v)KRWd z$(VTPDEpF)`nlu>>FV3uJ;eN~-OEyok`1E*eyA&b(3Z7y%s7o~tAN>k&-ADf{_tyayk2(1sbK|Ura#!;y1}JNDp6j>FrllbgPFKbx0*V!_eR+N{mtP4lLvo?p(&t4f)})>!RSAELL} z!kck#MW$JKeTdtL#b!rKNTp@GaXWq1o6L!MJzo1>YEDS5SIO5_m(q4Tu4{QzUn zwZV&!lrneCv)<~QZX3b9aNTc~ha;!4?7}3&Ab#N{$kib()^5*mV$GRAQ~N*MG&R05 zwwYn0>6D&3arsfL9HzB;Zp+DHRg+!k?*y+|XTzTH6Q4M@hZnrmZ2Yt{F1jfyjw{lv zhjW&xubIDj`B2S=JBk-ynw?;p<+s=*1UYc1&cwIj(q--YstnCcEfHUF^tH$W{})QO z4=1g1TkBU9Fx~fXBUf2H;QUj&YM%o0*JDMl8K$o4ZW+m(xOMf4I-|xN3?K3MmhmX& zSP!i~9L9tMlshxSWpF1a>gRCpvwmWyWUr_k4 z1}Em%ndarMt6SVJZ``w@iKAw^G-YTClsD|M@tv=tpFUplNzV3wVqwki$z>JS zjGT23o_M^n;ArWh<4JGk&)za^0^>_F-5D`9MGCikY{%9<+&ke4XX!6x`mWImw=;cy zuwnX&UJ*0)D`}lxZ>n!4vWQiked}kovl~9~*F)OPTOs`&>+i5`{aABBV?>_&MLQK_ z!l@^oRhFlxuzzy5wlA#Ks^qJ!6yDX*Ip&_8cdptZIG!aqH1A?F6kL+!yK#cSR|y`g=WB^Q*!6b-ORUJ=|8gsggHNSafBaetPq3@sJG5>Plm5SYex{ zs6owD_vYfEty%{amK`{?kK<|ZaxbfOR6&XU^ja$(?^T#%)^%;|*VTlE?vCFoozI2V z*^Sb8#0$99sAo9Fg~N+zZi%$YsozlTwX%MeXo5}o)}~>;DxS?ZKRwG=+@dr7&Pgra zhUi)W!|N~eHQSRLu(JJ(nA+is@_#9hzsj)~U%T?`ovNI?#SDw}HGdpFGFMk$P!u#X zG@~Jb+jRZLHnrRvmb)S^1#ZrCZkxsReWRY4lsK zKkdF*oL$=(o$a+H?9mLy?e9XbaA#2|k7?|ydUwdkD~dYiP=%j~f;GcYX}W?Z!=EwV zw(C!6C=Px^q3Z#b^oKZ4s%G|{?GCC8SPX+l;nZ(@gRemb1{oM+V32`91_l`zWMGhi eK?VjH7-V3Ofk6fa8Th|o0NjgqT#Yi!cl-we?@3z# literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.hoodie_partition_metadata b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.hoodie_partition_metadata new file mode 100644 index 00000000000..abeb826f8a4 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Tue Aug 30 08:36:48 UTC 2022 +commitTime=00000000000000 +partitionDepth=1 diff --git a/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/..hoodie_partition_metadata.crc b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/..hoodie_partition_metadata.crc new file mode 100644 index 0000000000000000000000000000000000000000..901b972aff0c2c1daa875aaf7ffe23d1fd77e34f GIT binary patch literal 12 TcmYc;N@ieSU}9)DlYRvN5nclQ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet.crc b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..eaff8b6a2b1a4363260f136c3a81f30a515c23f3 GIT binary patch literal 3432 zcmYc;N@ieSU}Et4_Q^jnZDTXn?nK7z2iUh?SX!{-Ka;_`y8G6=OaC^Clrt}?{;_xW zu>-E_;srQLpE);}OiL4z|3WPstIk$HtXbZZ=dCc4RZPdGen;n#L~P<9v8=N;X}?x< zUcx1>)jzQpmmEIv-`7$O(Kt_amnw^9iwYLc`+G)$Cj-a~oH@!qj@PI#|Kj#L~OZruZYP zy*7(~Gctd$us5V$)xhVLMQ5Zxqstq59}7}JM}hgW%T97H#3;ivl?=wfkB?#bw4==|f=+9Bv- z`1q50EUsEKyA|P6Lkqd9aZ@Eh=Jqb{EJrp!*z3w;WIi(cp&|cUWIj4O_0~L1bTJU$ zIqdK@WbyXk+7-xrWOnP-JQVjJ%h7}F@X>*v9`=x8N2-WQ_o}l;ucPa+TBg1ST@0Cj zLg_>rh`-O>b|r}J2s59KEKe+(Gw&(EIFYv~b%vRD_tf4uFt*L2V0joD8y!3Cq8~PK zLSl7l-Sg1Z+_;~gg)Ro-i#}*I!zPYPOvfgl3zr;7+*_goXFCp?zHP2cvAPSJT4b@n zs85B+d~ECnGuEXYj@!X)E^P9+#Fl-&1J3`r)!o7IB1u!&Pk gtRcgX?Dn7bagPbO+z}tYD-K<*&gZnGueNqE0Lh$Gn*aa+ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.hoodie_partition_metadata b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.hoodie_partition_metadata new file mode 100644 index 00000000000..a8c28b02044 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Tue Aug 30 08:36:50 UTC 2022 +commitTime=20220830083647456 +partitionDepth=3 diff --git a/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d929431ccd391e2d88189be66c42046c70a86252 GIT binary patch literal 437831 zcmeI*4Rl;podEDjOw-WqC+!<18ll7zK{_Q((tNPEkd`7dJBiDGIymYWX-W2iaZJeUoI8fqYO( zQZkcY543OIz4w3r_jljCcV8wmb4Kg?Rt9CkJA%`L9S?oxhWRH1r9oLFa>O+ik-t3lAtTNV&CI`D#^BqjNRt2S=&x2KjoD2=-uTfb*0;T z+CLi2EX-W~o;w%*?I#ySn$B6&uzoPwOC`ZmCBe2^?)+rMnBjumE9bvyWyP%Z#~$-S zb>|PBnl<-@^FDpW{~dAUckf?v^Y1&qe(7y{f7>>;FE{ z_L*mHdwa{fj#=9s`^uj;oUruR@=w42riZKVy6(G2Z94SgPd$9^4^R8>f9<{H$B*y& z=`+7Q=YiR~ufFPz(+)7ZNN_@s4t{*|m!3Uj%-I<{KrVRn*n1oQ`Ik3d+VgLpj{f4| ze|cfi(~n;EyBjjQW=B4J>^t9g_+aPRNWZ-1)_Z185NW}8HvIm3|M1)XKWu;E={@a- zX7-&}@$c#LZ#%ndLF5zbAKEq$sVt}pP7a>_!oHs@80Gb3c)*@|%dD*FOA_Z*99|!vi;5__>dNeedF}kwdOM`INj! z-@fOLo90Xq>9C(IKl90Dzxwra&-Pzb_UL6Fy6UQ}vu=F$uMdAB68UEL*5`*1-=L%Y z&B5y6n$KSQr`er)b?TJKVJef+jv^3Q< zrJGU>m9b^1`sVhI`gB9QF5l2;w_0}L%b=GPB$gPeXCkl<%O+Gbf9j(9_=qa~iG3p;F1GkZMAmrli*SMB8lz3_3sc+kq$s{{dEBaEK``Y>v?OpksZM~_EOm8wc zJQB=3sx-E^ZLo4%N2a^GGpyd(ojNLVT2=NOnJr)Pit^2=^LjG*6%Nf+7>wKYvD8KR z#nD{xxryGs&c4n}Px$QY+mv5rQLak5vn$os6F#AHrE|gr4rxzq?CfdlO7um7{KuJ| z!34_(5BdBivr|3Ekp_d0vemQA%hHM7)L_eb!zsNoqgew-VJ>7@*sQc~^I)U0En7O1 zLnYDj%Bsqj7Svo)70j(%RZ&)6IVNXN7A=o{wj}!VlA23uDyqgf=Z>)}8aaCOxp|Cw zk;>)aBC{+g%`T#||3spHw=7r^z3;G~JyP?la5bGxEfS_S=1dOcwx=Yzqa>_*LSy9U zvCrdUpWbW3r$pFm`1}id?OzteqW!U;H&XM*(ve=rI6MaW>@SU8R~pvL_PJl@{Mddy zwevH&JDbr>l|g?=q-ICBNXpI8IEUmwS~r$NpD77z*KD6vHNo{`f}(KZ#y<;A&~A*0 ze0*6DjsE1#L3oa<@vVDx+o8QM5vwiG5Jln^E z;hHsf{0yBk2M?N(=z}F;Y0ZE9SylduA%8pUC~qy1Uor;Xca942esg$mN~C7jypeI` z&dCFvZi;Rz2}{FKEeONs7D2I!+{!2$e)Qh_@R!$Qr)6n& z052{YnU-Mx4u@4>6(K)#nne^OF&_u`tz z53P!gEZ_1rL!2@9k8vRNXS>^XWUx9?vvWyRP&V`=46npPQ*d=jSP)J@ZW$OGUK?ho zVD#SnDfnS_3i_+FQ?M(%^Uqx)2d?e|;RmlzyMigaK0TJrVPAFj`V@Wc&}ey0Mf8V< zEeY3|U9rVOC*Ku`^LsNZHg{}FbthuCS8T2ByXf3h_3_n%Ypv?#)!Efj_;_TcH?+JP zD%yN*q9a>AYC#y5r#n+!$<5WrpS?8~CASXCTMCBt2D?1Fx@SvQSNL#qUvFp6#_DsH zS0^(aVd!vIcn25WaP?ImpRID~^6}ygEFC8$;K)*QQo;?d0|%C-lM-@xDf_yj4KIC1 zM@eS3gzNf&X&6U;sBxy}KpGD&szUcne6y?i1L<;j>a({1 zqXWG9a0l)b4y0k<=FzRXGDFX-acr~q9-~{0m*hE@Mq-OD=$E~d=5B#0ZR%p&{ zrdrpx_M|`1FtFQS1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5;&LK0Y4NXg2T009C72oNAZfB*pk1PBlyK!5-N0;2@{lyek`9j`&a@$(w6R3$)w z009C72oNAJGX>Vn%wZcf0RjXF5FkK+009C7rl&yoA<_O27$vZF)E+ws5FkK+009C7 z2oNAZ;2;RBJ_vp2f&c*m1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009E~E8sQE{gqOW z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0{bDben0A{LZFZYe78j*&6ZpV6o7ztHy6MpNs#~n0t5&UAV6TI3;0>UOg{pQ zp8x>@1PBlyK!5-N0t5&UAV7csftf3?uAqj#rl7KwECB)p2oNAZfB*pk1PDx>fIGyg zvlW>D0RjXF5FkK+009C72oNAZfB=E%Bj9JD(`PPZLVy4P0t5&UAV7cs0RjXF5FkK+ z009D1A>iZ7r$SXs0t5&UAV7csfpG*1>qAz?8AP$}HaW#Qv&Kb$009C72oNAZfB*pk z1tZ|*ykMqEq67#KAV7cs0RjXF5FkK+009C72oNAZfB=C)7x134LZ3nT6Cgl<009E8 zQNVBJyhbq92@oJafB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pkGfrURj2p256Cgl<009C7 z2oNAZfB*pk1jZ5Y%_if(*iC={0RjXF5FpU%MKuQj0RmGe;0!Z$wjvWCK!5-N0t5&U zC>nt^MKcluAwYlt0RjXF5FkK+z$6NIm1Pn!g_?!}>!)E(QX)Wr009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+0D&009C72oNAZfB*pk z1PBlyK!5-N0t5&UATYxP*3R(p8$JO71PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZ;57(%YwByjQk4J!0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C73PZrxo)*SD$&vs80t5&UAV7cs0RjXF5FkK+0D*!L z*jO-`N|XQr0t5&UAV7cs0RjXF5FkK+009C72uz;9n#l_#5&;4P2oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009CAQ^02u9!zb05gARv;54 zO#%c65FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UIA{WE4_Z$;B0zuu0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5Fl`X0{iu$S?dlkxMl z0t5&UAV7cs0RjXF5FkK+009C72+TAAzp*~kX54rQ5FkK+009C72oNAJ?FD@C>9n6B z2@)VcfB*pk1PBlyK!5-N0t5&UAV7csfte|=W@ZlCs0k1tK!5-N0t5&UAV7cs0RjXF z5FkK+009C72)r=@-evU0xa*Yw0RjXF5FkK+009C72oNYr0ly1fl+$WZ2S>n{1Rfk` zy$~QkfB*pk1PBlyK!5-N0t5&UAV7cs0RodR;AO(eHx-cp0RjXF5FkK+009C72oNAZ zfB*pk1PBlyP!IxMC@P3)k|Y5F1PBlyK%npiRu}#djDP?E0t5&UAV8p)1uR&Ld3KGB z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+0D)H%@bzo2CZwDI0RjYO zrhw05n3?l#)C33+AV7cs0Rq!pz(ReRPmuHo5FkKcItzGL{dArkxe*{hfB*pk1PBnA zLIJ-XHifuC6Cgl<009C72oNAZfIwjhtS_u|r!ZT;f-^a|j;P6T6omi*0t5&UAV7cs z0RjXFyiC9i_scjuAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C7idA53u@20*2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5GZ5; zUxHo8vnO`~1PBlyK!5-N0t5(5Hv!+SKHX+SJ_HC5AV7cs0RsCi@Xkmq*p$g6J5z1l zojq;6sg6uSSrv)B``kqDd0SF_@$TNG z=cjr%cV>E)u3WLQZbjYFquaN1b|sfKHKponJDO`->g$^uQyr`7I_f*piN=olhW4gZ jx<1URzOJsLrQw*$mljk-BK!Ye~1JEqzTD?!G-NC?o7ZX2NLJ6O8t#bo?N+Ai>(i!>tzs|XoxH(&35_$ zG9Q`!yF4@&nUBnNRQEY_&QbdevKXc8pOYtvfXvWr*ZThIa2E?9^K5^7NXDgRrJ>yl znB4Z~KkviX*yzKTUdLe*hlz1$#I1+1(dp8+%Vl9=6WM;o!`R0*+cm@3)S~kXs>@(z zo!KJGkZ|-hINhz@W9$tR2hoxaR`#*|huheCf?nbY5L9vrPaz-^w9VN8>1oKvvM^mnS z_z?pyTP(aU&X1@H;DnhQ`ZsVgh#mE|u^T2<6Ib>Q#zv=kPjhdEiQQZ+fZ|`6IE>~` zc>Dy$rZ#=Nw@wt7nT;QmP|`X~-4r*~y)ZU58a=*{)ts6o+mFmAm7VwX>tkZ{wEe8X zs3%fIRN!#~Gh;&Nu^lk>K+!vR-Qnq*PbY9OvOA=>*Ia<{jdv`T3z7<%!eb)|FC%8j zc&fnUanTlOTNZ-kcHF9=xb9Qh>yaydJd6X`u4es7IO_&vbuhNjcPSni8$>^}K7v<& zZ>_RWbzmJUvL0%(Lj^ZUfz0bs*bB~woVKrOi823X%mQ9w)S;_+?EWAXT@1vJ;qsWQ zaAxBgm|WnisqDPcw;mMIy3V*IJJo$b4$DKRKrsA)9yn zbmnXrf1S|=eHa^=HV<74&R27t_Y+-zm>RwXw^1tZ-hrntlMBvMks5t z+N%v?qtjQ{iI~8|n)2(VQ#xC|!uZH^p2Si`WIjIj>-DKWLGrmi7TKySr+;wS2NK`s!MO_n1=?lb literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.hoodie_partition_metadata b/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.hoodie_partition_metadata new file mode 100644 index 00000000000..a8c28b02044 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Tue Aug 30 08:36:50 UTC 2022 +commitTime=20220830083647456 +partitionDepth=3 diff --git a/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet b/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet new file mode 100644 index 0000000000000000000000000000000000000000..053841a0b91b8966c1f374a245825dc5ac0de548 GIT binary patch literal 438186 zcmeI*3v^s%odEDjnwGTE(%Rl(0#Ry(ShNXwH4oM`qy?7Bv*OC~@T8Mu(gxE6k`zG@ z0a+J|b`cTK!-}Bma(u8XdVmA#E)NA4N(9zLK{&GDE+`dOaVs2H-8)GpnUWNS<~5V~ zaTI3m{T~1C_rLefcPIDWIidLjRY6hk?%=rKnukC8#YOK83WK6Zw`$JIA{p2_}R{_`;-O?!iMGrVa3tfZLJ$N zY`AjchIx^Nci(kKvdyr{rwW6!gD?H&rmG8+bt2=}*>~>cN6tI)$a&Fw=N;D7(c0a5 zdbB6jbHUMfFTU#HB^C4APJ3%v+Zol#Q|9kn%hks$+QH3`*Uf8(wysv(8 z@BTXu`thUZpVo2eBPZXp{(=YJw*CGMosSLN_BY3!U%LL+XKlG5T6FKK%2VcFdhg%; z>bcW@e_L>2`G;=$(%*ih^s*guZlAaB|8D>L-+uJ5fB3-Mt{0y<{h}j|dFaf6Z?!-2 z)B%m>pIb4IjJYT{A!PW^KivJbf-&eP#y4y@YWuO}_aAfIsz(ob@PSq9j{Wo@73)^5 zD}VHmY1i8CZNKKz z-#j4}d@%8s759GPFP6V@)-A7m?AaF!Vz=G8`j|ZoFA|&06jVeW z{!?Mt`kVtw3%4#i>7M(VPJ4M>;a{Khbkh$9Ubv*V^LtH)|KQ-&4}SZBrhOj$&Z{4L z`T3@Y`q~ex7>XzJ#y@uS!S5KD`-p!`tlamxrqgdY_P)W!i-OYN@Zi_i?iko-4C^!D9&yyc_pds-{E^kwweuq9{_9UK zUo;IuXFv4Y_5El3wCU<|pS5^a6opEjT zPny<6F8#ttnPyK1zjsG+!vgvm_P8-Ib`Tc6YU`=r8w$J}*ud(S552q{d zKW5!<_P=t@OH~Uekp2GP@@JpC`1J^XwBh9c`TD0ncVT?vEB9|)e#+;6@Y{k*7M=Qs zr&e6@$20qWa^m8Jk%Nll4y3QMAq z$Y6zFOof)JmdcXYxXWwPDjrx_U$?rdvZ}7UE>YWBUR&GRTHctbY%H&hTWcWE}hInmqM7VkS` zOLu2~qP?ZBKi;3{JESk(-O|w;?{4etYwHP{&7@dbw1?HywpO)Pg<*75#yiSu>*~X3 ztE+0uYwK$g@%HMvhQ#XC<&`Z}VO-@6wJl>*N@5djqCPF0;?zO8Bvza{9;dHP9Ztj5 zsUvRM>ILn+oo6L_%d4wuN@8hm(ux$Pj%{HJsl!?N>eSIFT%9^lrLSIi_Idj}QC@Rc zxca}2-@EYfwv!G^Tb&X9o6phw-p=;as77C>^;`U*8=rdkqJnVCjTLVy`pS;v%e2)~ zutkY&euq;?>1Ry>nAfPkU#gBHU;C zTl(XzUFk1ddJ}Cuz3sumf3>9u^>)E`yGc4Y@IdM?rn9}4yIJth=Yu5KA z&g||fuVL=K9V>meRVXo`iyvQe`kMZPj|~1@%~NeMV6$BbaZwlTDrr%I<)a6yrCT-y~aoqoybieU{;LC-PvhOYk=kCLM+vs6y(EBq5(c25c z0`DFDtdEUM0K&nJ%m7B0rfwWAvXiYVlAT;0PN7qWwBZBn&}LH*eW)N@TXw_3()3wJ`s=WxwE0N-nsNBP ze{jh6!xh1ik+PSP)AnJ#)Db*wST8LKhn1Sf#75>e$zhFMnm(+@lf&Bf&g8IeUi{iF zc9FYzao9y_DikYD&5x2@j9r@E#qS2wGuXwBC8J#oe%7Rha7#hijwNCHskv0FI5oja zwm)`hdiz`7oy^RE$sydiY&0{wet->jG<5vBb6MH-OLOb^b^W`O8^VEu$FFVC(J>Bv z6c3K={OCYI*|uoe*Mrj7=-e>vWe7R$@^QAi+mhYwJSbQbDf@lcThZ`#7kOP29o)@tt^?ZD`P`rNqzzk&K(BZANusWGB$;ue*c|} zVP{zqH2RrXbU|5hbmxA{!dYxrZ0Ydfx*~p7Z%;*E+or_kc_^1~mG z#Fl)tH~fC4w_39O&4j`Tx-009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfIyxItjm*e8YTe(1PBlyFv|tj&hqhPHoWk0t^OfE zfB*pkvq`{1*=AEVjfDUK0t5&UAV7cs0RodC;2~U-fT#%q1PBlyK!5-N0t5);yMU9+ zsow%z$~E=IdLTf6009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 zvO>VK-?AbNg2+YzPmjvRtcdA#1su^|m&Zy11PBlyK!5-N0t5&UAV7e?lnVH6W=eTA zPJjRb0t5&UAV7csf!q*qwMlN|Nt6T#5FjwE0=_$)R&JdUAV7cs0RjXF5FkK+z!(8P zJu-&G0s;gG5FkK+009C72oNAZAb$m1nGfT1_%%!K!5-N0t5&UAV7csf!QJ8TY}k<$1EL%dkwNQF~TB1fB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjYaRlsi< zATVnMoPy8V z)EXE80<%oO@%nWVy5@D(Sw&!2z-D4on6Q9ntWB6u0|W>VAV7e?ObTqA$CQ#Md z}~xF_N{9I1PBlykY@sZUp3DXZny*p5FkK+009C72oNAZ zfB=EX5%AZX$#K*Qf!q?vpTAh=R$fF)fB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72xN(X=T2lvB62Z|OmFqHk>z4`#Y}(z0RjZ3T);)XQ%W=^c0t5&UAV7cs0Rq`1;L@UO z%7$175FkK+K$Z%aFl13;BqNKGArt}xa!0`519K-=A|*h8009C72oNAZfB*pk1PEk> zfN$foA`OBdK!5-N0t5&U$YX(Zc|3MQCqRGz0RjXF5FkK+009C72oNAZfIuz_xQQp1 zvnTe+5b!sSNl;uf32n@>CS1=p%Mxlh1PBlyKwuIDoP$jwrltrGAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5Xc9CwfQheVYBmn{ha$CT}D7W(`dIAIp5XeFS z7Yt@$QiPN(0#mztcmzYXak-A90D&nK@D=+M z;%a&V0-gjh0X$W+R={?hwW-Paz?|7)K(dJ}Cuz3nX@Nt_cqu(G~xbya0mU3p!iwza&rwzajq zF;UrAURP7u*ic;;uWx8iq}7g}y-V%(){c(0n)>?k%8rJ{^4gBJ)#VNG`0DcdnyS{; z_SV{}`gm<5_Wm>Cy=QJo^p|h$U4B-gx39COdwEqwRdq%6@|CSyI=kAJ*ViX%D%%<= z8*6GB>Jn|MtJ`YYI^uP0HMOnviH@3@>Z+RR>bAz(gG*lBr!*4D{C|;WzBVs%Lh}cz F{vYhxb9w*( literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/asia/india/chennai/..hoodie_partition_metadata.crc b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/..hoodie_partition_metadata.crc new file mode 100644 index 0000000000000000000000000000000000000000..901b972aff0c2c1daa875aaf7ffe23d1fd77e34f GIT binary patch literal 12 TcmYc;N@ieSU}9)DlYRvN5nclQ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet.crc b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..9fae4a530439b64300fa078deaf19eda992646cc GIT binary patch literal 3428 zcmYc;N@ieSU}9LgovlRl$*&o!w7J$i+&5ESa<^O7g0GXplb&tertr)`@_*KkCYS!s zSp_9GrruK^z`?M=oej-IZNbM&{?$(3GN^?ooM@W50 zN|X#fHG3AANrU9gr}Lb{rw&=}yO~EOG9MdTP<+czY~u71o4t5qBDL*$VO%drZ8J$V zZ`zv^;5^-XD1e7ly)gBfQ!8y@Y(lhQO|2?SjlY1mGmMRmUc8AR9Gf`0nA`W2OVGtY z{FJ3riPDcvFSW&_EK2SYGE@rj|2x;g5>)IAQ5V}Y?@baJ(l0gR1I-$~D9L*^61 z{=O%omKb%|)Nov|slp~steB~Wv?D&XKhAtOj!%9t$vx>ln}^LksYhh|&xoZ~Tk|S39;q5B@qb@UJVd_ITCTGFeLzTABT;@Pe vx2w(Ar)TD${v2lK^_B8BVQg%)Ps42ciE@c=u*reM=EWYrk1Y3UV;L&|A4@|J literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.hoodie_partition_metadata b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.hoodie_partition_metadata new file mode 100644 index 00000000000..a8c28b02044 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Tue Aug 30 08:36:50 UTC 2022 +commitTime=20220830083647456 +partitionDepth=3 diff --git a/tests/integration/test_storage_hudi/test_table/asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet new file mode 100644 index 0000000000000000000000000000000000000000..c1ba250ed09695e1f95880fc1bf3457620be17b2 GIT binary patch literal 437623 zcmeI*3wV{~eE{$i3?U#Og5UAPMnu#+UI->RNg&Qu0^%d;I#B1V)`fF&P9QTRm;^14 zm#Oukb6Q=&b+vVBr*^Gdt+VQGuCtSFEu#*nPMx$`M%iPlYqfoxb6)m+CppQ{1mqeL z^2-y#cfR-X|G&TYyS%xaa~3zATOAY!CkCGg?s)2Rw@*Gb7!wr7Vn^Ln8teI(mRY*A zs-~)@=E17^SUNeaYub6q*q?v%w@(hj@*oyW3FZdBdS>Hu2bYK46bB_i*s#BKb9(90 zr7bN>OJZeTeD{H<&#=kLpe6Y1OKZL~CTbEJyh(iQ+A~VdIHM%~PbKr(TGH+5-;H-1 z)^Y9G|2QR9JMFdC=l1mU7e&Ft;pZKjAIyax&{weGJCh!tF|oMuu#3L`+|=(Z{q9dT zr=I@3@kihF?yH}=@3xD6e%`(ReCfCE`&>)SygxYT=#!qEe$m!*-#_xoQ{&SfdSUhE z`l4-X&wt~w)9c@j9em?wFT5ZMSscs?76-4r{)3v1e}i{JrtrlqwFsSQLaq{L;-&y)u44yh%@7JbB){iDNgtu=u|Aqt5%a2M_-G zIaeR^>Rs#Zc;t~sXFR>|rZc|RxZ&SVIPT^Xv*R|bdG4IYuKU}?&zF?Uz2W@_F8RTq zC4c_2Z~WImZ$G|%!a2K{T`Z)yAh>_?#$S!w!8|^&?Zw9)-tyCF|CjyEw=+L^?}rbM zi+%FqI~MnnL_Iw6(pPRR-9Zl@|Jk3MRQ>cjfAsD>SCpN6_%Ba;Xk4s&`dO=UJroD! z!Rf(kcfI=LAp?B=Gn$VRHq1J^^55!eYD;2Q{=@TYC-1>n&kr`$?8#e{Z|`^@qMzzvjI^elzySv(Ls0hWgj{{m&CcI~Z!q(sjRl>+&o9{@sJVQTi{f z2QPc^pxF7pb<&c)9?lOQxbCI)9dz(sUBl7q{_2VE{`gN5Cq1$5%WEdbZu;>b{!K5{ zkRG{51oMNNZdv#KxPkK$UQmwTaL%IMX}Np#Th){HU|JRjufOrV8+W97P0Q3P{_@7p zU)Olm>vxTtaYU@_TeH5(D2w)wjndOzM-YEs{2wx3*`(ug%M|?d_@7 zvcw=e3!2Pps7a;jleO8(%-m#4Wo=d6oXS*kZe3+Il}y*vWa~2Z+3L!wrkct*$;!EP zO#?<;me|uS<`kq;`qizs969~H7fvfnlwLXcTlanX*{!D)7H|3Q<*(dx$d;)12WPKH z+&<^0g~byxovoK=J1c9dlVyp52ZcRMeECytFMei2)I---FIp43^_9XNN?$(l!iFz= z<;ziVtYrF`Z~pL`A1Hn*J~p=Xh3a`>@!RW`wodx})8`d*J_z+k`H$~x&Ezxg-!AC0 zbX)E3Jav6hc*G@255MP~m!r*s;(c%=mB+6NTB7r?tUSJ@D7dYtcl*sT!M6PNrm=lh zZZ7THe4{kDAz$Oo@%`Id5ALh7VPbH7zRJc){Z%$i4zA7bUq7X<`r4^|n{Q1GuE}RI ztv8bp;+-jH9<#ioBh#9l-JR{~Zt6~@+X^?EIIESPxQn8dWEp2|(l9V=F} zhSghFWRHuTQyyLDqVmH&RNj@nti7YK!qj|)o_?Dy&8{vij^~S4raHS@yIVWjn^vZ} zmlsw!G+(8qwJqD!p6X9vLYTn#baq*5dsADgI~Ei^?r85xu%zc0&vzJA8JBI(^f%~v z6jhJPid#~h*`Ah@`cnFEMq_gaVZO`au-TaIuAWB4t5&sUdQ0NPW#wh>98z&rc`&i8 zuC%zMY*5aiI9?pTr6~R%MHN?7l$H;2Ssi3oJa+uR%kLodVr8>(%h@r}dbKzh6N|;) zI662izBvfev5GC>?Iubs7N$1nOy+XCvnYOBQCRoX`q=S(uSL>M(Z{lchg zW(JA)x8p%)tm4gZy_}Ca$l;NT`JxrE@vkNo2Gs@_}!-EHkVilXl z_RrCF4#|RP+BCLeb8-1#cZ$LGg;O=ig>JB#gG|<=GlO{iM~R>%R&jH6$vf$1!v5GfB_X)ix zN&nR*=Y3K9hN7_i)c%`qqVJ9zje(#@2AjpFe(juZlefO;LPPQCM2>_Js1nRY2i(7^q-< zP*^eu-r8fLW41gvBUZ6tV*j}Em)+h`U0)QIhNH@_DiWpnrA9QWfqM%_^|fQdQQdoF zG^&l0J{+OYZR4acLVjVA=v%Es5eDupjPQ6AVM{K;rpf&ga<_-v04|JQTU4=Wa@c)- zVUsA$uXdvD2ktHG{`si;yQfD3SU;tIT0VHo$;IipF|40bvFVVZx#IjHH+$2gE6&=f z{i*fb^KvO&6MwQOEUUQx(DFq88mwS59RHyE2RT;O92=czM+Ng^6>o*nihIwq@P68R zm|PR?50UdrxkTUkEa1y4)=~*L&|xX4*cy$vkvyIMM58 zL^qlE9aG}P6{YcKg2TeKVq0Qb@8xxN>hjKx*&3m&UEg9Z zx4zsfUG}X<_o%nN_3OZ1G99bJ#d&TTwxi$MxTAe{8uzS>2evGj{q2P9T|f_P`q3HZ zn(kh{(b9Z(LiSC4w2mLx!AB1_x7^>IhTUBQTeWrco>|+mjlO{x*lK%8UUGFTap)DD zt=-wq*(*9S*|yoy)gnml!009C72oNAZfB*pk1PBlyK!5-N0t5&U7)k-Z%os{-u?P?# zK!5-N0t5&UAV7cs0RjXF5FkK+0D%!B;0>ep%~UPgH{I%$009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfWW>M@Tai*T3G)C2oNAZfB*pk1PBly zFp2~gj-s&|DuGcX;M5*PlWnL32oNAZfB*pk1PBlyK!5-N0t5&U7>)vdP(2)HMNR|= z5FkK+009C72oNAZfB=CZ74U{bL&_}}0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5)`QvuJS->1AfCqRGz z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF44uG|p$nCdAKZhDd+_0RjXF5FkKcL<#t%{)n0`$r2zyfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNB!hXURhbq@hGCqRGz0RjXF5FkK+009C72oTr@0$wS2AE4@z009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0tEJ( zz=Hi2SzrPL2oNAZfB*pk1PBlyK!5-N0s{pW4%}rA0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0Rkggz>DCHxJFBW009C72oNAZfB*pk1PBlyK!5-N0tAM; zz{264p!^6BAV6S$3M|^6!Nn#(fB*pk1PB}u0`m`uQ5XaP0t5&UAV7csfgu&})RQ6Q z7K{J^0t5&UAV7cs0RjXF5FkK+009C72oNAZV3Y~?v$jz-;f70q009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7e?5DIwdwjsn8iU0uu1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oTtzfLD6nA&)`=1PBlyK!5-N0t5&UAh3S~ycFF2aTk>U0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB=C#5%9{Yd!nZ;0RjXF5FkK+009CcN?^%| z%2u)j2oNAZfB*pk1PBlyK!5-N0tAMGz_Q_xgd7MEAV7cs0RjXF5Ez*Peoa3z=S|iG z2oNAZfB*pk2fV5FkK+009C72oNAZfB=EvE3j<%rY1821a>Uo^xd(KQUU}B5FkK+009C72<%UR z`TH}t*aQd=AV6R*2>g015iIZM$h2mgR+VUO=TN0bL$&gDywR% z>MLtg>2zg7DwU~h&L-=t)AgBbbF!hJb?Va}Xq~RfRM%(fQk5;~nsnH@rM@zis%x&y zCe!J#ESt(!C1Z)RR;D^HTb1puT+un>@@!{UYe)Nx>eYJ0b={eb!WU{6@SyR*8P GtN$O{ABc|t literal 0 HcmV?d00001 From 2abe284859263d7285903196884bcf1de6debcfd Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 30 Aug 2022 17:38:02 +0000 Subject: [PATCH 007/151] Fix engine bug --- src/Storages/StorageHudi.cpp | 11 ++++++----- src/Storages/StorageHudi.h | 1 + 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index efba2d3f85f..b4eba258d0e 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -33,6 +33,7 @@ StorageHudi::StorageHudi( : IStorage(table_id_) , base_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) , log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")")) + , table_path(uri_.key) { StorageInMemoryMetadata storage_metadata; updateS3Configuration(context_, base_configuration); @@ -42,7 +43,7 @@ StorageHudi::StorageHudi( auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); LOG_DEBUG(log, "New uri: {}", new_uri); - + LOG_DEBUG(log, "Table path: {}", table_path); auto s3_uri = S3::URI(Poco::URI(new_uri)); if (columns_.empty()) @@ -143,9 +144,9 @@ std::vector StorageHudi::getKeysFromS3() bool is_finished{false}; const auto bucket{base_configuration.uri.bucket}; - const std::string key = ""; request.SetBucket(bucket); + request.SetPrefix(table_path); while (!is_finished) { @@ -155,16 +156,16 @@ std::vector StorageHudi::getKeysFromS3() ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}", quoteString(bucket), - quoteString(key), + quoteString(table_path), backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); const auto & result_batch = outcome.GetResult().GetContents(); for (const auto & obj : result_batch) { - const auto & filename = obj.GetKey(); + const auto & filename = obj.GetKey().substr(table_path.size()); // object name without tablepath prefix keys.push_back(filename); - //LOG_DEBUG(log, "Found file: {}", filename); + LOG_DEBUG(log, "Found file: {}", filename); } request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index dd5cc18495e..47dff1c2a7b 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -53,6 +53,7 @@ private: StorageS3::S3Configuration base_configuration; std::shared_ptr s3engine; Poco::Logger * log; + String table_path; }; } From 6b95faf2b3a9bdae644156dcd0764030a9bb570a Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 30 Aug 2022 17:38:57 +0000 Subject: [PATCH 008/151] Add createquery test --- .../test_storage_hudi/configs/conf.xml | 11 +++++++++++ tests/integration/test_storage_hudi/test.py | 16 ++++++++++------ 2 files changed, 21 insertions(+), 6 deletions(-) create mode 100644 tests/integration/test_storage_hudi/configs/conf.xml diff --git a/tests/integration/test_storage_hudi/configs/conf.xml b/tests/integration/test_storage_hudi/configs/conf.xml new file mode 100644 index 00000000000..e3e8627d95e --- /dev/null +++ b/tests/integration/test_storage_hudi/configs/conf.xml @@ -0,0 +1,11 @@ + + + + + http://nginx:80/test_{_partition_id} + PUT + TSV + column1 UInt32, column2 UInt32, column3 UInt32 + + + diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index 716ec15524e..b2f6048f408 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -1,5 +1,6 @@ import logging import os +import json import helpers.client import pytest @@ -50,17 +51,19 @@ def prepare_s3_bucket(started_cluster): def upload_test_table(started_cluster): bucket = started_cluster.minio_bucket - for address, dirs, files in os.walk(f"{SCRIPT_DIR}/test_table"): + for address, dirs, files in os.walk(SCRIPT_DIR + "/test_table"): + address_without_prefix = address[len(SCRIPT_DIR):] + for name in files: - started_cluster.minio_client.fput_object(bucket, os.path.join(SCRIPT_DIR, address, name), os.path.join(address, name)) + started_cluster.minio_client.fput_object(bucket, os.path.join(address_without_prefix, name), os.path.join(address, name)) for obj in list( - minio.list_objects( + started_cluster.minio_client.list_objects( bucket, recursive=True, ) ): - logging.info(obj.name) + logging.info(obj.object_name) @pytest.fixture(scope="module") def started_cluster(): @@ -68,6 +71,7 @@ def started_cluster(): cluster = ClickHouseCluster(__file__) cluster.add_instance( "dummy", + main_configs=["configs/conf.xml"], with_minio=True ) @@ -99,9 +103,9 @@ def test_create_query(started_cluster): instance = started_cluster.instances["dummy"] bucket = started_cluster.minio_bucket - create_query = f"""CREATE TABLE hudi ENGINE=Hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/hudi', 'minio', 'minio123')""" + create_query = f"""CREATE TABLE hudi ENGINE=Hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" run_query(instance, create_query) def test_select_query(): - pass \ No newline at end of file + pass From 7d8cc20c4600fd5fd3aab76ca84584ed65c07896 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 31 Aug 2022 09:26:53 +0000 Subject: [PATCH 009/151] Add select test --- .../test_storage_hudi/configs/conf.xml | 11 ----- tests/integration/test_storage_hudi/test.py | 41 +++++++++++++------ 2 files changed, 28 insertions(+), 24 deletions(-) delete mode 100644 tests/integration/test_storage_hudi/configs/conf.xml diff --git a/tests/integration/test_storage_hudi/configs/conf.xml b/tests/integration/test_storage_hudi/configs/conf.xml deleted file mode 100644 index e3e8627d95e..00000000000 --- a/tests/integration/test_storage_hudi/configs/conf.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - - - http://nginx:80/test_{_partition_id} - PUT - TSV - column1 UInt32, column2 UInt32, column3 UInt32 - - - diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index b2f6048f408..e2d97f6489f 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -5,6 +5,7 @@ import json import helpers.client import pytest from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -57,21 +58,12 @@ def upload_test_table(started_cluster): for name in files: started_cluster.minio_client.fput_object(bucket, os.path.join(address_without_prefix, name), os.path.join(address, name)) - for obj in list( - started_cluster.minio_client.list_objects( - bucket, - recursive=True, - ) - ): - logging.info(obj.object_name) - @pytest.fixture(scope="module") def started_cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance( - "dummy", - main_configs=["configs/conf.xml"], + "main_server", with_minio=True ) @@ -100,12 +92,35 @@ def run_query(instance, query, stdin=None, settings=None): def test_create_query(started_cluster): - instance = started_cluster.instances["dummy"] + instance = started_cluster.instances["main_server"] bucket = started_cluster.minio_bucket create_query = f"""CREATE TABLE hudi ENGINE=Hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" run_query(instance, create_query) -def test_select_query(): - pass +def test_select_query(started_cluster): + instance = started_cluster.instances["main_server"] + bucket = started_cluster.minio_bucket + columns = ['_hoodie_commit_time', '_hoodie_commit_seqno', '_hoodie_record_key', + '_hoodie_partition_path', '_hoodie_file_name', 'begin_lat', + 'begin_lon', 'driver', 'end_lat', 'end_lon', + 'fare', 'partitionpath', 'rider', 'ts', 'uuid'] + + # create query in case table doesn't exist + create_query = f"""CREATE TABLE IF NOT EXISTS hudi ENGINE=Hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" + + run_query(instance, create_query) + + select_query = "SELECT {} FROM hudi FORMAT TSV" + + for column_name in columns: + result = run_query(instance, select_query.format(column_name)).splitlines() + assert(len(result) > 0) + + # test if all partition paths is presented in result + distinct_select_query = "SELECT DISTINCT partitionpath FROM hudi FORMAT TSV" + result = run_query(instance, distinct_select_query).splitlines() + expected = ['americas/brazil/sao_paulo', 'asia/india/chennai', 'americas/united_states/san_francisco'] + + assert TSV(result) == TSV(expected) \ No newline at end of file From 0e9c3f299c75c16c05cef5b65be04fa539c2b4cd Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 31 Aug 2022 09:47:46 +0000 Subject: [PATCH 010/151] Fix bug in test --- tests/integration/test_storage_hudi/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index e2d97f6489f..c9415e28151 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -119,8 +119,8 @@ def test_select_query(started_cluster): assert(len(result) > 0) # test if all partition paths is presented in result - distinct_select_query = "SELECT DISTINCT partitionpath FROM hudi FORMAT TSV" - result = run_query(instance, distinct_select_query).splitlines() - expected = ['americas/brazil/sao_paulo', 'asia/india/chennai', 'americas/united_states/san_francisco'] - + distinct_select_query = "SELECT DISTINCT partitionpath FROM hudi ORDER BY partitionpath FORMAT TSV" + result = run_query(instance, distinct_select_query) + expected = ['americas/brazil/sao_paulo', 'americas/united_states/san_francisco', 'asia/india/chennai'] + assert TSV(result) == TSV(expected) \ No newline at end of file From fc2c8f37b1efa1e4f3cecd6583758272223da28b Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Thu, 1 Sep 2022 19:21:53 +0000 Subject: [PATCH 011/151] Add DeltaLake storage template, file downlad from s3 --- src/Storages/StorageDelta.cpp | 294 ++++++++++++++++++++++++++++++ src/Storages/StorageDelta.h | 106 +++++++++++ src/Storages/registerStorages.cpp | 2 + 3 files changed, 402 insertions(+) create mode 100644 src/Storages/StorageDelta.cpp create mode 100644 src/Storages/StorageDelta.h diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp new file mode 100644 index 00000000000..8da5b5ce323 --- /dev/null +++ b/src/Storages/StorageDelta.cpp @@ -0,0 +1,294 @@ +#include +#include +#include + +#include +#include +#include + +#include +#include + +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int S3_ERROR; +} + +void DeltaLakeMetadata::add(const String & key, uint64_t timestamp) { + file_update_time[key] = timestamp; +} + +void DeltaLakeMetadata::remove(const String & key, uint64_t /*timestamp */) { + file_update_time.erase(key); +} + +std::vector DeltaLakeMetadata::ListCurrentFiles() && { + std::vector keys; + keys.reserve(file_update_time.size()); + + for (auto && [k, _] : file_update_time) { + keys.push_back(k); + } + + return keys; +} + +JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, + const String & table_path_, + Poco::Logger * log_) : + base_configuration(configuration_) + , table_path(table_path_) + , metadata() + , log(log_) + { + Init(); + } + +void JsonMetadataGetter::Init() { + auto keys = getJsonLogFiles(); + char localbuf[100]; + + for (const String & key : keys) { + auto buf = createS3ReadBuffer(key); + + while (!buf->eof()) { + buf->read(localbuf, 100); + + LOG_DEBUG(log, "{}", String(localbuf)); + } + } + +} + +std::vector JsonMetadataGetter::getJsonLogFiles() { + std::vector keys; + + const auto & client = base_configuration.client; + + Aws::S3::Model::ListObjectsV2Request request; + Aws::S3::Model::ListObjectsV2Outcome outcome; + + bool is_finished{false}; + const auto bucket{base_configuration.uri.bucket}; + + request.SetBucket(bucket); + request.SetPrefix(table_path + "_delta_log"); + + 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(table_path), + backQuote(outcome.GetError().GetExceptionName()), + quoteString(outcome.GetError().GetMessage())); + + const auto & result_batch = outcome.GetResult().GetContents(); + for (const auto & obj : result_batch) + { + const auto & filename = obj.GetKey(); + + if (filename.substr(filename.size() - 5) == ".json") + keys.push_back(filename); + } + + request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + is_finished = !outcome.GetResult().GetIsTruncated(); + } + + return keys; +} + +std::unique_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key) { + // size_t object_size = DB::S3::getObjectSize(base_configuration.client, base_configuration.uri.bucket, key, base_configuration.uri.version_id, false); + + // TBD: add parallel downloads + return std::make_unique(base_configuration.client, base_configuration.uri.bucket, key, base_configuration.uri.version_id, /* max single read retries */ 10, ReadSettings{}); +} + +StorageDelta::StorageDelta( + const S3::URI & uri_, + const String & access_key_, + const String & secret_access_key_, + const StorageID & table_id_, + ColumnsDescription /*columns_*/, + const ConstraintsDescription & /*constraints_*/, + const String & /*comment*/, + ContextPtr context_) + : IStorage(table_id_) + , base_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) + , log(&Poco::Logger::get("StorageDeltaLake (" + table_id_.table_name + ")")) + , table_path(uri_.key) +{ + //StorageInMemoryMetadata storage_metadata; + updateS3Configuration(context_, base_configuration); + + Init(); + + // auto keys = getKeysFromS3(); + + // auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); + + // LOG_DEBUG(log, "New uri: {}", new_uri); + // LOG_DEBUG(log, "Table path: {}", table_path); + // auto s3_uri = S3::URI(Poco::URI(new_uri)); + + // if (columns_.empty()) + // { + // columns_ + // = StorageS3::getTableStructureFromData(String("Parquet"), s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + // storage_metadata.setColumns(columns_); + // } + // else + // storage_metadata.setColumns(columns_); + + // storage_metadata.setConstraints(constraints_); + // storage_metadata.setComment(comment); + // setInMemoryMetadata(storage_metadata); + + // s3engine = std::make_shared( + // s3_uri, + // access_key_, + // secret_access_key_, + // table_id_, + // String("Parquet"), // format name + // base_configuration.rw_settings, + // columns_, + // constraints_, + // comment, + // context_, + // std::nullopt); +} + +void StorageDelta::Init() { + JsonMetadataGetter getter{base_configuration, table_path, log}; +} + +Pipe StorageDelta::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) +{ + updateS3Configuration(context, base_configuration); + + return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); +} + +void StorageDelta::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) +{ + auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString()); + + bool need_update_configuration = settings != S3Settings{}; + if (need_update_configuration) + { + if (upd.rw_settings != settings.rw_settings) + upd.rw_settings = settings.rw_settings; + } + + upd.rw_settings.updateFromSettingsIfEmpty(ctx->getSettings()); + + if (upd.client && (!upd.access_key_id.empty() || settings.auth_settings == upd.auth_settings)) + return; + + Aws::Auth::AWSCredentials credentials(upd.access_key_id, upd.secret_access_key); + HeaderCollection headers; + if (upd.access_key_id.empty()) + { + credentials = Aws::Auth::AWSCredentials(settings.auth_settings.access_key_id, settings.auth_settings.secret_access_key); + headers = settings.auth_settings.headers; + } + + S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( + settings.auth_settings.region, + ctx->getRemoteHostFilter(), + ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, + ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging); + + client_configuration.endpointOverride = upd.uri.endpoint; + client_configuration.maxConnections = upd.rw_settings.max_connections; + + upd.client = S3::ClientFactory::instance().create( + client_configuration, + upd.uri.is_virtual_hosted_style, + credentials.GetAWSAccessKeyId(), + credentials.GetAWSSecretKey(), + settings.auth_settings.server_side_encryption_customer_key_base64, + std::move(headers), + settings.auth_settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)), + settings.auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false))); + + upd.auth_settings = std::move(settings.auth_settings); +} + +String StorageDelta::generateQueryFromKeys(std::vector && keys) +{ + String new_query; + + for (auto && key : keys) + { + if (!new_query.empty()) + { + new_query += ","; + } + new_query += key; + } + new_query = "{" + new_query + "}"; + + return new_query; +} + + +void registerStorageDelta(StorageFactory & factory) +{ + factory.registerStorage( + "DeltaLake", + [](const StorageFactory::Arguments & args) + { + auto & engine_args = args.engine_args; + if (engine_args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + + auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); + + configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); + configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + + + S3::URI s3_uri(Poco::URI(configuration.url)); + + return std::make_shared( + s3_uri, + configuration.auth_settings.access_key_id, + configuration.auth_settings.secret_access_key, + args.table_id, + args.columns, + args.constraints, + args.comment, + args.getContext()); + }, + { + .supports_settings = true, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} + +} diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h new file mode 100644 index 00000000000..2aec4f815f3 --- /dev/null +++ b/src/Storages/StorageDelta.h @@ -0,0 +1,106 @@ +#pragma once + +#include "config_core.h" + +#include +#include + +#include +#include + +namespace Poco +{ +class Logger; +} + +namespace Aws::S3 +{ +class S3Client; +} + +namespace DB +{ + +// class to parse json deltalake metadata and find files needed for query in table +class DeltaLakeMetadata { +public: + DeltaLakeMetadata() = default; + +public: + void add(const String & filename, uint64_t timestamp); + void remove(const String & filename, uint64_t timestamp); + +public: + std::vector ListCurrentFiles() &&; + +private: + std::unordered_map file_update_time; +}; + +// class to get deltalake log json files and read json from them +class JsonMetadataGetter +{ +public: + JsonMetadataGetter(StorageS3::S3Configuration & configuration_, + const String & table_path_, + Poco::Logger * log_ + ); + +private: + void Init(); + + std::vector getJsonLogFiles(); + +private: + std::unique_ptr createS3ReadBuffer(const String & key); + +public: + std::vector getFiles() { return std::move(metadata).ListCurrentFiles(); } + +private: + StorageS3::S3Configuration base_configuration; + String table_path; + DeltaLakeMetadata metadata; + Poco::Logger * log; +}; + +class StorageDelta : public IStorage +{ +public: + StorageDelta( + const S3::URI & uri_, + const String & access_key_, + const String & secret_access_key_, + const StorageID & table_id_, + ColumnsDescription columns_, + const ConstraintsDescription & constraints_, + const String & comment, + ContextPtr context_); + + String getName() const override { return "DeltaLake"; } + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + +private: + void Init(); + static void updateS3Configuration(ContextPtr, StorageS3::S3Configuration &); + + +private: + String generateQueryFromKeys(std::vector && keys); + +private: + StorageS3::S3Configuration base_configuration; + std::shared_ptr s3engine; + Poco::Logger * log; + String table_path; +}; + +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index b5561243e56..c0d153a5efa 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -89,6 +89,7 @@ void registerStorageSQLite(StorageFactory & factory); #endif void registerStorageHudi(StorageFactory & factory); +void registerStorageDelta(StorageFactory & factory); void registerStorages() { @@ -174,6 +175,7 @@ void registerStorages() #endif registerStorageHudi(factory); + registerStorageDelta(factory); } } From c68257d711cbcc61150adc9c4a503eea23063a0b Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Fri, 2 Sep 2022 06:54:16 +0000 Subject: [PATCH 012/151] Implement StorageDelta --- src/Storages/StorageDelta.cpp | 137 ++++++++++++++++++++++------------ 1 file changed, 90 insertions(+), 47 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 8da5b5ce323..05b418e4208 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -56,15 +56,58 @@ JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuratio void JsonMetadataGetter::Init() { auto keys = getJsonLogFiles(); - char localbuf[100]; - + + // read data from every json log file for (const String & key : keys) { auto buf = createS3ReadBuffer(key); + String json_str; + size_t opening(0), closing(0); + char c; - while (!buf->eof()) { - buf->read(localbuf, 100); + while (buf->read(c)) { + // skip all space characters for JSON to parse correctly + if (isspace(c)) { + continue; + } + + json_str.push_back(c); + + if (c == '{') + opening++; + else if (c == '}') + closing++; + + if (opening == closing) { + + LOG_DEBUG(log, "JSON {}, {}", json_str, json_str.size()); + + JSON json(json_str); + + if (json.has("add")) { + auto path = json["add"]["path"].getString(); + auto timestamp = json["add"]["modificationTime"].getInt(); + + metadata.add(path, timestamp); + + LOG_DEBUG(log, "Path {}", path); + LOG_DEBUG(log, "Timestamp {}", timestamp); + } else if (json.has("remove")) { + auto path = json["remove"]["path"].getString(); + auto timestamp = json["remove"]["modificationTime"].getInt(); + + metadata.remove(path, timestamp); + + LOG_DEBUG(log, "Path {}", path); + LOG_DEBUG(log, "Timestamp {}", timestamp); + } + + // reset + opening = 0; + closing = 0; + json_str.clear(); + + } - LOG_DEBUG(log, "{}", String(localbuf)); } } @@ -124,57 +167,57 @@ StorageDelta::StorageDelta( const String & access_key_, const String & secret_access_key_, const StorageID & table_id_, - ColumnsDescription /*columns_*/, - const ConstraintsDescription & /*constraints_*/, - const String & /*comment*/, + ColumnsDescription columns_, + const ConstraintsDescription & constraints_, + const String & comment, ContextPtr context_) : IStorage(table_id_) , base_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) , log(&Poco::Logger::get("StorageDeltaLake (" + table_id_.table_name + ")")) , table_path(uri_.key) { - //StorageInMemoryMetadata storage_metadata; + StorageInMemoryMetadata storage_metadata; updateS3Configuration(context_, base_configuration); - Init(); - - // auto keys = getKeysFromS3(); - - // auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); - - // LOG_DEBUG(log, "New uri: {}", new_uri); - // LOG_DEBUG(log, "Table path: {}", table_path); - // auto s3_uri = S3::URI(Poco::URI(new_uri)); - - // if (columns_.empty()) - // { - // columns_ - // = StorageS3::getTableStructureFromData(String("Parquet"), s3_uri, access_key_, secret_access_key_, "", false, {}, context_); - // storage_metadata.setColumns(columns_); - // } - // else - // storage_metadata.setColumns(columns_); - - // storage_metadata.setConstraints(constraints_); - // storage_metadata.setComment(comment); - // setInMemoryMetadata(storage_metadata); - - // s3engine = std::make_shared( - // s3_uri, - // access_key_, - // secret_access_key_, - // table_id_, - // String("Parquet"), // format name - // base_configuration.rw_settings, - // columns_, - // constraints_, - // comment, - // context_, - // std::nullopt); -} - -void StorageDelta::Init() { JsonMetadataGetter getter{base_configuration, table_path, log}; + + auto keys = getter.getFiles(); + + for (const String & path : keys) { + LOG_DEBUG(log, "{}", path); + } + + auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); + + LOG_DEBUG(log, "New uri: {}", new_uri); + LOG_DEBUG(log, "Table path: {}", table_path); + auto s3_uri = S3::URI(Poco::URI(new_uri)); + + if (columns_.empty()) + { + columns_ + = StorageS3::getTableStructureFromData(String("Parquet"), s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + storage_metadata.setColumns(columns_); + } + else + storage_metadata.setColumns(columns_); + + storage_metadata.setConstraints(constraints_); + storage_metadata.setComment(comment); + setInMemoryMetadata(storage_metadata); + + s3engine = std::make_shared( + s3_uri, + access_key_, + secret_access_key_, + table_id_, + String("Parquet"), // format name + base_configuration.rw_settings, + columns_, + constraints_, + comment, + context_, + std::nullopt); } Pipe StorageDelta::read( From 4cc397da84e6bf3bd6157b7be7ce578cb02aba4c Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Fri, 2 Sep 2022 07:06:24 +0000 Subject: [PATCH 013/151] Add tests for deltalake storage --- .../test_storage_delta/__init__.py | 0 tests/integration/test_storage_delta/test.py | 118 ++++++++++++++++++ .../_delta_log/.00000000000000000000.json.crc | Bin 0 -> 32 bytes .../_delta_log/.00000000000000000001.json.crc | Bin 0 -> 40 bytes .../_delta_log/00000000000000000000.json | 9 ++ .../_delta_log/00000000000000000001.json | 13 ++ ...-831e-2ab223e7c176.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-b556-cd6ebe7630c9.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-b929-a68f54aa1e6b.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-8144-d0d9d0ff572c.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...42ca-831e-2ab223e7c176.c000.snappy.parquet | Bin 0 -> 2795 bytes ...4514-b556-cd6ebe7630c9.c000.snappy.parquet | Bin 0 -> 2795 bytes ...42de-b929-a68f54aa1e6b.c000.snappy.parquet | Bin 0 -> 2878 bytes ...493a-8144-d0d9d0ff572c.c000.snappy.parquet | Bin 0 -> 2878 bytes ...-afe4-658c02e1aeb5.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-a37c-1539c1bb57b1.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-b8e2-817f80097b3b.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-8135-23184ffdc617.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...41e5-afe4-658c02e1aeb5.c000.snappy.parquet | Bin 0 -> 2966 bytes ...47f5-a37c-1539c1bb57b1.c000.snappy.parquet | Bin 0 -> 2966 bytes ...466b-b8e2-817f80097b3b.c000.snappy.parquet | Bin 0 -> 2878 bytes ...4fe8-8135-23184ffdc617.c000.snappy.parquet | Bin 0 -> 2879 bytes ...-acd2-d2bab8e66748.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-841b-22762fcfc509.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-9527-a3dcd269f99e.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-8ca0-7594340b2c66.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...424f-acd2-d2bab8e66748.c000.snappy.parquet | Bin 0 -> 2795 bytes ...41e9-841b-22762fcfc509.c000.snappy.parquet | Bin 0 -> 2794 bytes ...463a-9527-a3dcd269f99e.c000.snappy.parquet | Bin 0 -> 2795 bytes ...45c7-8ca0-7594340b2c66.c000.snappy.parquet | Bin 0 -> 2795 bytes 30 files changed, 140 insertions(+) create mode 100644 tests/integration/test_storage_delta/__init__.py create mode 100644 tests/integration/test_storage_delta/test.py create mode 100644 tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000000.json.crc create mode 100644 tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000001.json.crc create mode 100644 tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000000.json create mode 100644 tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000001.json create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet diff --git a/tests/integration/test_storage_delta/__init__.py b/tests/integration/test_storage_delta/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py new file mode 100644 index 00000000000..a1cc6345619 --- /dev/null +++ b/tests/integration/test_storage_delta/test.py @@ -0,0 +1,118 @@ +import logging +import os +import json + +import helpers.client +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +def prepare_s3_bucket(started_cluster): + bucket_read_write_policy = { + "Version": "2012-10-17", + "Statement": [ + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetBucketLocation", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:ListBucket", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetObject", + "Resource": "arn:aws:s3:::root/*", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:PutObject", + "Resource": "arn:aws:s3:::root/*", + }, + ], + } + + minio_client = started_cluster.minio_client + minio_client.set_bucket_policy( + started_cluster.minio_bucket, json.dumps(bucket_read_write_policy) + ) + +def upload_test_table(started_cluster): + bucket = started_cluster.minio_bucket + + for address, dirs, files in os.walk(SCRIPT_DIR + "/test_table"): + address_without_prefix = address[len(SCRIPT_DIR):] + + for name in files: + started_cluster.minio_client.fput_object(bucket, os.path.join(address_without_prefix, name), os.path.join(address, name)) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "main_server", + with_minio=True + ) + + logging.info("Starting cluster...") + cluster.start() + + prepare_s3_bucket(cluster) + logging.info("S3 bucket created") + + upload_test_table(cluster) + logging.info("Test table uploaded") + + yield cluster + + finally: + cluster.shutdown() + +def run_query(instance, query, stdin=None, settings=None): + # type: (ClickHouseInstance, str, object, dict) -> str + + logging.info("Running query '{}'...".format(query)) + result = instance.query(query, stdin=stdin, settings=settings) + logging.info("Query finished") + + return result + + +def test_create_query(started_cluster): + instance = started_cluster.instances["main_server"] + bucket = started_cluster.minio_bucket + + create_query = f"""CREATE TABLE deltalake ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" + + run_query(instance, create_query) + +def test_select_query(started_cluster): + instance = started_cluster.instances["main_server"] + bucket = started_cluster.minio_bucket + columns = ['begin_lat', + 'begin_lon', 'driver', 'end_lat', 'end_lon', + 'fare', 'rider', 'ts', 'uuid'] + + # create query in case table doesn't exist + create_query = f"""CREATE TABLE IF NOT EXISTS deltalake ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" + + run_query(instance, create_query) + + select_query = "SELECT {} FROM deltalake FORMAT TSV" + + for column_name in columns: + result = run_query(instance, select_query.format(column_name)).splitlines() + assert(len(result) > 0) diff --git a/tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000000.json.crc b/tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000000.json.crc new file mode 100644 index 0000000000000000000000000000000000000000..327e28983364280c30587f7c0432a548421b39d2 GIT binary patch literal 32 ocmYc;N@ieSU}6Y-KR?}UN#0|x{?hJ)>i?zFf&yZyPN~ZS0KQ@jbpQYW literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000001.json.crc b/tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000001.json.crc new file mode 100644 index 0000000000000000000000000000000000000000..fd48c97957675d0896f9a5abd1f7ab17ed1036e2 GIT binary patch literal 40 wcmYc;N@ieSU}Bibee&Zz8;eO*Q@6+W^~-ViO9mCsF7{}b3b9qWw(i1403d-6qW}N^ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000000.json b/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..45fd233fd48 --- /dev/null +++ b/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000000.json @@ -0,0 +1,9 @@ +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"6eae6736-e014-439d-8301-070bfa5fc358","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"begin_lat\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"begin_lon\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"driver\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"end_lat\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"end_lon\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"fare\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"partitionpath\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"rider\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"ts\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"uuid\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["partitionpath"],"configuration":{},"createdTime":1661963201495}} +{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2795,"modificationTime":1661963202988,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2966,"modificationTime":1661963203028,"dataChange":true}} +{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795,"modificationTime":1661963203056,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2878,"modificationTime":1661963202988,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2878,"modificationTime":1661963203044,"dataChange":true}} +{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795,"modificationTime":1661963203072,"dataChange":true}} +{"commitInfo":{"timestamp":1661963203129,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"partitionpath\"]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"6","numOutputRows":"10","numOutputBytes":"17107"},"engineInfo":"Apache-Spark/3.2.2 Delta-Lake/1.1.0"}} diff --git a/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000001.json b/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000001.json new file mode 100644 index 00000000000..408d5e1ded7 --- /dev/null +++ b/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000001.json @@ -0,0 +1,13 @@ +{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2795,"modificationTime":1661964654518,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2966,"modificationTime":1661964654558,"dataChange":true}} +{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2794,"modificationTime":1661964654586,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2878,"modificationTime":1661964654518,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2879,"modificationTime":1661964654558,"dataChange":true}} +{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795,"modificationTime":1661964654582,"dataChange":true}} +{"remove":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2795}} +{"remove":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2966}} +{"remove":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2878}} +{"remove":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795}} +{"remove":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2878}} +{"remove":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795}} +{"commitInfo":{"timestamp":1661964655251,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[\"partitionpath\"]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numFiles":"6","numOutputRows":"10","numOutputBytes":"17107"},"engineInfo":"Apache-Spark/3.2.2 Delta-Lake/1.1.0"}} diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..0d07fe9805f910e56d8fd1faa94b16fa1e52f59d GIT binary patch literal 32 ocmYc;N@ieSU}8{`ik6si`R}A+<}WX0E{ka&($LSF9RBwn0IB>8uK)l5 literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..1b17a91ca75be612199b8cca4cc7bdb81484cea2 GIT binary patch literal 32 ocmYc;N@ieSU}E^@y?B#jm+++B2@)xGM;lKa($LSF9RBwn0J^6QasU7T literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..f7f1df8479de818fab8d3ab6f8bc07dacc4763f3 GIT binary patch literal 32 qcmV+*0N?*(a$^7h00IE(#)#e7^VS5TQ*7%Xbga<4Zrk`Q1Y?EWUaC*81LFTU|% zbf2)hM;o=buYN$oMsW(id*$*wM})AE(+*n%I4kYit5!Rm%@{7{`&a(FEeXL)`dZ_Y z<IR_e|`J)J)aQv z=t(ETFl5-SRrVzKW&itIvJlG>eC*EsJIfM6K=~qm8UMZf`Q?xhrhnm39s|mJzL+T& zGuiZ5wp2;y%zQCDQOuXpf)cEj&kR|0rNqGr(f^klTNV76a5+bN&38YVu9 zaOAkGhD+9bt$DU?cK5DZO@yQ14?4GX9G#9)9=auDH7k6Bv!dbD=dYS=%h?&Q>&!O# z!6|n;j?3Tc)he!s$XZNWOTw#(Xn6$%;ZnSmuuNoficBY!Dc*D(5j5(xBDX>jq1oZc z2$7qI@CaE8;wMmYc2Jfb}XzRAA653T#n=h6SwJC!DdvqL8fkCS%(5C`x3Ki(XP-6mi!+7cUeZbD%EJu`m&?>TDqwHHY`wp

X1q{ElCQ)@NVfT3sgEK*MlM0C#Ab(me}dsbI(w_ej5R@bbU zT~qzRznJL0zF;LL6P<3mSMDZKiE7QNS2~Hw7Zwvuvq4?nJOQs8U1{}7bqm(bUcGKo zI0^f$RDw|_CKs2MQvarCH6NV98KitaBwTCG51)cNbveO7&tMb$pA*p8{^0WV=YVfN z=<&N9zPH{wNa5(mhVLCt_oXGIo^o3*-Dt<%9*t-84D@JxwA8EBEAc|XGRDdiV@1Q5 s7`Mv#Y}qJR&GE94D;2D&VZi^#Y_?p?ozT|c>-k?l^j-*|8Tgy~54oVi`2YX_ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..a9652efacb0111d1ad3b72e982d3f5cafc7594b4 GIT binary patch literal 2795 zcmbtWU1%It6uvY2H@jVIt+|&OSa4a|C6(PFyR)0khTz|W)KWr_5=vF-?Cxx`xIgLs zC?y0W6a2hlRBPPB6VCNU*fUvN+bM*7+J1p$yD_`#T4+y*c?Xlm!{&?nT5KZDI@cr+;`O7DS zJ$Bm3Fbo;CYn6Qoe%1fswk*W51RuZq;O?@75Kz8^U%`Jbe{n4&gz2{&$`e4D&lfZ0 zVkVm&%a$tXoS84CCyMz}dfc$`#X@ePFjgJc9+fp!VFec;mNkNq<7R~hH-myZ!i%e^ z!a@j9SXqIALD9|132p{OH!CB!85G=MuXLKqu@IoJn!$bs1^0=e>ZA@8=F-N9f~TQ; zNQ;qK53Ni1f@|E9JnNS1!M@LfyFPs0jot8j)|CKWk*FDTK06)lEST+AFLuiFR>Q={ z5ssXc)o{t0t2Hmy&F=nntBG(F{6XioPNK6h%0st=tY(F8a8@*&`utV1Z81WA>IO+3#ZQLK{H2)-d9axXySq(a^e zj=+3NscNe|r<)5VOd;JKNIHCJJGE9*4H$Y>&m#5IKt$(KU5D9qsb_WdcI$P$VRg-l z*)`Q4{fmk2d5^N>poBz0yfczPOlZnhomm<_UP+=t`?ss#~yb_Ud($ z!b#X~r4o!fF}b+3l=?SCtNHL0&LHLcA>mqcZuk`3smlotdj^}}|D1r%_J^0ZKL>pK zVUOSK@V)icVG2h-Hhk}Jx-Tyw^^Dtc=|(&5_Gmn#XP`&pqorQ0UWpe9mN8bI7%Lja t#JE+?XUj&pYL1tUT&ZAH4Fmo^X0zpD?v%C$U(f&gq4z=v&A{K>e*k^vziI#g literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..fad00b6c557f37bf65d19b8958bc6d1564f89044 GIT binary patch literal 2878 zcmbtWU1%It6uvV%yP4f4@rT@DMi+@ox+bu*Y<4!gn;5%kz#`IzQh)FVTW04cE^cd;#Lj-!*~#Xym;$IU=p*KL#RI$0%W~X zc-^ST$x#Im@5cl9^xtPs_7g;|bpbxyx&ZRol$Sm^e=gL4@+)KU=!=j2mRnohzfik$ zh2HS(<7>aO9o(Cbe)q&jx3+U{zTe!Kkba@=U;lajaxTo=J^mGg)m+Hnee5PU&e=R!8$`IhohYw60}!Euj-X!_ZRWrkR(M z86_*Hv~)sN6KPd8Rl`t{hLTOFN$CMr5_wv9SUR<5B>|z9nHCz%POV*Y$StoVdN$Z~ z;c2}9ftJu~ru6|cEt%I$>jGw{G%k41sfHxFYi~TwGQ>NHcH@V-$)_X^b(2n;H7xM> z5Tp-DVZn4(D=eP257$}ewBd7>%bc@p_#NLZzjJyqfUhwG0(hEF^;YN9%4>(K+N;K_ ziXTKcw3`)!d1IzfI$Tt1ckjz3gnPjrXjN-B+8-u3B&5eE>25%4N8E{xGu4XGiu7A~ zrhyz7*;=(_;_CH+Zd(LS3QMa|K8!=s7>@!_6z58q6*%D~;ZT&HbUV#$KLHIp{pX%S z65&~ZWEfN%eRx1<@^}x5n(YMTl0#qu0qs?hrwvR9+E2X%n}lG~B3OZm(hCwekYH!c zf?s5WWhT19MlUd7Z@+ujK;(G|UJH_9*4$%>B+e3byAKP(S`hC;(RJXYX`^LvG_A8B ziasCFc~eAG`>9vdiGCp14w9@jV36NU;7MA;2L$v1iadRlgrNP@OVA($%@842=3D}+ zd7SGnbAV3YugKF!P0-sPHLyE#Vy|+VDrOb79X2@GODE9SxrI z4%OHoY-_vlG>UEk8)r8$^tz*D%MVKEBoTQ!UWk(RQ?HUOOUW*hr;CA>67oQ-=VPJ& z#0b|IuuZ1Xa90CI>}q;>?{MrsEDCGA_%wrrzj_2N$_uCX0T@*=Q7l(x;_93V10`#7FB-5f)r&=yEGHn|h(#!LWMY0{A@*;G za_Qb7S~Jm&hYJ@ao9-uD3PBmBn|3?8C)AvqqO?EeYr}Ar6H?~;bN#e+N!;S5Z z_mu@CK5e(0JX;yGdvtI#J_20PlL>ZvX%Q literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..1e79eb2d238771c7dbcda8b6dc5308d818038e67 GIT binary patch literal 2878 zcmbtWU1%It6uvt=KEC+P z>dT*P|2Sqai%@SW2*_%s z@S0YMC5I(IydU@BbAMl0>LrM*c>peC9sqeQ$}69&tp?jreyPo$eE!j2b1TaS=W183 z(G$M)_R4Qe8~5fT-#`Art(&PZAZP^F;mPS2-jKW}ukVSFro=h`U`LU-Q#_6PDGWTD7Zeb zG1E_HM&#FBe#Q+3?xEJ3f2&vLA1H2n6snpX#p8$dy-*SCv<^ zX&FC&aBv?Z1oGNcp>(_`*Y5r8>`C|&5ZOJ zd8UCZ7@1ntG_m!1LA5MAD~8172p7UZam0@}D2ja~%%TK?f!Fi zA&GD;Kr#%fjUL>`H#xi?Mf6sJa>*vpfq?d^z|jUK1g*Pnf^9;uZ4fNNMCk?zEJ(1j zX28#T_ytd7k%?ULgxvk^UIUTqCb%s~j2d%~A(B`_)a@S3^D6=TB#Nv9CrukIi>+y$ z0a5h&2=<*KqFQ&|qE7b$!OZ~4S_208-2|?rHM~GTFQCBDTS*97cijXHLeLBnf(6zg zFq+3c{sIf=^!^GQz0?G~b=Qsl1wr37&>Qe<(VCe3Xu!Y8zw3!Ke330SlJp>jkZ-R>Na2&W20!G%?wu0j!p{TCV4d8hn@P#iC4>V-T-J!xTC^Haj;L z{WnCpbng)6OmyPmz=hIO=SQ#xr~|ND4F3$zt(J>TSNg3U?H`U0Lyz_k=Ie!`+Mmg2QbNflMkOhm)|8Q?A}P9@R-{xu eqv?_aKSq;DWi&M;F2h6kUw^33h0r1R<@^s%AmzsZ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..ded74b8da6d88f7d8d20b2ac33a21855f6d673d4 GIT binary patch literal 32 ocmYc;N@ieSU}7i?k3TYPRdRK{e@9!*kEv_aA3vL{@}lK70K+;Cvj6}9 literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..e0123ca128f2bc4783823c8f5b0c4f1c0af8cf24 GIT binary patch literal 32 ocmYc;N@ieSU}AXu&+L9ypMxH6vLf$lJ>?4Z$Im9KylA-%0I>)RmjD0& literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..065f09eab7a820723d2a81886e88180dbe0e3b5f GIT binary patch literal 32 ocmYc;N@ieSU}A6)6~1(D_9OEblk0^KX)J3xp>Xw_!r`Om0lUc#EC2ui literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..8688bc7218dab15bb560ebf9a764482f98a9c33d GIT binary patch literal 32 ocmYc;N@ieSU}C8HX3}>qXQo=B?MI`lqMw>IuISI4!18eq0Kb9_ivR!s literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..83d3695feb61efb287689fdfc8bcffe140eed604 GIT binary patch literal 2966 zcmbtWZ)h837=LrUrkB{R)%86u5=ISO)4 zd++l;&+qxY&+mCIy$7ETV1&DHH~x6`_Uaxtw!^C(a|rD#FbJWVy2dM?+&cdEM%>N3 z(fsU}bMg6wKgPJ!=kbc$*81#g&TZTVH#U0O?*A=*{PFe&7izD@zqs`NUp0GINznOkRWmsZpP=9ZmyLQV#^6}?s7t?hC+XL=H2Zw zzPfk)z)I%#__51>K2;c4jL*D%D*4m$8qN5Fi$A|rw@t=ltC4~8!5`w+Z6}f5emPz^ zn`ORva*4W~`)K5AW*fJLW3M)@h*J0L+2>9=u6K8zUO#x?=%t6cf#?HxA3l9$?j5c| zw88HHqPbLJcRb%*|MBK_34eBRdhL`Aq15w-e?);b78l>pkNSg`QX(p33GXW9%;17|VIq zSl*~L&d%74-N`yU&5#g)bK%kY{D6`>d?>HIr1vZMK7^h78Ns3IeW}c$v{Lx@dp3h` zJ=pE#YVAi)xJex1Qm1DUrbGFS949t5spRysXKm@8sUr(UrjRe2*kUo2s91Qr%k7%* za&GK&g*jx0q8KS*hPU%C^BFJKWh%{-ACKA${h7OPiO@bk86keO5jXQI4*W3k4pswX zGX}sQ08p(892GDDsNS^#j1z!yCjpp*(NYf-SfEh3wNm&I8$V_Pt9RArw(8xw1|Ul( zSS`ra0b_^-Hb99y5I@z3d4AD}A3@$tz@*nk%VKcaWPlWHAHnD;LaKV#O6n^D@ZLRy z)N;+80G3Nzt^okrfC5Lmk^ofiS^<^`z)A^Vo;3kV;xW>nXCXT6UxA}dO`=!tTBC1~ z=;QSy{TsDrP0D&Kg+IZM+Q8=hwAO7I0hT3zn4`h6zfCnpn@Bb4aS!s&*xi_|h#0hP zFqyFf6FNx*W4sV1)w@}Yyf zbC_9XRr)-G-(81fi9RIze8AeryAY^lWK4n0Gf#NI879V{6^Qmk`sFi#{Ul0lH zRVVUC(mohNM~ZsEm&+dY_3H&Cp%fHhi5>D3UK!9mot}IlSJVog4o@K*sdl!AqX~g}mvT&hF;o$all^-5KtS!$^3nA~~@?*HY1F zOTZTZjkdI@#Z)@cBFnnu*P{NIBt;{-77l8XmQ*5|6jEh9DM|3BHW<`mp?22=URDms&hSI3|!i1 zKJaZK6nUcg;G1)airL2M)wkSRm<`{&+1LKiZ;98R?s(*S?Rw(#xr=|)>|Rb_7JG3k zzA$!s#o2=y%p$Za;s&H-K0U1G#c)Uh&rP@mpIf-MvWs|T%K>rTazLntLVt7S!_tLK z5wGuF-M5teJ#p;Dt%JtUd}8Lklc}E;S7^i^U-|jnx-B9;y&N675dI-?$8iGb9XArj z*&OrLvkTPh+@-NEnJvs3j=j;iBuc$k&%SWNb*H!Y)aw3yN9P{vg|HvSd+@36uf5Mz z5^V^&Aklm}xwAaqS^e?eRsnx@WqReL1EKVbhkqr{>(~CiO`cGmzZ`gs!Eb+k`IiP; z$j3LHFPE?l!q8K)uE}ah)YNE9j7VWwRO7+8nA9RcNtQK5l4Ina8j0(2G^UDCRnx?X zt|+1!RFh&#*K|1^Pj!V;K}Y|YV5BRiXnIoY3dKWWL<{L+EEq|NU9y@|lByaC1!Ye& z;}JMYAUINPVaCHFR8~`>fSPj0rgmF2@d&mT+ZB%X9XwDf*wwVZpf+19cOHEpTQ z+GTdnBUoqK1&($D?9GN)?%TFiC!uXS-Kn5@c@Afxe8}VFht1O@gZnDNHO7=K)|kgx z);!*=HSf+iP3dGEo@R&(z`1;&zA&WZ4<9OMujqpceh^{zUPf@K`an8+D5Dtve$QnQ zt_Qud+^xOn887ieOzQM((lRK&k?Taqrj)#1wyZ7HnHsX7WQ;;t#TJX{WJSX}Jzmdw zKkO~Hr^|^rXo?vUW_Tz6DxdXpJr>g}`thjUcHdE!F@49YBGjQ%L}-rFMWt0PeZ8CHKeqaR=!9pVoS9E5LRH5NkEq#)L_>*P@cAI@A_GzypOW_}hM#8qlXX2&)V{5SBW!DX z*<+Y)!U8|vfTtXY|HQ>R{T%TiQk`#;TT31+;w3HInq=+i(DM^Aqwc)T9k92s>!-w#{ NekfgsP(S=5{|8dZ|6Tw9 literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..0fe5f31b711527987d2a509cce139e6711fdc643 GIT binary patch literal 2878 zcmbtW|7#m%7=Q0_Nv~PkS?PP+Nstn{qs1g7xin2nF>{E_DOkjPLF>vTcd5ZN>GDO9 zQkjL7{sF@Y4vNqpg0%Z#Klo)Zs2#G==?{ujr&MKBM$@9QNbd$mJUU^I7)7>AT6NKOY$U za^-f?JiR^t$2W(Y9OqKcc|Vx@Df#p5dwaXb*OCC}!@YR%{QY~LQOsfvp{|%0khNm= z4ZRqQMifAN1oz-8e|>bRiy(6K0C-cxHPzxcUh_{@eb}b$1hV^!|5ipLUNO{pE!Y_WShdqY&O> z_;Gw;CsG3O(>DzU@W012}O-XGehK@o`|a%H9j1Sr54C|S2G?IwvaU~*kvy#Zuio=pN0cIrup}Lt?7|gWj4zt&;uOvDi97lLs zI6$y!=OCa30yC|d)0-9s%vNn|?{=#kl4!4;{xrSdPc!Z?_qCEwN$hJSohGbL;PE)5 z4@o{j!)MLa0*jBC%XQY;t-7q`JZCMdZtHB#ZSCIj;4Owg0MGLAj?%PRe0`#neodcJ z@uLWPkFa8UN}tT;Cvs}}-_M0S!X04u)T?y_J?|qp#HCHoYqmrEi5w?3E~6Ipx~JR7 zGj-&^$d*fW6IZEZHPa$+l25uUL-%^6K^}RaC{{?A6*%FHFfQ|>cBk3xC!l_-|Lk2z zBHRm*43lc36ZZ(qJbn_%S|dRrZxFmJoRu{opuH;cw1Ejhe&fEl3I&gO4SWI78Imomdd6?f4lauLCDd8!d~a zX`Ka8^!kX_og$(dcb%dxb^*b!?Idee*MkI(q*Yx&Krf)k(_2Xh8h4!p9}~sCM8h3=;O+1Wx9KCB6qnRX`LA#mSI5t-?SF z8M7q>@9k2ykQd!bC>n|)aiHNu>0B-ZW9VE(FNcbSH$zi;S=H3CD&FB{{N=Z%b^nOJ zR4!K1Wq-h*$?7?+@|C|Hxi0KkM{PbD%qUYJ2a##;dCNA44*`CJv|srD{1%`N-MF{ eke*Q#_!^By)5Ea==`y?s|LcdcZ3vCQm-9cOPvge` literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..b43afdfcd30b48121adaf093c9cb4a089ca3cdfe GIT binary patch literal 2879 zcmbtWZ)h837=Q0_Nv~PgS?PP+Nstn{&|xklcWIiIYUV!3{D(#69~4(E$z2VmNtZv0 zlu8*oO1}sqo8X`bLnL(lun%(zldT&JYCp-)PML~qj?$oGl_C1Y=e@i4?pz|I69Ref zectEydw$RJywBalvY%&O?z?pQ`n6wZ zgr~pRy3;Pg=7-GsQtF3He0FnGzHp8n|NZXPd4C&p^zp{(7kv}^Z$8(qcBjq?39|O-OyLB=WT4u%u0ZSxG?1GSdo!nHJq+4!ZS~M9+ig2u}+K z2u|%h1hhb4rZw|=)53t+sg3L1?UX|j-L==BrWgEa#y#f2PVy;Fg| z4(?^e?yNplC`}f%+P|O6C4_sx9xmpEHqOv0y5;FN^GqE% zFtW9(W#a1fLf*CroD`ByMxc9x(r^IrP!uO5%nF?Fif}l>kGq}bwx588oc?onA&GD= zKr&3Kjor9kSmE(wC}Ok{luHhQ0R*&HMV>Y=A!yz85;O@x(LZcf$fgPHj)9pM-gY4{_*av{$M@GODE9SxrSHr3cL ztFa3oK#?YSOV~{ez3wP!`aubuBqC483sKU#>s9ibsbm+)(^|k%LN17TeN6Np8Q}&4 zw(%4i>S*AI9ZfH94Das2qR{BUD-06;>=L*LFMP`P!>EdiV!1LE(`GanC^2)k#NfSM zEtE^5UyUg-1&RAwPE?N+V=#t})b(1dQhqBot=F`?R@205+-$h^_KY4L3s-BEdaf3Z zhK+(=%vZx>ug-=`+B7jaqXDe;>3q4KE$Z-GsuznIS&o6f9t~6I@Yw9!T=ZUua_Rmd zteNQgL%@a7ROcbs1JniBsRk3^|0p11`u^$7$?m%ERDP}M`WDMONgR1^xW1k7zBY%% zr|g!Krz-*d>A454cnH|v$RfVrn YMahjO_em$=MfhJoROmuz0zRGp0nPZ}-~a#s literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..6756a13f94b6124c35aba9fbc36430f94f6cb589 GIT binary patch literal 32 ocmYc;N@ieSU}7+nO`pM%)grPo=iZb5l`+DHH1zW(hyT3?0Hk^hod5s; literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..c099f0af8e41171ba95d8d9b461fa5a50bcf7024 GIT binary patch literal 32 ocmYc;N@ieSU}E5ZXt}{4&gbbjUR8ekdm`T#+%%E&Sa!ex0I07E<^TWy literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..73d821f134ba262e5b03a94466fc5b4e2ec13a26 GIT binary patch literal 32 ocmYc;N@ieSU}8{w^|hQKl+|V4-?W?3rjjv-H1zW(hyT3?0IAjtI{*Lx literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..48b7b139c7afe8ebc53035ed7573f8b0aac32497 GIT binary patch literal 32 ocmYc;N@ieSU}CW0e|F?5&uf>E&bnik%6)qeY3S!o4*z=(0Jf(MQ~&?~ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..4fde2de24ac6e65d2ba8eaec23e06f3e3c4b33a4 GIT binary patch literal 2795 zcmbtWU1%d!6uvV_+9Vq*YVTzRf`qUex{?e{CQX`B{6nOy%d&`+Wp`zFGBc^c{B-l< z52X*TK8OpIMIUM@vSLL9Q9(fvk?u={r3yZ%*wsbQhf+|A5B2Avc<#*Hxpy}a(iQS> z=KOr$Ip>~xZcaY*coriZ#|eDtv)!xVIF{ja2!|0G$;$|#fV433@jK6b`|AUq+s1o) zyTLwgUp-b=-YD)+w=tZ?m*2Yb>Yaky=y4Y<3Yb=_`i$Mmgb+ZU#pm(6pZ>NZ)}ejkLLLUlQn_Mhv&G3w z$+8QXe7-oIDHU`pW1AC&iOE8_m^V%JE?HH=tl@%)WtAZ0npva4%tXN)<;_(U;X`l< zv$g^P6Gby?CzzQinpqpc%tXN)@mr@VToxR{tY;9Q>eMpUy zSzlU{@M%xEFZtR{*%wy=zPKI0r##>Fps!sD;U$Txf#?01SbN@RJ$P~IJiqW#&$dJ=#78nGh z+-bX}V7FVfJPVQ4xVoBxdmU9L;5gw>{FJavWO9t0NQI~Q&~b^NF@F@f7Mcjd4kaT* zHV)xYvO0wCN2%Fff@Z@fm<0kht6ZW4y;_lAixO-(1Zz?p$Gj4}t~v0tpCG4F+W^n6ydVNWU%g1NK`E9a_+gY((rnFX#=HSrNOKmF1}|;9+H5ExUCU`Xq#W(>XkVylu)8jF?T*%JKC9L3j$s)c zL-`@NknB7^Zzrdc?M|y}c9Q92rE1r#c5>>>Lb74hsmb3bVD-GMX183k;k(hT)eI_5 zLA;$#GU(*g!s253R)}Wf_95Iw%Htv7YGZEj6uhm=0S;P&1Mq(;pu7F{>7D0*haa^3 zPMgQpng?kdeQbE_V7yN*BITGja+!K7;mv3wt7TzE6JzCWwPq!XMOz;?C&x>=K3Raj pNjXzDD@MW8^W~yl(RKL$n9G@^{84okUeEvfp*M#SIthPs{{_TK!qNZ$ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d936d4315277401fb42f95e78be35aedfcdb7ba0 GIT binary patch literal 2794 zcmbtWU1%It6uvY2liBTBQFAXdu*kBsLn^yNc4xDj4MDAlgc?ebAf>Ievoo`CaevbN zQ6%KS_##>=RS2;ZsqsNXi7$dsBv2>{0Smrpv|0*6kf4&M_(L8%cV_O)Z8yTE!akfi zKi_xGx#ym{CyqXs!w4sF3g5YQyO+cwd_}Q@(7}R;5DE#aM>jV zeVtM7eE;pu&>n6m`smco&z~{3tKVI}F74s=&%5Vp54`d8lR&EC$M9cYfB83$_Q;f* zVGJ^C*Q~n|+`hH;XM7LHw_f|@_M(6gK%T+p@JC<#zQxy}eeOaY0?2ZuYUgsL@od?$ zi`hbNS)~?Xf+*zW8}Cx0|abV1Bnv!Yk7hVO0eM&tO!Y*@JjHy=D@4g;4M7QZwD|TOGEe>r0xO_YZW^Rzm8oX@~=!F(L&fo3bZ`IVgd+$8ls(A z7w;wD8nrG00lRd8L<#z}Ji#(0xGYhEWmsyg0%u_S^p`_`&hBg=QS^Q-kG@LLHyrd$ zKxgOR&Y%N7Lr$uj03Td=P6UEI^*qH5O0g8fPa$;;o|mw9)G*Y3`fFiGpRJ5QqUrZ* zdF&sZ^kcL+7a~FW&OLF^-xSD20c?|l?Ulj2Ag_V8)L>|QJ=+N^lchLb6%hF&LPS*} z??i`Tzhz{()t=RiIRmzk<}4%)UfNEr)s!Q;me=w~KGNsWIbYXccb)IqU9H`EMQhky z!!o*t{6ly?-FUmqOUZrlsce7Wo z8&sTtcsrA3(CLZ!g@w$$5Uu9@L%54Hh=+u0&Dp_I@V0IMu-_USfd5kg-R<{J?>q-V z_Gt=%(Kr_2GmIsKo|zC4LIGj@e)sto_O@Sg-R|7j zSdZ-EcJIN(-5ZDQQMYlN#vfn*;_4yJZS1sz76r^otM-oB%H(sJi}}x2cXvaaGox;= z&s|1WcNz7={^x&3_i_96_-yIZ>(`ju#+A!o#`bahnS{{&?cB>in!?BN!&^T*3UF?R zPdXV!Aj4Lz@>qf&{J7o^bG|IW{BN<}-xm-9$n*Fu{MSdAzjYvw0Hj_t3{%sx znOvco%S@PsY^GSq8kv$_F6OfZvt*PD%Cn*(ORV9Xh((1U5bg?84LlnsW9>z~_4b9fanW4R@pA}A zkBV}nY|hsj7wWq8_;<5`a188W=d_NZ({YMJLq^O-#Sd`4s5`XA39$mXxtk`UkgoyVTY0t zA~#3z80m%aizqcSNYHF}1T#RuX4RJ{!Jw8W=u?8eO|T}!am+2j?V1fgL*`Oz!ZbhX zybK)NCEzVQ&x27+$jS(Q4W;&ghqa0wg;&QO5P4U|muMmEBKcaL;KLXY+#R8v+7h26 z;2O0h0s*^pzC;NIwLHNpCFn_%U=@}ctH2%@Fa6a3ptC#cOB8)j%cFlo(fc;~4xqDh zaAweke}l}Xb^zYL@|*|+`|5d$o0MWDf?q|c&0rh{-J^z~_R`-BLi%iF_!3QjP|IWg zVy7RW&FR83!_v3!iH&|mAYB1$)5GnR;k+QPp|;d;XnZ|?#;{CQqWEh8kv~F2OiARE z@EGj3v@ADU^Qyk6!xmEQg`~nu+paYma!6D2Y97hQ20YrA>MHE6OC8fvTg`XX1=G?i zx~0q8!KI}2-lCbDO17<5$FP#=WVL42EA8aexus-7U!W#$pMcf%t~5L4x(VNnPQ9*E zaSGzibdo_Qr_?!C;wJcQ7OZApeCv&|;UqP7oGN-0HTSyrV^CX;P2n{1Pb zBFnz0tk8$8g2=L6)Y5f*a1kE_K@|3>zU+%2r43R_@nILCiXc`WR?nT8J9FEtkgbr1 zGw0{~&N=tob94UW3k8g@ic|Q3JFV}{(bUP@$R#p+s*5n z8}WVI?%lt*d;Rb|>NbkA`2B02Tsh3SjhuDS5`bCjG+wtlxne>0F#q}d&TfQrX4Kt{ znI&{(mr+0HfBa`+AGf>5rz;;`yUN@)FE4$T+{f()5P~Ipy3w)R$cWQt78nMj zXm?#xxYujcJPQ#ORawhODo!XTqDX>6@l(Pg5y=^HE+b6_Ll;N{jryY)XrYNP>`*d7 zX0U{K2w^eI8#Ay^kwob*cYy5_)7k(tc8Fv*WP zF9Qd633v<7bN>(~WF>~5N0~j~VXb0E;n%STME;ctBw7f&NP(6ocqa)2cVe_t+v1}H zT%)!{AYhj+kSM{RmM2)H1Zxr{ScRp=DsTqIPk%K8=^5|bu^uB|>1L*7= z+!=J>UnJ9+9e@w6JSPIdzIvYG2Blbu<5MWJ6;^TBJ8BqeKmDyRq|a7HAkp*(wLJDu zPWo}$oMm`sSo+RAanLUdWLW^)0tpMcf#uC;sBrUl=vUbAUX zaRTD4Y??u*Cl(eLvyVcwTaORnF47<#5^l6+hfl%Vx&gpJYj6PmPX%}nNshf49Wa?wpajUND@c*$`G%I7rlr?xg|Lcc7iXn6!{^tG*9k0SS literal 0 HcmV?d00001 From 616b72d665540e4748fa4b45f3321e61121d4456 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Fri, 2 Sep 2022 07:11:47 +0000 Subject: [PATCH 014/151] Fix bug --- src/Storages/StorageDelta.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 05b418e4208..52f955dee0a 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -93,7 +93,7 @@ void JsonMetadataGetter::Init() { LOG_DEBUG(log, "Timestamp {}", timestamp); } else if (json.has("remove")) { auto path = json["remove"]["path"].getString(); - auto timestamp = json["remove"]["modificationTime"].getInt(); + auto timestamp = json["remove"]["deletionTimestamp"].getInt(); metadata.remove(path, timestamp); From a77aa6b2b0bfaf80fc8078bc29789c091cc98a8f Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 6 Sep 2022 15:01:34 +0000 Subject: [PATCH 015/151] Remove unneeded comments, format --- src/Storages/StorageDelta.cpp | 83 ++++++++++++++++++++--------------- src/Storages/StorageDelta.h | 12 +++-- 2 files changed, 52 insertions(+), 43 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 52f955dee0a..5fa60206360 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -2,9 +2,9 @@ #include #include -#include #include #include +#include #include #include @@ -24,49 +24,52 @@ namespace ErrorCodes extern const int S3_ERROR; } -void DeltaLakeMetadata::add(const String & key, uint64_t timestamp) { +void DeltaLakeMetadata::add(const String & key, uint64_t timestamp) +{ file_update_time[key] = timestamp; } -void DeltaLakeMetadata::remove(const String & key, uint64_t /*timestamp */) { +void DeltaLakeMetadata::remove(const String & key, uint64_t /*timestamp */) +{ file_update_time.erase(key); } -std::vector DeltaLakeMetadata::ListCurrentFiles() && { +std::vector DeltaLakeMetadata::ListCurrentFiles() && +{ std::vector keys; keys.reserve(file_update_time.size()); - for (auto && [k, _] : file_update_time) { + for (auto && [k, _] : file_update_time) + { keys.push_back(k); } return keys; } -JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, - const String & table_path_, - Poco::Logger * log_) : - base_configuration(configuration_) - , table_path(table_path_) - , metadata() - , log(log_) - { - Init(); - } +JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, Poco::Logger * log_) + : base_configuration(configuration_), table_path(table_path_), metadata(), log(log_) +{ + Init(); +} -void JsonMetadataGetter::Init() { +void JsonMetadataGetter::Init() +{ auto keys = getJsonLogFiles(); - + // read data from every json log file - for (const String & key : keys) { + for (const String & key : keys) + { auto buf = createS3ReadBuffer(key); String json_str; size_t opening(0), closing(0); char c; - while (buf->read(c)) { + while (buf->read(c)) + { // skip all space characters for JSON to parse correctly - if (isspace(c)) { + if (isspace(c)) + { continue; } @@ -77,13 +80,14 @@ void JsonMetadataGetter::Init() { else if (c == '}') closing++; - if (opening == closing) { - + if (opening == closing) + { LOG_DEBUG(log, "JSON {}, {}", json_str, json_str.size()); JSON json(json_str); - if (json.has("add")) { + if (json.has("add")) + { auto path = json["add"]["path"].getString(); auto timestamp = json["add"]["modificationTime"].getInt(); @@ -91,7 +95,9 @@ void JsonMetadataGetter::Init() { LOG_DEBUG(log, "Path {}", path); LOG_DEBUG(log, "Timestamp {}", timestamp); - } else if (json.has("remove")) { + } + else if (json.has("remove")) + { auto path = json["remove"]["path"].getString(); auto timestamp = json["remove"]["deletionTimestamp"].getInt(); @@ -100,20 +106,18 @@ void JsonMetadataGetter::Init() { LOG_DEBUG(log, "Path {}", path); LOG_DEBUG(log, "Timestamp {}", timestamp); } - + // reset opening = 0; closing = 0; json_str.clear(); - } - } } - } -std::vector JsonMetadataGetter::getJsonLogFiles() { +std::vector JsonMetadataGetter::getJsonLogFiles() +{ std::vector keys; const auto & client = base_configuration.client; @@ -143,7 +147,7 @@ std::vector JsonMetadataGetter::getJsonLogFiles() { for (const auto & obj : result_batch) { const auto & filename = obj.GetKey(); - + if (filename.substr(filename.size() - 5) == ".json") keys.push_back(filename); } @@ -155,11 +159,17 @@ std::vector JsonMetadataGetter::getJsonLogFiles() { return keys; } -std::unique_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key) { - // size_t object_size = DB::S3::getObjectSize(base_configuration.client, base_configuration.uri.bucket, key, base_configuration.uri.version_id, false); +std::unique_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key) +{ // TBD: add parallel downloads - return std::make_unique(base_configuration.client, base_configuration.uri.bucket, key, base_configuration.uri.version_id, /* max single read retries */ 10, ReadSettings{}); + return std::make_unique( + base_configuration.client, + base_configuration.uri.bucket, + key, + base_configuration.uri.version_id, + /* max single read retries */ 10, + ReadSettings{}); } StorageDelta::StorageDelta( @@ -178,12 +188,13 @@ StorageDelta::StorageDelta( { StorageInMemoryMetadata storage_metadata; updateS3Configuration(context_, base_configuration); - + JsonMetadataGetter getter{base_configuration, table_path, log}; auto keys = getter.getFiles(); - for (const String & path : keys) { + for (const String & path : keys) + { LOG_DEBUG(log, "{}", path); } @@ -200,7 +211,7 @@ StorageDelta::StorageDelta( storage_metadata.setColumns(columns_); } else - storage_metadata.setColumns(columns_); + storage_metadata.setColumns(columns_); storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index 2aec4f815f3..4e9199a5faa 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -22,7 +22,8 @@ namespace DB { // class to parse json deltalake metadata and find files needed for query in table -class DeltaLakeMetadata { +class DeltaLakeMetadata +{ public: DeltaLakeMetadata() = default; @@ -38,13 +39,10 @@ private: }; // class to get deltalake log json files and read json from them -class JsonMetadataGetter +class JsonMetadataGetter { public: - JsonMetadataGetter(StorageS3::S3Configuration & configuration_, - const String & table_path_, - Poco::Logger * log_ - ); + JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, Poco::Logger * log_); private: void Init(); @@ -91,7 +89,7 @@ public: private: void Init(); static void updateS3Configuration(ContextPtr, StorageS3::S3Configuration &); - + private: String generateQueryFromKeys(std::vector && keys); From f75376b7f8eaafd38a3c4d0ad13b875a73d1b778 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 6 Sep 2022 18:05:33 +0000 Subject: [PATCH 016/151] Apply black formatter to tests --- tests/integration/test_storage_delta/test.py | 38 +++++++++---- tests/integration/test_storage_hudi/test.py | 59 ++++++++++++++------ 2 files changed, 68 insertions(+), 29 deletions(-) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index a1cc6345619..76dab554a57 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -9,6 +9,7 @@ from helpers.test_tools import TSV SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + def prepare_s3_bucket(started_cluster): bucket_read_write_policy = { "Version": "2012-10-17", @@ -49,23 +50,26 @@ def prepare_s3_bucket(started_cluster): started_cluster.minio_bucket, json.dumps(bucket_read_write_policy) ) + def upload_test_table(started_cluster): bucket = started_cluster.minio_bucket for address, dirs, files in os.walk(SCRIPT_DIR + "/test_table"): - address_without_prefix = address[len(SCRIPT_DIR):] + address_without_prefix = address[len(SCRIPT_DIR) :] for name in files: - started_cluster.minio_client.fput_object(bucket, os.path.join(address_without_prefix, name), os.path.join(address, name)) + started_cluster.minio_client.fput_object( + bucket, + os.path.join(address_without_prefix, name), + os.path.join(address, name), + ) + @pytest.fixture(scope="module") def started_cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "main_server", - with_minio=True - ) + cluster.add_instance("main_server", with_minio=True) logging.info("Starting cluster...") cluster.start() @@ -77,10 +81,11 @@ def started_cluster(): logging.info("Test table uploaded") yield cluster - + finally: cluster.shutdown() + def run_query(instance, query, stdin=None, settings=None): # type: (ClickHouseInstance, str, object, dict) -> str @@ -99,13 +104,22 @@ def test_create_query(started_cluster): run_query(instance, create_query) + def test_select_query(started_cluster): instance = started_cluster.instances["main_server"] bucket = started_cluster.minio_bucket - columns = ['begin_lat', - 'begin_lon', 'driver', 'end_lat', 'end_lon', - 'fare', 'rider', 'ts', 'uuid'] - + columns = [ + "begin_lat", + "begin_lon", + "driver", + "end_lat", + "end_lon", + "fare", + "rider", + "ts", + "uuid", + ] + # create query in case table doesn't exist create_query = f"""CREATE TABLE IF NOT EXISTS deltalake ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" @@ -115,4 +129,4 @@ def test_select_query(started_cluster): for column_name in columns: result = run_query(instance, select_query.format(column_name)).splitlines() - assert(len(result) > 0) + assert len(result) > 0 diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index c9415e28151..549421afd89 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -9,6 +9,7 @@ from helpers.test_tools import TSV SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + def prepare_s3_bucket(started_cluster): bucket_read_write_policy = { "Version": "2012-10-17", @@ -49,23 +50,26 @@ def prepare_s3_bucket(started_cluster): started_cluster.minio_bucket, json.dumps(bucket_read_write_policy) ) + def upload_test_table(started_cluster): bucket = started_cluster.minio_bucket for address, dirs, files in os.walk(SCRIPT_DIR + "/test_table"): - address_without_prefix = address[len(SCRIPT_DIR):] + address_without_prefix = address[len(SCRIPT_DIR) :] for name in files: - started_cluster.minio_client.fput_object(bucket, os.path.join(address_without_prefix, name), os.path.join(address, name)) + started_cluster.minio_client.fput_object( + bucket, + os.path.join(address_without_prefix, name), + os.path.join(address, name), + ) + @pytest.fixture(scope="module") def started_cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "main_server", - with_minio=True - ) + cluster.add_instance("main_server", with_minio=True) logging.info("Starting cluster...") cluster.start() @@ -77,10 +81,11 @@ def started_cluster(): logging.info("Test table uploaded") yield cluster - + finally: cluster.shutdown() + def run_query(instance, query, stdin=None, settings=None): # type: (ClickHouseInstance, str, object, dict) -> str @@ -99,14 +104,28 @@ def test_create_query(started_cluster): run_query(instance, create_query) + def test_select_query(started_cluster): instance = started_cluster.instances["main_server"] bucket = started_cluster.minio_bucket - columns = ['_hoodie_commit_time', '_hoodie_commit_seqno', '_hoodie_record_key', - '_hoodie_partition_path', '_hoodie_file_name', 'begin_lat', - 'begin_lon', 'driver', 'end_lat', 'end_lon', - 'fare', 'partitionpath', 'rider', 'ts', 'uuid'] - + columns = [ + "_hoodie_commit_time", + "_hoodie_commit_seqno", + "_hoodie_record_key", + "_hoodie_partition_path", + "_hoodie_file_name", + "begin_lat", + "begin_lon", + "driver", + "end_lat", + "end_lon", + "fare", + "partitionpath", + "rider", + "ts", + "uuid", + ] + # create query in case table doesn't exist create_query = f"""CREATE TABLE IF NOT EXISTS hudi ENGINE=Hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" @@ -116,11 +135,17 @@ def test_select_query(started_cluster): for column_name in columns: result = run_query(instance, select_query.format(column_name)).splitlines() - assert(len(result) > 0) + assert len(result) > 0 - # test if all partition paths is presented in result - distinct_select_query = "SELECT DISTINCT partitionpath FROM hudi ORDER BY partitionpath FORMAT TSV" + # test if all partition paths is presented in result + distinct_select_query = ( + "SELECT DISTINCT partitionpath FROM hudi ORDER BY partitionpath FORMAT TSV" + ) result = run_query(instance, distinct_select_query) - expected = ['americas/brazil/sao_paulo', 'americas/united_states/san_francisco', 'asia/india/chennai'] + expected = [ + "americas/brazil/sao_paulo", + "americas/united_states/san_francisco", + "asia/india/chennai", + ] - assert TSV(result) == TSV(expected) \ No newline at end of file + assert TSV(result) == TSV(expected) From 07d12338ca7acfb828737c0744fb01ceeb5a6233 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 6 Sep 2022 18:12:51 +0000 Subject: [PATCH 017/151] Fix ErrorCodes style --- src/Storages/StorageDelta.cpp | 2 +- src/Storages/StorageHudi.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 5fa60206360..518c25c2dff 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -20,8 +20,8 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int S3_ERROR; + extern const int BAD_ARGUMENTS; } void DeltaLakeMetadata::add(const String & key, uint64_t timestamp) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index b4eba258d0e..f3e16d2969a 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -17,7 +17,7 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; extern const int S3_ERROR; } From ba08366af18bd89ea47e15146028b0c8f280f928 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 7 Sep 2022 07:16:32 +0000 Subject: [PATCH 018/151] Fix fasttest build --- src/Storages/StorageDelta.cpp | 7 ++++++- src/Storages/StorageDelta.h | 4 ++++ src/Storages/StorageHudi.cpp | 7 ++++++- src/Storages/StorageHudi.h | 4 ++++ 4 files changed, 20 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 518c25c2dff..9f53d0edba8 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -1,5 +1,8 @@ -#include #include + +#if USE_AWS_S3 + +#include #include #include @@ -346,3 +349,5 @@ void registerStorageDelta(StorageFactory & factory) } } + +#endif diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index 4e9199a5faa..cf9cc737526 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -2,6 +2,8 @@ #include "config_core.h" +#if USE_AWS_S3 + #include #include @@ -102,3 +104,5 @@ private: }; } + +#endif diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index f3e16d2969a..6162f2ba4dc 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -1,5 +1,8 @@ -#include #include + +#if USE_AWS_S3 + +#include #include #include @@ -280,3 +283,5 @@ void registerStorageHudi(StorageFactory & factory) } } + +#endif diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index 47dff1c2a7b..5e1aae81182 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -2,6 +2,8 @@ #include "config_core.h" +#if USE_AWS_S3 + #include #include @@ -57,3 +59,5 @@ private: }; } + +#endif From 152da1a034b79798d996b6e412f20f3bb9537d2f Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 7 Sep 2022 08:13:08 +0000 Subject: [PATCH 019/151] Fix build --- src/Storages/registerStorages.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index c0d153a5efa..c64c2e8d2cd 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -34,6 +34,8 @@ void registerStorageMeiliSearch(StorageFactory& factory); #if USE_AWS_S3 void registerStorageS3(StorageFactory & factory); void registerStorageCOS(StorageFactory & factory); +void registerStorageHudi(StorageFactory & factory); +void registerStorageDelta(StorageFactory & factory); #endif #if USE_HDFS @@ -88,8 +90,6 @@ void registerStorageFileLog(StorageFactory & factory); void registerStorageSQLite(StorageFactory & factory); #endif -void registerStorageHudi(StorageFactory & factory); -void registerStorageDelta(StorageFactory & factory); void registerStorages() { @@ -121,6 +121,8 @@ void registerStorages() #if USE_AWS_S3 registerStorageS3(factory); registerStorageCOS(factory); + registerStorageHudi(factory); + registerStorageDelta(factory); #endif #if USE_HDFS @@ -174,8 +176,6 @@ void registerStorages() registerStorageSQLite(factory); #endif - registerStorageHudi(factory); - registerStorageDelta(factory); } } From 3f9022bc7f2d326a41382c23fd4709be6bc99c27 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 7 Sep 2022 10:48:11 +0000 Subject: [PATCH 020/151] Fix createClientConfiguration arguments --- src/Storages/StorageDelta.cpp | 3 ++- src/Storages/StorageHudi.cpp | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 9f53d0edba8..c2d421c827f 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -276,7 +276,8 @@ void StorageDelta::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configurat settings.auth_settings.region, ctx->getRemoteHostFilter(), ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, - ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging); + ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, + /* for_disk_s3 = */ false); client_configuration.endpointOverride = upd.uri.endpoint; client_configuration.maxConnections = upd.rw_settings.max_connections; diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 6162f2ba4dc..9b6207338eb 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -118,7 +118,8 @@ void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configurati settings.auth_settings.region, ctx->getRemoteHostFilter(), ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, - ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging); + ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, + /* for_disk_s3 = */ false); client_configuration.endpointOverride = upd.uri.endpoint; client_configuration.maxConnections = upd.rw_settings.max_connections; From 0a1734bb69f85ae9ccc009b720567efbcdf56301 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Mon, 19 Sep 2022 15:23:07 +0000 Subject: [PATCH 021/151] Fix clang-tidy errors --- src/Storages/StorageDelta.cpp | 10 +++++----- src/Storages/StorageDelta.h | 2 +- src/Storages/StorageHudi.h | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index c2d421c827f..46db33279ab 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -27,14 +27,14 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -void DeltaLakeMetadata::add(const String & key, uint64_t timestamp) +void DeltaLakeMetadata::add(const String & filename, uint64_t timestamp) { - file_update_time[key] = timestamp; + file_update_time[filename] = timestamp; } -void DeltaLakeMetadata::remove(const String & key, uint64_t /*timestamp */) +void DeltaLakeMetadata::remove(const String & filename, uint64_t /*timestamp */) { - file_update_time.erase(key); + file_update_time.erase(filename); } std::vector DeltaLakeMetadata::ListCurrentFiles() && @@ -51,7 +51,7 @@ std::vector DeltaLakeMetadata::ListCurrentFiles() && } JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, Poco::Logger * log_) - : base_configuration(configuration_), table_path(table_path_), metadata(), log(log_) + : base_configuration(configuration_), table_path(table_path_), log(log_) { Init(); } diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index cf9cc737526..f4547666c8c 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -94,7 +94,7 @@ private: private: - String generateQueryFromKeys(std::vector && keys); + static String generateQueryFromKeys(std::vector && keys); private: StorageS3::S3Configuration base_configuration; diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index 5e1aae81182..ceb12f5f550 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -49,7 +49,7 @@ private: private: std::vector getKeysFromS3(); - std::string generateQueryFromKeys(std::vector && keys); + static std::string generateQueryFromKeys(std::vector && keys); private: StorageS3::S3Configuration base_configuration; From 3fb26aefa39eda4d1f23b5b189699ea264fa8af1 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 20 Sep 2022 10:24:47 +0000 Subject: [PATCH 022/151] Remove unneeded logging, better --- src/Storages/StorageDelta.cpp | 39 ++++++++++------------------------- src/Storages/StorageDelta.h | 12 ++--------- 2 files changed, 13 insertions(+), 38 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 46db33279ab..a79762e29db 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -18,6 +18,9 @@ #include +#include +#include + namespace DB { @@ -25,6 +28,7 @@ namespace ErrorCodes { extern const int S3_ERROR; extern const int BAD_ARGUMENTS; + extern const int INCORRECT_DATA; } void DeltaLakeMetadata::add(const String & filename, uint64_t timestamp) @@ -34,7 +38,9 @@ void DeltaLakeMetadata::add(const String & filename, uint64_t timestamp) void DeltaLakeMetadata::remove(const String & filename, uint64_t /*timestamp */) { - file_update_time.erase(filename); + bool erase = file_update_time.erase(filename); + if (!erase) + throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid table metadata, tried to remove {} before adding it", filename); } std::vector DeltaLakeMetadata::ListCurrentFiles() && @@ -95,9 +101,6 @@ void JsonMetadataGetter::Init() auto timestamp = json["add"]["modificationTime"].getInt(); metadata.add(path, timestamp); - - LOG_DEBUG(log, "Path {}", path); - LOG_DEBUG(log, "Timestamp {}", timestamp); } else if (json.has("remove")) { @@ -105,9 +108,6 @@ void JsonMetadataGetter::Init() auto timestamp = json["remove"]["deletionTimestamp"].getInt(); metadata.remove(path, timestamp); - - LOG_DEBUG(log, "Path {}", path); - LOG_DEBUG(log, "Timestamp {}", timestamp); } // reset @@ -132,7 +132,7 @@ std::vector JsonMetadataGetter::getJsonLogFiles() const auto bucket{base_configuration.uri.bucket}; request.SetBucket(bucket); - request.SetPrefix(table_path + "_delta_log"); + request.SetPrefix(std::filesystem::path(table_path) / "_delta_log"); while (!is_finished) { @@ -151,7 +151,7 @@ std::vector JsonMetadataGetter::getJsonLogFiles() { const auto & filename = obj.GetKey(); - if (filename.substr(filename.size() - 5) == ".json") + if (std::filesystem::path(filename).extension() == ".json") keys.push_back(filename); } @@ -164,7 +164,6 @@ std::vector JsonMetadataGetter::getJsonLogFiles() std::unique_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key) { - // TBD: add parallel downloads return std::make_unique( base_configuration.client, @@ -196,11 +195,6 @@ StorageDelta::StorageDelta( auto keys = getter.getFiles(); - for (const String & path : keys) - { - LOG_DEBUG(log, "{}", path); - } - auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); LOG_DEBUG(log, "New uri: {}", new_uri); @@ -225,7 +219,7 @@ StorageDelta::StorageDelta( access_key_, secret_access_key_, table_id_, - String("Parquet"), // format name + "Parquet", // format name base_configuration.rw_settings, columns_, constraints_, @@ -297,18 +291,7 @@ void StorageDelta::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configurat String StorageDelta::generateQueryFromKeys(std::vector && keys) { - String new_query; - - for (auto && key : keys) - { - if (!new_query.empty()) - { - new_query += ","; - } - new_query += key; - } - new_query = "{" + new_query + "}"; - + std::string new_query = fmt::format("{{{}}}", fmt::join(keys, ",")); return new_query; } diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index f4547666c8c..f7fa4120495 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -29,11 +29,9 @@ class DeltaLakeMetadata public: DeltaLakeMetadata() = default; -public: void add(const String & filename, uint64_t timestamp); void remove(const String & filename, uint64_t timestamp); -public: std::vector ListCurrentFiles() &&; private: @@ -46,18 +44,15 @@ class JsonMetadataGetter public: JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, Poco::Logger * log_); + std::vector getFiles() { return std::move(metadata).ListCurrentFiles(); } + private: void Init(); std::vector getJsonLogFiles(); -private: std::unique_ptr createS3ReadBuffer(const String & key); -public: - std::vector getFiles() { return std::move(metadata).ListCurrentFiles(); } - -private: StorageS3::S3Configuration base_configuration; String table_path; DeltaLakeMetadata metadata; @@ -92,11 +87,8 @@ private: void Init(); static void updateS3Configuration(ContextPtr, StorageS3::S3Configuration &); - -private: static String generateQueryFromKeys(std::vector && keys); -private: StorageS3::S3Configuration base_configuration; std::shared_ptr s3engine; Poco::Logger * log; From 745331a3f3991440d4a87554fac3c3f8b26add75 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 20 Sep 2022 11:43:20 +0000 Subject: [PATCH 023/151] Add format argument with default value to storage --- src/Storages/StorageDelta.cpp | 33 +++++++++++++++++++-------------- src/Storages/StorageDelta.h | 1 + 2 files changed, 20 insertions(+), 14 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index a79762e29db..9b36b234fc5 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -27,7 +27,7 @@ namespace DB namespace ErrorCodes { extern const int S3_ERROR; - extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_DATA; } @@ -179,6 +179,7 @@ StorageDelta::StorageDelta( const String & access_key_, const String & secret_access_key_, const StorageID & table_id_, + const String & format_name_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, @@ -204,7 +205,7 @@ StorageDelta::StorageDelta( if (columns_.empty()) { columns_ - = StorageS3::getTableStructureFromData(String("Parquet"), s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + = StorageS3::getTableStructureFromData(format_name_, s3_uri, access_key_, secret_access_key_, "", false, {}, context_); storage_metadata.setColumns(columns_); } else @@ -219,7 +220,7 @@ StorageDelta::StorageDelta( access_key_, secret_access_key_, table_id_, - "Parquet", // format name + format_name_, base_configuration.rw_settings, columns_, constraints_, @@ -303,23 +304,27 @@ void registerStorageDelta(StorageFactory & factory) [](const StorageFactory::Arguments & args) { auto & engine_args = args.engine_args; - if (engine_args.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + if (engine_args.empty() || engine_args.size() < 3) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage DeltaLake requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); - auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); + + String table_url = checkAndGetLiteralArgument(engine_args[0], "url"); + String access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + String secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + + String format = "Parquet"; + if (engine_args.size() == 4) { + format = checkAndGetLiteralArgument(engine_args[3], "format"); + } - configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); - configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); - configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); - - - S3::URI s3_uri(Poco::URI(configuration.url)); + auto s3_uri = S3::URI(Poco::URI(table_url)); return std::make_shared( s3_uri, - configuration.auth_settings.access_key_id, - configuration.auth_settings.secret_access_key, + access_key_id, + secret_access_key, args.table_id, + format, args.columns, args.constraints, args.comment, diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index f7fa4120495..010fc53546d 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -67,6 +67,7 @@ public: const String & access_key_, const String & secret_access_key_, const StorageID & table_id_, + const String & format_name_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, From 8fe4485ee8ddc049cff3775a7b9f6293a4a457a6 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 20 Sep 2022 14:16:27 +0000 Subject: [PATCH 024/151] Change JSON parsing --- src/Storages/StorageDelta.cpp | 97 ++++++++++++++++++++--------------- src/Storages/StorageDelta.h | 9 +++- 2 files changed, 64 insertions(+), 42 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 9b36b234fc5..ec0f233f9ed 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -70,51 +70,18 @@ void JsonMetadataGetter::Init() for (const String & key : keys) { auto buf = createS3ReadBuffer(key); - String json_str; - size_t opening(0), closing(0); - char c; - while (buf->read(c)) + while (!buf->eof()) { - // skip all space characters for JSON to parse correctly - if (isspace(c)) - { + String json_str = readJSONStringFromBuffer(buf); + + if (json_str.empty()) { continue; } - json_str.push_back(c); + const JSON json(json_str); - if (c == '{') - opening++; - else if (c == '}') - closing++; - - if (opening == closing) - { - LOG_DEBUG(log, "JSON {}, {}", json_str, json_str.size()); - - JSON json(json_str); - - if (json.has("add")) - { - auto path = json["add"]["path"].getString(); - auto timestamp = json["add"]["modificationTime"].getInt(); - - metadata.add(path, timestamp); - } - else if (json.has("remove")) - { - auto path = json["remove"]["path"].getString(); - auto timestamp = json["remove"]["deletionTimestamp"].getInt(); - - metadata.remove(path, timestamp); - } - - // reset - opening = 0; - closing = 0; - json_str.clear(); - } + handleJSON(json); } } } @@ -162,10 +129,10 @@ std::vector JsonMetadataGetter::getJsonLogFiles() return keys; } -std::unique_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key) +std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key) { // TBD: add parallel downloads - return std::make_unique( + return std::make_shared( base_configuration.client, base_configuration.uri.bucket, key, @@ -174,6 +141,54 @@ std::unique_ptr JsonMetadataGetter::createS3ReadBuffer(const String ReadSettings{}); } +String JsonMetadataGetter::readJSONStringFromBuffer(std::shared_ptr buf) { + String json_str; + + int32_t opening(0), closing(0); + + do { + char c; + + if (!buf->read(c)) + return json_str; + + // skip all space characters for JSON to parse correctly + if (isspace(c)) + { + continue; + } + + json_str.push_back(c); + + if (c == '{') + opening++; + else if (c == '}') + closing++; + + } while (opening != closing || opening == 0); + + LOG_DEBUG(log, "JSON {}", json_str); + + return json_str; +} + +void JsonMetadataGetter::handleJSON(const JSON & json) { + if (json.has("add")) + { + auto path = json["add"]["path"].getString(); + auto timestamp = json["add"]["modificationTime"].getInt(); + + metadata.add(path, timestamp); + } + else if (json.has("remove")) + { + auto path = json["remove"]["path"].getString(); + auto timestamp = json["remove"]["deletionTimestamp"].getInt(); + + metadata.remove(path, timestamp); + } +} + StorageDelta::StorageDelta( const S3::URI & uri_, const String & access_key_, diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index 010fc53546d..1b9ee960080 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -51,7 +51,14 @@ private: std::vector getJsonLogFiles(); - std::unique_ptr createS3ReadBuffer(const String & key); + std::shared_ptr createS3ReadBuffer(const String & key); + + /* every metadata file contains many jsons + this function reads one json from buffer + */ + String readJSONStringFromBuffer(std::shared_ptr buf); + + void handleJSON(const JSON & json); StorageS3::S3Configuration base_configuration; String table_path; From 3f7779e874ac27ab772dafdd36a923b32700db3c Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 21 Sep 2022 16:14:51 +0000 Subject: [PATCH 025/151] Rename method, change parsing method --- src/Storages/StorageDelta.cpp | 50 +++++++++-------------------------- src/Storages/StorageDelta.h | 7 +---- 2 files changed, 14 insertions(+), 43 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index ec0f233f9ed..5818e770a37 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -31,7 +32,7 @@ namespace ErrorCodes extern const int INCORRECT_DATA; } -void DeltaLakeMetadata::add(const String & filename, uint64_t timestamp) +void DeltaLakeMetadata::setLastModifiedTime(const String & filename, uint64_t timestamp) { file_update_time[filename] = timestamp; } @@ -72,12 +73,18 @@ void JsonMetadataGetter::Init() auto buf = createS3ReadBuffer(key); while (!buf->eof()) - { - String json_str = readJSONStringFromBuffer(buf); + { + // may be some invalid characters before json + char c; + while ( buf->peek(c) && c != '{') buf->ignore(); + if (buf->eof()) + break; + + String json_str; + readJSONObjectPossiblyInvalid(json_str, *buf); - if (json_str.empty()) { + if (json_str.empty()) continue; - } const JSON json(json_str); @@ -141,44 +148,13 @@ std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String ReadSettings{}); } -String JsonMetadataGetter::readJSONStringFromBuffer(std::shared_ptr buf) { - String json_str; - - int32_t opening(0), closing(0); - - do { - char c; - - if (!buf->read(c)) - return json_str; - - // skip all space characters for JSON to parse correctly - if (isspace(c)) - { - continue; - } - - json_str.push_back(c); - - if (c == '{') - opening++; - else if (c == '}') - closing++; - - } while (opening != closing || opening == 0); - - LOG_DEBUG(log, "JSON {}", json_str); - - return json_str; -} - void JsonMetadataGetter::handleJSON(const JSON & json) { if (json.has("add")) { auto path = json["add"]["path"].getString(); auto timestamp = json["add"]["modificationTime"].getInt(); - metadata.add(path, timestamp); + metadata.setLastModifiedTime(path, timestamp); } else if (json.has("remove")) { diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index 1b9ee960080..542d0b7c4c0 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -29,7 +29,7 @@ class DeltaLakeMetadata public: DeltaLakeMetadata() = default; - void add(const String & filename, uint64_t timestamp); + void setLastModifiedTime(const String & filename, uint64_t timestamp); void remove(const String & filename, uint64_t timestamp); std::vector ListCurrentFiles() &&; @@ -52,11 +52,6 @@ private: std::vector getJsonLogFiles(); std::shared_ptr createS3ReadBuffer(const String & key); - - /* every metadata file contains many jsons - this function reads one json from buffer - */ - String readJSONStringFromBuffer(std::shared_ptr buf); void handleJSON(const JSON & json); From 4d1abdc80891a468869007c19120d324c877e499 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 21 Sep 2022 16:54:02 +0000 Subject: [PATCH 026/151] Remove unneeded logging, add comment, better --- src/Storages/StorageDelta.cpp | 6 +++--- src/Storages/StorageDelta.h | 3 +-- src/Storages/StorageHudi.cpp | 7 ++----- 3 files changed, 6 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 5818e770a37..951337f88c3 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -57,8 +57,8 @@ std::vector DeltaLakeMetadata::ListCurrentFiles() && return keys; } -JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, Poco::Logger * log_) - : base_configuration(configuration_), table_path(table_path_), log(log_) +JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_) + : base_configuration(configuration_), table_path(table_path_) { Init(); } @@ -183,7 +183,7 @@ StorageDelta::StorageDelta( StorageInMemoryMetadata storage_metadata; updateS3Configuration(context_, base_configuration); - JsonMetadataGetter getter{base_configuration, table_path, log}; + JsonMetadataGetter getter{base_configuration, table_path}; auto keys = getter.getFiles(); diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index 542d0b7c4c0..fb6e19b7488 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -42,7 +42,7 @@ private: class JsonMetadataGetter { public: - JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, Poco::Logger * log_); + JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_); std::vector getFiles() { return std::move(metadata).ListCurrentFiles(); } @@ -58,7 +58,6 @@ private: StorageS3::S3Configuration base_configuration; String table_path; DeltaLakeMetadata metadata; - Poco::Logger * log; }; class StorageDelta : public IStorage diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 9b6207338eb..55f7e4b9c10 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -186,11 +186,7 @@ std::string StorageHudi::generateQueryFromKeys(std::vector && keys) keys, [](const std::string & s) { - if (s.size() >= 8) - { - return s.substr(s.size() - 8) != ".parquet"; - } - return true; + return std::filesystem::path(s).extension() != "parquet"; }); // for each partition path take only latest parquet file @@ -210,6 +206,7 @@ std::string StorageHudi::generateQueryFromKeys(std::vector && keys) path = key.substr(0, slash); } + // every filename contains metadata splitted by "_", timestamp is after last "_" uint64_t timestamp = std::stoul(key.substr(key.find_last_of("_") + 1)); auto it = latest_parquets.find(path); From d1c85f68e8983a516690dffcfe5820d9dfa2b8a4 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 26 Sep 2022 18:29:15 +0800 Subject: [PATCH 027/151] manually snapshot creation for keeper --- docs/en/operations/clickhouse-keeper.md | 8 +++++++- src/Coordination/FourLetterCommand.cpp | 8 ++++++++ src/Coordination/FourLetterCommand.h | 13 +++++++++++++ src/Coordination/KeeperDispatcher.h | 6 ++++++ src/Coordination/KeeperServer.cpp | 5 +++++ src/Coordination/KeeperServer.h | 2 ++ .../test_keeper_four_word_command/test.py | 15 +++++++++++++++ 7 files changed, 56 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 8bf64bca28f..6597e4e5be0 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -123,7 +123,7 @@ clickhouse keeper --config /etc/your_path_to_config/config.xml ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively. -The 4lw commands has a white list configuration `four_letter_word_white_list` which has default value `conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro`. +The 4lw commands has a white list configuration `four_letter_word_white_list` which has default value `conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro,csnp`. You can issue the commands to ClickHouse Keeper via telnet or nc, at the client port. @@ -306,6 +306,12 @@ Sessions with Ephemerals (1): /clickhouse/task_queue/ddl ``` +- `csnp`: Schedule a snapshot creation task. Return `"Snapshot creation scheduled."` if successfully scheduled or Fail to scheduled snapshot creation.` if failed. + +``` +Snapshot creation scheduled. +``` + ## [experimental] Migration from ZooKeeper {#migration-from-zookeeper} Seamlessly migration from ZooKeeper to ClickHouse Keeper is impossible you have to stop your ZooKeeper cluster, convert data and start ClickHouse Keeper. `clickhouse-keeper-converter` tool allows converting ZooKeeper logs and snapshots to ClickHouse Keeper snapshot. It works only with ZooKeeper > 3.4. Steps for migration: diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index c33630a913b..70009703c5a 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -136,6 +136,9 @@ void FourLetterCommandFactory::registerCommands(KeeperDispatcher & keeper_dispat FourLetterCommandPtr api_version_command = std::make_shared(keeper_dispatcher); factory.registerCommand(api_version_command); + FourLetterCommandPtr create_snapshot_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(create_snapshot_command); + factory.initializeAllowList(keeper_dispatcher); factory.setInitialize(true); } @@ -472,4 +475,9 @@ String ApiVersionCommand::run() return toString(static_cast(Coordination::current_keeper_api_version)); } +String CreateSnapshotCommand::run() +{ + return keeper_dispatcher.createSnapshot() ? "Snapshot creation scheduled." : "Fail to scheduled snapshot creation."; +} + } diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index 8a98b94b33a..25cc281d5e1 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -327,4 +327,17 @@ struct ApiVersionCommand : public IFourLetterCommand String run() override; ~ApiVersionCommand() override = default; }; + +/// Create snapshot manually +struct CreateSnapshotCommand : public IFourLetterCommand +{ + explicit CreateSnapshotCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } + + String name() override { return "csnp"; } + String run() override; + ~CreateSnapshotCommand() override = default; +}; } diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 5e2701299f4..9b52721b951 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -201,6 +201,12 @@ public: { keeper_stats.reset(); } + + /// Create snapshot manually + bool createSnapshot() + { + return server->createSnapshot(); + } }; } diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 8186ddd0c00..f03f453aada 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -903,4 +903,9 @@ Keeper4LWInfo KeeperServer::getPartiallyFilled4LWInfo() const return result; } +bool KeeperServer::createSnapshot() +{ + return raft_instance->create_snapshot(); +} + } diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 6873ef2a01e..f969e9ee063 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -130,6 +130,8 @@ public: /// Wait configuration update for action. Used by followers. /// Return true if update was successfully received. bool waitConfigurationUpdate(const ConfigUpdateAction & task); + + bool createSnapshot(); }; } diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index e8136d322d3..0995adb199c 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -634,3 +634,18 @@ def test_cmd_wchp(started_cluster): assert "/test_4lw_normal_node_1" in list_data finally: destroy_zk_client(zk) + + +def test_cmd_csnp(started_cluster): + zk = None + try: + wait_nodes() + clear_znodes() + reset_node_stats() + + zk = get_fake_zk(node1.name, timeout=30.0) + + data = send_4lw_cmd(cmd="csnp") + assert data == "Snapshot creation scheduled." + finally: + destroy_zk_client(zk) From dfb2be3a6732aed0890a82b9d79d12dc1b8ea841 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 26 Sep 2022 18:34:53 +0800 Subject: [PATCH 028/151] fix docs --- docs/en/operations/clickhouse-keeper.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 6597e4e5be0..1b8b1e02aa8 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -123,7 +123,7 @@ clickhouse keeper --config /etc/your_path_to_config/config.xml ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively. -The 4lw commands has a white list configuration `four_letter_word_white_list` which has default value `conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro,csnp`. +The 4lw commands has a white list configuration `four_letter_word_white_list` which has default value `conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro`. You can issue the commands to ClickHouse Keeper via telnet or nc, at the client port. From 13e50c771c3b18db30e817b9ba7ae8a6fe1abb6e Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 28 Sep 2022 11:03:43 +0000 Subject: [PATCH 029/151] Fix bug, use static method --- src/Storages/StorageDelta.cpp | 52 ++------------------------------ src/Storages/StorageDelta.h | 2 -- src/Storages/StorageHudi.cpp | 56 +++-------------------------------- src/Storages/StorageHudi.h | 4 --- src/Storages/StorageS3.h | 2 ++ 5 files changed, 8 insertions(+), 108 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 951337f88c3..2c7f348ef94 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -181,7 +181,7 @@ StorageDelta::StorageDelta( , table_path(uri_.key) { StorageInMemoryMetadata storage_metadata; - updateS3Configuration(context_, base_configuration); + StorageS3::updateS3Configuration(context_, base_configuration); JsonMetadataGetter getter{base_configuration, table_path}; @@ -229,65 +229,17 @@ Pipe StorageDelta::read( size_t max_block_size, unsigned num_streams) { - updateS3Configuration(context, base_configuration); + StorageS3::updateS3Configuration(context, base_configuration); return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } -void StorageDelta::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) -{ - auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString()); - - bool need_update_configuration = settings != S3Settings{}; - if (need_update_configuration) - { - if (upd.rw_settings != settings.rw_settings) - upd.rw_settings = settings.rw_settings; - } - - upd.rw_settings.updateFromSettingsIfEmpty(ctx->getSettings()); - - if (upd.client && (!upd.access_key_id.empty() || settings.auth_settings == upd.auth_settings)) - return; - - Aws::Auth::AWSCredentials credentials(upd.access_key_id, upd.secret_access_key); - HeaderCollection headers; - if (upd.access_key_id.empty()) - { - credentials = Aws::Auth::AWSCredentials(settings.auth_settings.access_key_id, settings.auth_settings.secret_access_key); - headers = settings.auth_settings.headers; - } - - S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( - settings.auth_settings.region, - ctx->getRemoteHostFilter(), - ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, - ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, - /* for_disk_s3 = */ false); - - client_configuration.endpointOverride = upd.uri.endpoint; - client_configuration.maxConnections = upd.rw_settings.max_connections; - - upd.client = S3::ClientFactory::instance().create( - client_configuration, - upd.uri.is_virtual_hosted_style, - credentials.GetAWSAccessKeyId(), - credentials.GetAWSSecretKey(), - settings.auth_settings.server_side_encryption_customer_key_base64, - std::move(headers), - settings.auth_settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)), - settings.auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false))); - - upd.auth_settings = std::move(settings.auth_settings); -} - String StorageDelta::generateQueryFromKeys(std::vector && keys) { std::string new_query = fmt::format("{{{}}}", fmt::join(keys, ",")); return new_query; } - void registerStorageDelta(StorageFactory & factory) { factory.registerStorage( diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index fb6e19b7488..7b25d2c618e 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -87,8 +87,6 @@ public: private: void Init(); - static void updateS3Configuration(ContextPtr, StorageS3::S3Configuration &); - static String generateQueryFromKeys(std::vector && keys); StorageS3::S3Configuration base_configuration; diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 55f7e4b9c10..80555efcafd 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -39,7 +39,7 @@ StorageHudi::StorageHudi( , table_path(uri_.key) { StorageInMemoryMetadata storage_metadata; - updateS3Configuration(context_, base_configuration); + StorageS3::updateS3Configuration(context_, base_configuration); auto keys = getKeysFromS3(); @@ -84,59 +84,11 @@ Pipe StorageHudi::read( QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) -{ - updateS3Configuration(context, base_configuration); - +{ + StorageS3::updateS3Configuration(context, base_configuration); return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } -void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) -{ - auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString()); - - bool need_update_configuration = settings != S3Settings{}; - if (need_update_configuration) - { - if (upd.rw_settings != settings.rw_settings) - upd.rw_settings = settings.rw_settings; - } - - upd.rw_settings.updateFromSettingsIfEmpty(ctx->getSettings()); - - if (upd.client && (!upd.access_key_id.empty() || settings.auth_settings == upd.auth_settings)) - return; - - Aws::Auth::AWSCredentials credentials(upd.access_key_id, upd.secret_access_key); - HeaderCollection headers; - if (upd.access_key_id.empty()) - { - credentials = Aws::Auth::AWSCredentials(settings.auth_settings.access_key_id, settings.auth_settings.secret_access_key); - headers = settings.auth_settings.headers; - } - - S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( - settings.auth_settings.region, - ctx->getRemoteHostFilter(), - ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, - ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, - /* for_disk_s3 = */ false); - - client_configuration.endpointOverride = upd.uri.endpoint; - client_configuration.maxConnections = upd.rw_settings.max_connections; - - upd.client = S3::ClientFactory::instance().create( - client_configuration, - upd.uri.is_virtual_hosted_style, - credentials.GetAWSAccessKeyId(), - credentials.GetAWSSecretKey(), - settings.auth_settings.server_side_encryption_customer_key_base64, - std::move(headers), - settings.auth_settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)), - settings.auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false))); - - upd.auth_settings = std::move(settings.auth_settings); -} - std::vector StorageHudi::getKeysFromS3() { std::vector keys; @@ -186,7 +138,7 @@ std::string StorageHudi::generateQueryFromKeys(std::vector && keys) keys, [](const std::string & s) { - return std::filesystem::path(s).extension() != "parquet"; + return std::filesystem::path(s).extension() != ".parquet"; }); // for each partition path take only latest parquet file diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index ceb12f5f550..e0d5cf5a329 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -44,14 +44,10 @@ public: size_t max_block_size, unsigned num_streams) override; -private: - static void updateS3Configuration(ContextPtr, StorageS3::S3Configuration &); - private: std::vector getKeysFromS3(); static std::string generateQueryFromKeys(std::vector && keys); -private: StorageS3::S3Configuration base_configuration; std::shared_ptr s3engine; Poco::Logger * log; diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 3a02237570d..b6589bf49aa 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -216,6 +216,8 @@ public: private: friend class StorageS3Cluster; friend class TableFunctionS3Cluster; + friend class StorageHudi; + friend class StorageDelta; S3Configuration s3_configuration; std::vector keys; From db7cb4e9b5249c7d31ffbe377d3feffcbc2b0af4 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 28 Sep 2022 11:21:32 +0000 Subject: [PATCH 030/151] apply clang-format --- src/Storages/StorageDelta.cpp | 54 +++++++++++++++++++---------------- src/Storages/StorageDelta.h | 10 +++---- src/Storages/StorageHudi.cpp | 27 +++++++----------- src/Storages/StorageHudi.h | 4 +-- 4 files changed, 47 insertions(+), 48 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 2c7f348ef94..027312385bb 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -2,25 +2,25 @@ #if USE_AWS_S3 -#include -#include +# include +# include -#include -#include -#include -#include +# include +# include +# include +# include -#include -#include +# include +# include -#include -#include -#include +# include +# include +# include -#include +# include -#include -#include +# include +# include namespace DB { @@ -73,16 +73,17 @@ void JsonMetadataGetter::Init() auto buf = createS3ReadBuffer(key); while (!buf->eof()) - { + { // may be some invalid characters before json char c; - while ( buf->peek(c) && c != '{') buf->ignore(); - if (buf->eof()) + while (buf->peek(c) && c != '{') + buf->ignore(); + if (buf->eof()) break; String json_str; readJSONObjectPossiblyInvalid(json_str, *buf); - + if (json_str.empty()) continue; @@ -148,7 +149,8 @@ std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String ReadSettings{}); } -void JsonMetadataGetter::handleJSON(const JSON & json) { +void JsonMetadataGetter::handleJSON(const JSON & json) +{ if (json.has("add")) { auto path = json["add"]["path"].getString(); @@ -195,8 +197,7 @@ StorageDelta::StorageDelta( if (columns_.empty()) { - columns_ - = StorageS3::getTableStructureFromData(format_name_, s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + columns_ = StorageS3::getTableStructureFromData(format_name_, s3_uri, access_key_, secret_access_key_, "", false, {}, context_); storage_metadata.setColumns(columns_); } else @@ -248,15 +249,18 @@ void registerStorageDelta(StorageFactory & factory) { auto & engine_args = args.engine_args; if (engine_args.empty() || engine_args.size() < 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage DeltaLake requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage DeltaLake requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); + - String table_url = checkAndGetLiteralArgument(engine_args[0], "url"); String access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); String secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); - + String format = "Parquet"; - if (engine_args.size() == 4) { + if (engine_args.size() == 4) + { format = checkAndGetLiteralArgument(engine_args[3], "format"); } diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index 7b25d2c618e..c5dc0b2fd07 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -4,11 +4,11 @@ #if USE_AWS_S3 -#include -#include +# include +# include -#include -#include +# include +# include namespace Poco { @@ -52,7 +52,7 @@ private: std::vector getJsonLogFiles(); std::shared_ptr createS3ReadBuffer(const String & key); - + void handleJSON(const JSON & json); StorageS3::S3Configuration base_configuration; diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 80555efcafd..2da5a1e8b5e 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -2,18 +2,18 @@ #if USE_AWS_S3 -#include -#include +# include +# include -#include -#include -#include +# include +# include +# include -#include -#include -#include +# include +# include +# include -#include +# include namespace DB { @@ -84,7 +84,7 @@ Pipe StorageHudi::read( QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) -{ +{ StorageS3::updateS3Configuration(context, base_configuration); return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } @@ -134,12 +134,7 @@ std::vector StorageHudi::getKeysFromS3() std::string StorageHudi::generateQueryFromKeys(std::vector && keys) { // filter only .parquet files - std::erase_if( - keys, - [](const std::string & s) - { - return std::filesystem::path(s).extension() != ".parquet"; - }); + std::erase_if(keys, [](const std::string & s) { return std::filesystem::path(s).extension() != ".parquet"; }); // for each partition path take only latest parquet file diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index e0d5cf5a329..d782106d2b0 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -4,8 +4,8 @@ #if USE_AWS_S3 -#include -#include +# include +# include namespace Poco { From 0d3670d516f8dbae493784dc50c09c726d21793a Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 28 Sep 2022 17:30:15 +0000 Subject: [PATCH 031/151] Fix typo --- src/Storages/StorageHudi.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 2da5a1e8b5e..4db372e7e96 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -153,7 +153,7 @@ std::string StorageHudi::generateQueryFromKeys(std::vector && keys) path = key.substr(0, slash); } - // every filename contains metadata splitted by "_", timestamp is after last "_" + // every filename contains metadata split by "_", timestamp is after last "_" uint64_t timestamp = std::stoul(key.substr(key.find_last_of("_") + 1)); auto it = latest_parquets.find(path); From 74f026ba65f53c54f33bb83d175a40d17465e66e Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Thu, 29 Sep 2022 20:15:27 +0000 Subject: [PATCH 032/151] Change logic error, add format argument to Hudi engine --- src/Storages/StorageHudi.cpp | 49 ++++++++++++++++++++++-------------- src/Storages/StorageHudi.h | 3 ++- 2 files changed, 32 insertions(+), 20 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 4db372e7e96..78f3a0dd319 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -20,6 +20,7 @@ namespace DB namespace ErrorCodes { + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; extern const int S3_ERROR; } @@ -29,6 +30,7 @@ StorageHudi::StorageHudi( const String & access_key_, const String & secret_access_key_, const StorageID & table_id_, + const String & format_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, @@ -43,7 +45,7 @@ StorageHudi::StorageHudi( auto keys = getKeysFromS3(); - auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); + auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys), format_); LOG_DEBUG(log, "New uri: {}", new_uri); LOG_DEBUG(log, "Table path: {}", table_path); @@ -51,8 +53,7 @@ StorageHudi::StorageHudi( if (columns_.empty()) { - columns_ - = StorageS3::getTableStructureFromData(String("Parquet"), s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + columns_ = StorageS3::getTableStructureFromData(format_, s3_uri, access_key_, secret_access_key_, "", false, {}, context_); storage_metadata.setColumns(columns_); } else @@ -67,7 +68,7 @@ StorageHudi::StorageHudi( access_key_, secret_access_key_, table_id_, - String("Parquet"), // format name + format_, base_configuration.rw_settings, columns_, constraints_, @@ -131,12 +132,15 @@ std::vector StorageHudi::getKeysFromS3() return keys; } -std::string StorageHudi::generateQueryFromKeys(std::vector && keys) +std::string StorageHudi::generateQueryFromKeys(std::vector && keys, String format) { - // filter only .parquet files - std::erase_if(keys, [](const std::string & s) { return std::filesystem::path(s).extension() != ".parquet"; }); + // make format lowercase + std::transform(format.begin(), format.end(), format.begin(), [](unsigned char c) { return std::tolower(c); }); - // for each partition path take only latest parquet file + // filter only files with specific format + std::erase_if(keys, [&format](const std::string & s) { return std::filesystem::path(s).extension() != "." + format; }); + + // for each partition path take only latest file std::unordered_map> latest_parquets; @@ -198,23 +202,30 @@ void registerStorageHudi(StorageFactory & factory) [](const StorageFactory::Arguments & args) { auto & engine_args = args.engine_args; - if (engine_args.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); - - auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); - - configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); - configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); - configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + if (engine_args.empty() || engine_args.size() < 3) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage Hudi requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); - S3::URI s3_uri(Poco::URI(configuration.url)); + String table_url = checkAndGetLiteralArgument(engine_args[0], "url"); + String access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + String secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + + String format = "Parquet"; + if (engine_args.size() == 4) + { + format = checkAndGetLiteralArgument(engine_args[3], "format"); + } + + auto s3_uri = S3::URI(Poco::URI(table_url)); return std::make_shared( s3_uri, - configuration.auth_settings.access_key_id, - configuration.auth_settings.secret_access_key, + access_key_id, + secret_access_key, args.table_id, + format, args.columns, args.constraints, args.comment, diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index d782106d2b0..7b647345441 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -28,6 +28,7 @@ public: const String & access_key_, const String & secret_access_key_, const StorageID & table_id_, + const String & format_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, @@ -46,7 +47,7 @@ public: private: std::vector getKeysFromS3(); - static std::string generateQueryFromKeys(std::vector && keys); + static std::string generateQueryFromKeys(std::vector && keys, String format); StorageS3::S3Configuration base_configuration; std::shared_ptr s3engine; From 97dd75194a92913ef91bd2bc23d8203694ff171e Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Thu, 29 Sep 2022 20:36:54 +0000 Subject: [PATCH 033/151] Remove unused error code --- src/Storages/StorageHudi.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 78f3a0dd319..4aff4ff3a43 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -21,7 +21,6 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_ARGUMENTS; extern const int S3_ERROR; } From 68abf43767d6f6e998373094147b7901b6222063 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sun, 9 Oct 2022 00:00:14 +0000 Subject: [PATCH 034/151] Better INTERVAL parsing and execution --- src/Functions/FunctionBinaryArithmetic.h | 71 ++++++++++++ src/Functions/vectorFunctions.cpp | 105 ++++++++++++++++++ src/Parsers/ExpressionListParsers.cpp | 81 ++++++++------ .../02457_tuple_of_intervals.reference | 17 +++ .../0_stateless/02457_tuple_of_intervals.sql | 21 ++++ 5 files changed, 262 insertions(+), 33 deletions(-) create mode 100644 tests/queries/0_stateless/02457_tuple_of_intervals.reference create mode 100644 tests/queries/0_stateless/02457_tuple_of_intervals.sql diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 174e98dd81f..399cffac85e 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -708,6 +708,41 @@ class FunctionBinaryArithmetic : public IFunction return FunctionFactory::instance().get(function_name, context); } + static FunctionOverloadResolverPtr + getFunctionForTupleOfIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) + { + bool first_is_date_or_datetime = isDate(type0) || isDateTime(type0) || isDateTime64(type0); + bool second_is_date_or_datetime = isDate(type1) || isDateTime(type1) || isDateTime64(type1); + + /// Exactly one argument must be Date or DateTime + if (first_is_date_or_datetime == second_is_date_or_datetime) + return {}; + + if (!isTuple(type0) && !isTuple(type1)) + return {}; + + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. + /// We construct another function and call it. + if constexpr (!is_plus && !is_minus) + return {}; + + if (isTuple(type0) && second_is_date_or_datetime && is_minus) + throw Exception("Wrong order of arguments for function " + String(name) + ": argument of Tuple type cannot be first", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + std::string function_name; + if (is_plus) + { + function_name = "addTupleOfIntervals"; + } + else if (is_minus) + { + function_name = "subtractTupleOfIntervals"; + } + + return FunctionFactory::instance().get(function_name, context); + } + static FunctionOverloadResolverPtr getFunctionForTupleArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { @@ -906,6 +941,20 @@ class FunctionBinaryArithmetic : public IFunction return function->execute(new_arguments, result_type, input_rows_count); } + ColumnPtr executeDateTimeTupleOfIntervalsPlusMinus(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, + size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const + { + ColumnsWithTypeAndName new_arguments = arguments; + + /// Tuple argument must be second. + if (isTuple(arguments[0].type)) + std::swap(new_arguments[0], new_arguments[1]); + + auto function = function_builder->build(new_arguments); + + return function->execute(new_arguments, result_type, input_rows_count); + } + ColumnPtr executeTupleNumberOperator(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const { @@ -1121,6 +1170,22 @@ public: return function->getResultType(); } + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. + if (auto function_builder = getFunctionForTupleOfIntervalsArithmetic(arguments[0], arguments[1], context)) + { + ColumnsWithTypeAndName new_arguments(2); + + for (size_t i = 0; i < 2; ++i) + new_arguments[i].type = arguments[i]; + + /// Tuple argument must be second. + if (isTuple(new_arguments[0].type)) + std::swap(new_arguments[0], new_arguments[1]); + + auto function = function_builder->build(new_arguments); + return function->getResultType(); + } + /// Special case when the function is multiply or divide, one of arguments is Tuple and another is Number. if (auto function_builder = getFunctionForTupleAndNumberArithmetic(arguments[0], arguments[1], context)) { @@ -1553,6 +1618,12 @@ public: return executeDateTimeIntervalPlusMinus(arguments, result_type, input_rows_count, function_builder); } + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. + if (auto function_builder = getFunctionForTupleOfIntervalsArithmetic(arguments[0].type, arguments[1].type, context)) + { + return executeDateTimeTupleOfIntervalsPlusMinus(arguments, result_type, input_rows_count, function_builder); + } + /// Special case when the function is plus, minus or multiply, both arguments are tuples. if (auto function_builder = getFunctionForTupleArithmetic(arguments[0].type, arguments[1].type, context)) { diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index 20571f67eff..4e9f2a71f8c 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -415,6 +415,108 @@ public: } }; +template +class FunctionDateOrDateTimeOperationTupleOfIntervals : public ITupleFunction +{ +public: + static constexpr auto name = Impl::name; + + explicit FunctionDateOrDateTimeOperationTupleOfIntervals(ContextPtr context_) : ITupleFunction(context_) {} + static FunctionPtr create(ContextPtr context_) + { + return std::make_shared(context_); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) + throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}. Should be a date or a date with time", + arguments[0].type->getName(), getName()}; + + const auto * cur_tuple = checkAndGetDataType(arguments[1].type.get()); + + if (!cur_tuple) + throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}. Should be a tuple", + arguments[0].type->getName(), getName()}; + + const auto & cur_types = cur_tuple->getElements(); + + Columns cur_elements; + if (arguments[1].column) + cur_elements = getTupleElements(*arguments[1].column); + + size_t tuple_size = cur_types.size(); + if (tuple_size == 0) + return arguments[0].type; + + auto plus = FunctionFactory::instance().get(Impl::func_name, context); + DataTypePtr res_type = arguments[0].type; + for (size_t i = 0; i < tuple_size; ++i) + { + try + { + ColumnWithTypeAndName left{res_type, {}}; + ColumnWithTypeAndName right{cur_elements.empty() ? nullptr : cur_elements[i], cur_types[i], {}}; + auto plus_elem = plus->build({left, right}); + res_type = plus_elem->getResultType(); + } + catch (DB::Exception & e) + { + e.addMessage("While executing function {} for tuple element {}", getName(), i); + throw; + } + } + + return res_type; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto * cur_tuple = checkAndGetDataType(arguments[1].type.get()); + const auto & cur_types = cur_tuple->getElements(); + auto cur_elements = getTupleElements(*arguments[1].column); + + size_t tuple_size = cur_elements.size(); + if (tuple_size == 0) + return arguments[0].column; + + auto plus = FunctionFactory::instance().get(Impl::func_name, context); + ColumnWithTypeAndName res; + for (size_t i = 0; i < tuple_size; ++i) + { + ColumnWithTypeAndName column{cur_elements[i], cur_types[i], {}}; + auto elem_plus = plus->build(ColumnsWithTypeAndName{i == 0 ? arguments[0] : res, column}); + auto res_type = elem_plus->getResultType(); + res.column = elem_plus->execute({i == 0 ? arguments[0] : res, column}, res_type, input_rows_count); + res.type = res_type; + } + + return res.column; + } +}; + +struct AddTupleOfIntervalsImpl +{ + static constexpr auto name = "addTupleOfIntervals"; + static constexpr auto func_name = "plus"; +}; + +struct SubtractTupleOfIntervalsImpl +{ + static constexpr auto name = "subtractTupleOfIntervals"; + static constexpr auto func_name = "minus"; +}; + +using FunctionAddTupleOfIntervals = FunctionDateOrDateTimeOperationTupleOfIntervals; + +using FunctionSubtractTupleOfIntervals = FunctionDateOrDateTimeOperationTupleOfIntervals; + /// this is for convenient usage in LNormalize template class FunctionLNorm : public ITupleFunction {}; @@ -1282,6 +1384,9 @@ REGISTER_FUNCTION(VectorFunctions) factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index f7a016a59e4..353f22b03b6 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -1684,6 +1684,16 @@ private: class IntervalLayer : public Layer { public: + bool getResult(ASTPtr & node) override + { + if (elements.size() == 1) + node = elements[0]; + else + node = makeASTFunction("tuple", std::move(elements)); + + return true; + } + bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { /// INTERVAL 1 HOUR or INTERVAL expr HOUR @@ -1693,49 +1703,54 @@ public: if (state == 0) { + state = 1; + auto begin = pos; auto init_expected = expected; ASTPtr string_literal; + String literal; + //// A String literal followed INTERVAL keyword, /// the literal can be a part of an expression or /// include Number and INTERVAL TYPE at the same time - if (ParserStringLiteral{}.parse(pos, string_literal, expected)) + if (ParserStringLiteral{}.parse(pos, string_literal, expected) + && string_literal->as().value.tryGet(literal)) { - String literal; - if (string_literal->as().value.tryGet(literal)) + Tokens tokens(literal.data(), literal.data() + literal.size()); + IParser::Pos token_pos(tokens, 0); + Expected token_expected; + ASTPtr expr; + + if (!ParserNumber{}.parse(token_pos, expr, token_expected)) + return false; + + /// case: INTERVAL '1' HOUR + /// back to begin + if (!token_pos.isValid()) { - Tokens tokens(literal.data(), literal.data() + literal.size()); - IParser::Pos token_pos(tokens, 0); - Expected token_expected; - ASTPtr expr; - - if (!ParserNumber{}.parse(token_pos, expr, token_expected)) - { - return false; - } - else - { - /// case: INTERVAL '1' HOUR - /// back to begin - if (!token_pos.isValid()) - { - pos = begin; - expected = init_expected; - } - else - { - /// case: INTERVAL '1 HOUR' - if (!parseIntervalKind(token_pos, token_expected, interval_kind)) - return false; - - elements = {makeASTFunction(interval_kind.toNameOfFunctionToIntervalDataType(), expr)}; - finished = true; - return true; - } - } + pos = begin; + expected = init_expected; + return true; } + + /// case: INTERVAL '1 HOUR' + if (!parseIntervalKind(token_pos, token_expected, interval_kind)) + return false; + + pushResult(makeASTFunction(interval_kind.toNameOfFunctionToIntervalDataType(), expr)); + + /// case: INTERVAL '1 HOUR 1 SECOND ...' + while (token_pos.isValid()) + { + if (!ParserNumber{}.parse(token_pos, expr, token_expected) || + !parseIntervalKind(token_pos, token_expected, interval_kind)) + return false; + + pushResult(makeASTFunction(interval_kind.toNameOfFunctionToIntervalDataType(), expr)); + } + + finished = true; } - state = 1; return true; } diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.reference b/tests/queries/0_stateless/02457_tuple_of_intervals.reference new file mode 100644 index 00000000000..40bbfb35d91 --- /dev/null +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.reference @@ -0,0 +1,17 @@ +SELECT (toIntervalSecond(-1), toIntervalMinute(2), toIntervalMonth(-3), toIntervalYear(1)) +- +2022-11-12 +2022-11-12 +2022-11-12 +- +2023-07-11 00:01:59 +2023-07-11 00:01:59 +2023-07-11 00:01:59 +- +2021-07-31 23:00:00 +2021-07-31 23:00:00 +2021-07-31 23:00:00 +- +2021-06-10 23:59:59.000 +2021-06-10 23:59:59.000 +2021-06-10 23:59:59.000 diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.sql b/tests/queries/0_stateless/02457_tuple_of_intervals.sql new file mode 100644 index 00000000000..2c2feaf522a --- /dev/null +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.sql @@ -0,0 +1,21 @@ +EXPLAIN SYNTAX SELECT INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; + +SELECT '-'; +SELECT '2022-10-11'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH; +SELECT '2022-10-11'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH); +SELECT '2022-10-11'::Date + INTERVAL '1 DAY 1 MONTH'; + +SELECT '-'; +SELECT '2022-10-11'::Date + INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR; +SELECT '2022-10-11'::Date + (INTERVAL -1 SECOND, INTERVAL 2 MINUTE, INTERVAL -3 MONTH, INTERVAL 1 YEAR); +SELECT '2022-10-11'::Date + INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; + +SELECT '-'; +SELECT '2022-10-11'::DateTime - INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR; +SELECT '2022-10-11'::DateTime - (INTERVAL 1 QUARTER, INTERVAL -3 WEEK, INTERVAL 1 YEAR, INTERVAL 1 HOUR); +SELECT '2022-10-11'::DateTime - INTERVAL '1 QUARTER -3 WEEK 1 YEAR 1 HOUR'; + +SELECT '-'; +SELECT '2022-10-11'::DateTime64 - INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND; +SELECT '2022-10-11'::DateTime64 - (INTERVAL 1 YEAR, INTERVAL 4 MONTH, INTERVAL 1 SECOND); +SELECT '2022-10-11'::DateTime64 - INTERVAL '1 YEAR 4 MONTH 1 SECOND'; From f823b9a177e77ee1a6c25f6779c13f9e031c662b Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 10 Oct 2022 13:31:10 +0800 Subject: [PATCH 035/151] add not like to atom map --- src/Storages/MergeTree/KeyCondition.cpp | 21 +++++++++++++++++++ ...58_key_condition_not_like_prefix.reference | 1 + .../002458_key_condition_not_like_prefix.sql | 5 +++++ 3 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/002458_key_condition_not_like_prefix.reference create mode 100644 tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 24b3a4a60b9..ccdd26eb333 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -560,6 +560,27 @@ const KeyCondition::AtomMap KeyCondition::atom_map return true; } }, + { + "notLike", + [] (RPNElement & out, const Field & value) + { + if (value.getType() != Field::Types::String) + return false; + + String prefix = extractFixedPrefixFromLikePattern(value.get()); + if (prefix.empty()) + return false; + + String right_bound = firstStringThatIsGreaterThanAllStringsWithPrefix(prefix); + + out.function = RPNElement::FUNCTION_NOT_IN_RANGE; + out.range = !right_bound.empty() + ? Range(prefix, true, right_bound, false) + : Range::createLeftBounded(prefix, true); + + return true; + } + }, { "startsWith", [] (RPNElement & out, const Field & value) diff --git a/tests/queries/0_stateless/002458_key_condition_not_like_prefix.reference b/tests/queries/0_stateless/002458_key_condition_not_like_prefix.reference new file mode 100644 index 00000000000..87766d889a3 --- /dev/null +++ b/tests/queries/0_stateless/002458_key_condition_not_like_prefix.reference @@ -0,0 +1 @@ +200000 diff --git a/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql b/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql new file mode 100644 index 00000000000..211fa5662e7 --- /dev/null +++ b/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql @@ -0,0 +1,5 @@ +CREATE TABLE data (str String) ENGINE=MergeTree ORDER BY str; +INSERT INTO data (str) SELECT 'aa' FROM numbers(100000); +INSERT INTO data (str) SELECT 'ba' FROM numbers(100000); +INSERT INTO data (str) SELECT 'ca' FROM numbers(100000); +SELECT count()FROM data WHERE str NOT LIKE 'a%' SETTINGS force_primary_key=1; \ No newline at end of file From b724b7a74a2b88e10ef08a6aacc32b64a38105cf Mon Sep 17 00:00:00 2001 From: lixuchun Date: Wed, 12 Oct 2022 11:57:35 +0800 Subject: [PATCH 036/151] update docs error --- docs/en/engines/database-engines/replicated.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/replicated.md b/docs/en/engines/database-engines/replicated.md index f0ef1e981fe..43d1ce5ec3f 100644 --- a/docs/en/engines/database-engines/replicated.md +++ b/docs/en/engines/database-engines/replicated.md @@ -86,7 +86,7 @@ node1 :) SELECT materialize(hostName()) AS host, groupArray(n) FROM r.d GROUP BY ``` text ┌─hosts─┬─groupArray(n)─┐ -│ node1 │ [1,3,5,7,9] │ +│ node3 │ [1,3,5,7,9] │ │ node2 │ [0,2,4,6,8] │ └───────┴───────────────┘ ``` From af1d306b12756e3c2f5d5de4bb7df0086c95ba77 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 14 Oct 2022 02:16:12 +0000 Subject: [PATCH 037/151] Add Interval arithmetics --- src/DataTypes/IDataType.h | 1 + src/Functions/FunctionBinaryArithmetic.h | 100 +++++++- src/Functions/FunctionUnaryArithmetic.h | 10 +- src/Functions/vectorFunctions.cpp | 227 +++++++++++++++++- .../02457_tuple_of_intervals.reference | 31 ++- .../0_stateless/02457_tuple_of_intervals.sql | 53 ++-- 6 files changed, 380 insertions(+), 42 deletions(-) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index c93128ced95..45353796f3c 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -408,6 +408,7 @@ inline bool isDecimal(const DataTypePtr & data_type) { return WhichDataType(data inline bool isTuple(const DataTypePtr & data_type) { return WhichDataType(data_type).isTuple(); } inline bool isArray(const DataTypePtr & data_type) { return WhichDataType(data_type).isArray(); } inline bool isMap(const DataTypePtr & data_type) {return WhichDataType(data_type).isMap(); } +inline bool isInterval(const DataTypePtr & data_type) {return WhichDataType(data_type).isInterval(); } inline bool isNothing(const DataTypePtr & data_type) { return WhichDataType(data_type).isNothing(); } inline bool isUUID(const DataTypePtr & data_type) { return WhichDataType(data_type).isUUID(); } diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 399cffac85e..e4919d3e9d7 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -633,7 +634,8 @@ class FunctionBinaryArithmetic : public IFunction DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64, DataTypeInt128, DataTypeInt256, DataTypeDecimal32, DataTypeDecimal64, DataTypeDecimal128, DataTypeDecimal256, DataTypeDate, DataTypeDateTime, - DataTypeFixedString, DataTypeString>; + DataTypeFixedString, DataTypeString, + DataTypeInterval>; using Floats = TypeList; @@ -709,10 +711,10 @@ class FunctionBinaryArithmetic : public IFunction } static FunctionOverloadResolverPtr - getFunctionForTupleOfIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) + getFunctionForDateTupleOfIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { - bool first_is_date_or_datetime = isDate(type0) || isDateTime(type0) || isDateTime64(type0); - bool second_is_date_or_datetime = isDate(type1) || isDateTime(type1) || isDateTime64(type1); + bool first_is_date_or_datetime = isDateOrDate32(type0) || isDateTime(type0) || isDateTime64(type0); + bool second_is_date_or_datetime = isDateOrDate32(type1) || isDateTime(type1) || isDateTime64(type1); /// Exactly one argument must be Date or DateTime if (first_is_date_or_datetime == second_is_date_or_datetime) @@ -735,7 +737,7 @@ class FunctionBinaryArithmetic : public IFunction { function_name = "addTupleOfIntervals"; } - else if (is_minus) + else { function_name = "subtractTupleOfIntervals"; } @@ -743,6 +745,47 @@ class FunctionBinaryArithmetic : public IFunction return FunctionFactory::instance().get(function_name, context); } + static FunctionOverloadResolverPtr + getFunctionForMergeIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) + { + /// Special case when the function is plus or minus, first argument is Interval or Tuple of Intervals + /// and the second argument is the Inteval of a different kind. + /// We construct another function (example: addIntervals) and call it + + if constexpr (!is_plus && !is_minus) + return {}; + + const auto * tuple_data_type_0 = checkAndGetDataType(type0.get()); + const auto * interval_data_type_0 = checkAndGetDataType(type0.get()); + const auto * interval_data_type_1 = checkAndGetDataType(type1.get()); + + if ((!tuple_data_type_0 && !interval_data_type_0) || !interval_data_type_1) + return {}; + + if (interval_data_type_0 && interval_data_type_0->equals(*interval_data_type_1)) + return {}; + + if (tuple_data_type_0) + { + auto & tuple_types = tuple_data_type_0->getElements(); + for (auto & type : tuple_types) + if (!isInterval(type)) + return {}; + } + + std::string function_name; + if (is_plus) + { + function_name = "addInterval"; + } + else + { + function_name = "subtractInterval"; + } + + return FunctionFactory::instance().get(function_name, context); + } + static FunctionOverloadResolverPtr getFunctionForTupleArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { @@ -955,6 +998,16 @@ class FunctionBinaryArithmetic : public IFunction return function->execute(new_arguments, result_type, input_rows_count); } + ColumnPtr executeIntervalTupleOfIntervalsPlusMinus(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, + size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const + { + ColumnsWithTypeAndName new_arguments = arguments; + + auto function = function_builder->build(new_arguments); + + return function->execute(new_arguments, result_type, input_rows_count); + } + ColumnPtr executeTupleNumberOperator(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const { @@ -1171,7 +1224,7 @@ public: } /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. - if (auto function_builder = getFunctionForTupleOfIntervalsArithmetic(arguments[0], arguments[1], context)) + if (auto function_builder = getFunctionForDateTupleOfIntervalsArithmetic(arguments[0], arguments[1], context)) { ColumnsWithTypeAndName new_arguments(2); @@ -1186,6 +1239,18 @@ public: return function->getResultType(); } + /// Special case when the function is plus or minus, one of arguments is Interval/Tuple of Intervals and another is Interval. + if (auto function_builder = getFunctionForMergeIntervalsArithmetic(arguments[0], arguments[1], context)) + { + ColumnsWithTypeAndName new_arguments(2); + + for (size_t i = 0; i < 2; ++i) + new_arguments[i].type = arguments[i]; + + auto function = function_builder->build(new_arguments); + return function->getResultType(); + } + /// Special case when the function is multiply or divide, one of arguments is Tuple and another is Number. if (auto function_builder = getFunctionForTupleAndNumberArithmetic(arguments[0], arguments[1], context)) { @@ -1237,6 +1302,21 @@ public: type_res = std::make_shared(); return true; } + else if constexpr (std::is_same_v || std::is_same_v) + { + if constexpr (std::is_same_v && + std::is_same_v) + { + if constexpr (is_plus || is_minus) + { + if (left.getKind() == right.getKind()) + { + type_res = std::make_shared(left.getKind()); + return true; + } + } + } + } else { using ResultDataType = typename BinaryOperationTraits::ResultDataType; @@ -1619,11 +1699,17 @@ public: } /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. - if (auto function_builder = getFunctionForTupleOfIntervalsArithmetic(arguments[0].type, arguments[1].type, context)) + if (auto function_builder = getFunctionForDateTupleOfIntervalsArithmetic(arguments[0].type, arguments[1].type, context)) { return executeDateTimeTupleOfIntervalsPlusMinus(arguments, result_type, input_rows_count, function_builder); } + /// Special case when the function is plus or minus, one of arguments is Interval/Tuple of Intervals and another is Interval. + if (auto function_builder = getFunctionForMergeIntervalsArithmetic(arguments[0].type, arguments[1].type, context)) + { + return executeIntervalTupleOfIntervalsPlusMinus(arguments, result_type, input_rows_count, function_builder); + } + /// Special case when the function is plus, minus or multiply, both arguments are tuples. if (auto function_builder = getFunctionForTupleArithmetic(arguments[0].type, arguments[1].type, context)) { diff --git a/src/Functions/FunctionUnaryArithmetic.h b/src/Functions/FunctionUnaryArithmetic.h index 445eb45fd9d..f5ddc5cb67c 100644 --- a/src/Functions/FunctionUnaryArithmetic.h +++ b/src/Functions/FunctionUnaryArithmetic.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -145,7 +146,8 @@ class FunctionUnaryArithmetic : public IFunction DataTypeDecimal, DataTypeDecimal, DataTypeDecimal, - DataTypeFixedString + DataTypeFixedString, + DataTypeInterval >(type, std::forward(f)); } @@ -211,6 +213,12 @@ public: return false; result = std::make_shared(type.getN()); } + else if constexpr (std::is_same_v) + { + if constexpr (!IsUnaryOperation::negate) + return false; + result = std::make_shared(type.getKind()); + } else { using T0 = typename DataType::FieldType; diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index 4e9f2a71f8c..007875a8b81 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -517,6 +518,172 @@ using FunctionAddTupleOfIntervals = FunctionDateOrDateTimeOperationTupleOfInterv using FunctionSubtractTupleOfIntervals = FunctionDateOrDateTimeOperationTupleOfIntervals; +template +struct FunctionTupleOperationInterval : public ITupleFunction +{ +public: + static constexpr auto name = is_minus ? "subtractInterval" : "addInterval"; + + explicit FunctionTupleOperationInterval(ContextPtr context_) : ITupleFunction(context_) {} + + static FunctionPtr create(ContextPtr context_) + { + return std::make_shared(context_); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isTuple(arguments[0]) && !isInterval(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}, must be Tuple or Interval", + arguments[0]->getName(), getName()); + + if (!isInterval(arguments[1])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}, must be Interval", + arguments[0]->getName(), getName()); + + DataTypes types; + + const auto * tuple = checkAndGetDataType(arguments[0].get()); + + if (tuple) + { + const auto & cur_types = tuple->getElements(); + + for (auto & type : cur_types) + if (!isInterval(type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of Tuple element of first argument of function {}, must be Interval", + types.back()->getName(), getName()); + + types = cur_types; + } + else + { + types = {arguments[0]}; + } + + const auto * interval_last = checkAndGetDataType(types.back().get()); + const auto * interval_new = checkAndGetDataType(arguments[1].get()); + + if (!interval_last->equals(*interval_new)) + types.push_back(arguments[1]); + + return std::make_shared(types); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + if (!isInterval(arguments[1].type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}, must be Interval", + arguments[0].type->getName(), getName()); + + Columns tuple_columns; + + const auto * first_tuple = checkAndGetDataType(arguments[0].type.get()); + const auto * first_interval = checkAndGetDataType(arguments[0].type.get()); + const auto * second_interval = checkAndGetDataType(arguments[1].type.get()); + + bool can_be_merged; + + if (first_interval) + { + can_be_merged = first_interval->equals(*second_interval); + + if (can_be_merged) + tuple_columns.resize(1); + else + tuple_columns.resize(2); + + tuple_columns[0] = arguments[0].column->convertToFullColumnIfConst(); + } + else if (first_tuple) + { + const auto & cur_types = first_tuple->getElements(); + + for (auto & type : cur_types) + if (!isInterval(type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of Tuple element of first argument of function {}, must be Interval", + type->getName(), getName()); + + auto cur_elements = getTupleElements(*arguments[0].column); + size_t tuple_size = cur_elements.size(); + + if (tuple_size == 0) + { + can_be_merged = false; + } + else + { + const auto * tuple_last_interval = checkAndGetDataType(cur_types.back().get()); + can_be_merged = tuple_last_interval->equals(*second_interval); + } + + if (can_be_merged) + tuple_columns.resize(tuple_size); + else + tuple_columns.resize(tuple_size + 1); + + for (size_t i = 0; i < tuple_size; ++i) + tuple_columns[i] = cur_elements[i]; + } + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}, must be Tuple or Interval", + arguments[0].type->getName(), getName()); + + + ColumnPtr & last_column = tuple_columns.back(); + + if (can_be_merged) + { + ColumnWithTypeAndName left{last_column, arguments[1].type, {}}; + + if constexpr (is_minus) + { + auto minus = FunctionFactory::instance().get("minus", context); + auto elem_minus = minus->build({left, arguments[1]}); + last_column = elem_minus->execute({left, arguments[1]}, arguments[1].type, input_rows_count) + ->convertToFullColumnIfConst(); + } + else + { + auto plus = FunctionFactory::instance().get("plus", context); + auto elem_plus = plus->build({left, arguments[1]}); + last_column = elem_plus->execute({left, arguments[1]}, arguments[1].type, input_rows_count) + ->convertToFullColumnIfConst(); + } + } + else + { + if constexpr (is_minus) + { + auto negate = FunctionFactory::instance().get("negate", context); + auto elem_negate = negate->build({arguments[1]}); + last_column = elem_negate->execute({arguments[1]}, arguments[1].type, input_rows_count); + } + else + { + last_column = arguments[1].column; + } + } + + return ColumnTuple::create(tuple_columns); + } +}; + +using FunctionTupleAddInterval = FunctionTupleOperationInterval; + +using FunctionTupleSubtractInterval = FunctionTupleOperationInterval; + + /// this is for convenient usage in LNormalize template class FunctionLNorm : public ITupleFunction {}; @@ -1384,8 +1551,64 @@ REGISTER_FUNCTION(VectorFunctions) factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction( + { + R"( +Consecutively adds a tuple of intervals to a Date or a DateTime. +[example:tuple] +)", + Documentation::Examples{ + {"tuple", "WITH toDate('2018-01-01') AS date SELECT addTupleOfIntervals(date, (INTERVAL 1 DAY, INTERVAL 1 YEAR))"}, + }, + Documentation::Categories{"Tuple", "Interval", "Date", "DateTime"} + }); + + factory.registerFunction( + { + R"( +Consecutively subtracts a tuple of intervals from a Date or a DateTime. +[example:tuple] +)", + Documentation::Examples{ + {"tuple", "WITH toDate('2018-01-01') AS date SELECT subtractTupleOfIntervals(date, (INTERVAL 1 DAY, INTERVAL 1 YEAR))"}, + }, + Documentation::Categories{"Tuple", "Interval", "Date", "DateTime"} + }); + + factory.registerFunction( + { + R"( +Adds an interval to another interval or tuple of intervals. The returned value is tuple of intervals. +[example:tuple] +[example:interval1] + +If the types of the first interval (or the interval in the tuple) and the second interval are the same they will be merged into one interval. +[example:interval2] +)", + Documentation::Examples{ + {"tuple", "SELECT addInterval((INTERVAL 1 DAY, INTERVAL 1 YEAR), INTERVAL 1 MONTH)"}, + {"interval1", "SELECT addInterval(INTERVAL 1 DAY, INTERVAL 1 MONTH)"}, + {"interval2", "SELECT addInterval(INTERVAL 1 DAY, INTERVAL 1 DAY)"}, + }, + Documentation::Categories{"Tuple", "Interval"} + }); + factory.registerFunction( + { + R"( +Adds an negated interval to another interval or tuple of intervals. The returned value is tuple of intervals. +[example:tuple] +[example:interval1] + +If the types of the first interval (or the interval in the tuple) and the second interval are the same they will be merged into one interval. +[example:interval2] +)", + Documentation::Examples{ + {"tuple", "SELECT subtractInterval((INTERVAL 1 DAY, INTERVAL 1 YEAR), INTERVAL 1 MONTH)"}, + {"interval1", "SELECT subtractInterval(INTERVAL 1 DAY, INTERVAL 1 MONTH)"}, + {"interval2", "SELECT subtractInterval(INTERVAL 2 DAY, INTERVAL 1 DAY)"}, + }, + Documentation::Categories{"Tuple", "Interval"} + }); factory.registerFunction(); factory.registerFunction(); diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.reference b/tests/queries/0_stateless/02457_tuple_of_intervals.reference index 40bbfb35d91..dd190dce891 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.reference +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.reference @@ -1,17 +1,16 @@ SELECT (toIntervalSecond(-1), toIntervalMinute(2), toIntervalMonth(-3), toIntervalYear(1)) -- -2022-11-12 -2022-11-12 -2022-11-12 -- -2023-07-11 00:01:59 -2023-07-11 00:01:59 -2023-07-11 00:01:59 -- -2021-07-31 23:00:00 -2021-07-31 23:00:00 -2021-07-31 23:00:00 -- -2021-06-10 23:59:59.000 -2021-06-10 23:59:59.000 -2021-06-10 23:59:59.000 +--- +3 IntervalSecond +(1,2) Tuple(IntervalHour, IntervalSecond) +(1,1,1) Tuple(IntervalSecond, IntervalHour, IntervalSecond) +(2,1) Tuple(IntervalSecond, IntervalHour) +--- +-3 IntervalSecond +(-1,-2) Tuple(IntervalHour, IntervalSecond) +(-1,-1,-1) Tuple(IntervalSecond, IntervalHour, IntervalSecond) +(-2,-1) Tuple(IntervalSecond, IntervalHour) +--- +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.sql b/tests/queries/0_stateless/02457_tuple_of_intervals.sql index 2c2feaf522a..d4065ab98f8 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.sql +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.sql @@ -1,21 +1,42 @@ EXPLAIN SYNTAX SELECT INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; -SELECT '-'; -SELECT '2022-10-11'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH; -SELECT '2022-10-11'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH); -SELECT '2022-10-11'::Date + INTERVAL '1 DAY 1 MONTH'; +SELECT '---'; -SELECT '-'; -SELECT '2022-10-11'::Date + INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR; -SELECT '2022-10-11'::Date + (INTERVAL -1 SECOND, INTERVAL 2 MINUTE, INTERVAL -3 MONTH, INTERVAL 1 YEAR); -SELECT '2022-10-11'::Date + INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; +WITH INTERVAL 1 SECOND + INTERVAL 1 SECOND + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 HOUR + INTERVAL 1 SECOND + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 SECOND + INTERVAL 1 HOUR + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 SECOND + INTERVAL 1 SECOND + INTERVAL 1 HOUR as expr SELECT expr, toTypeName(expr); -SELECT '-'; -SELECT '2022-10-11'::DateTime - INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR; -SELECT '2022-10-11'::DateTime - (INTERVAL 1 QUARTER, INTERVAL -3 WEEK, INTERVAL 1 YEAR, INTERVAL 1 HOUR); -SELECT '2022-10-11'::DateTime - INTERVAL '1 QUARTER -3 WEEK 1 YEAR 1 HOUR'; +SELECT '---'; -SELECT '-'; -SELECT '2022-10-11'::DateTime64 - INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND; -SELECT '2022-10-11'::DateTime64 - (INTERVAL 1 YEAR, INTERVAL 4 MONTH, INTERVAL 1 SECOND); -SELECT '2022-10-11'::DateTime64 - INTERVAL '1 YEAR 4 MONTH 1 SECOND'; +WITH - INTERVAL 1 SECOND - INTERVAL 1 SECOND - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH - INTERVAL 1 HOUR - INTERVAL 1 SECOND - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH - INTERVAL 1 SECOND - INTERVAL 1 HOUR - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH - INTERVAL 1 SECOND - INTERVAL 1 SECOND - INTERVAL 1 HOUR as expr SELECT expr, toTypeName(expr); + +SELECT '---'; + +WITH '2022-10-11'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH AS e1, + '2022-10-11'::Date + (INTERVAL 1 DAY + INTERVAL 1 MONTH) AS e2, + '2022-10-11'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH) AS e3, + '2022-10-11'::Date + INTERVAL '1 DAY 1 MONTH' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4; + +WITH '2022-10-11'::Date + INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR AS e1, + '2022-10-11'::Date + (INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR) AS e2, + '2022-10-11'::Date + (INTERVAL -1 SECOND, INTERVAL 2 MINUTE, INTERVAL -3 MONTH, INTERVAL 1 YEAR) AS e3, + '2022-10-11'::Date + INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4; + +WITH '2022-10-11'::DateTime - INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR AS e1, + '2022-10-11'::DateTime + (- INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR) AS e2, + '2022-10-11'::DateTime - (INTERVAL 1 QUARTER, INTERVAL -3 WEEK, INTERVAL 1 YEAR, INTERVAL 1 HOUR) AS e3, + '2022-10-11'::DateTime - INTERVAL '1 QUARTER -3 WEEK 1 YEAR 1 HOUR' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4; + + +WITH '2022-10-11'::DateTime64 - INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND AS e1, + '2022-10-11'::DateTime64 + (- INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND) AS e2, + '2022-10-11'::DateTime64 - (INTERVAL 1 YEAR, INTERVAL 4 MONTH, INTERVAL 1 SECOND) AS e3, + '2022-10-11'::DateTime64 - INTERVAL '1 YEAR 4 MONTH 1 SECOND' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4; \ No newline at end of file From cf6471c6b9965a82acb16b4a576e5f8a1ecf123d Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 14 Oct 2022 02:48:04 +0000 Subject: [PATCH 038/151] Fix style --- src/Functions/FunctionBinaryArithmetic.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index e4919d3e9d7..c13cc67a1bc 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -749,7 +749,7 @@ class FunctionBinaryArithmetic : public IFunction getFunctionForMergeIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { /// Special case when the function is plus or minus, first argument is Interval or Tuple of Intervals - /// and the second argument is the Inteval of a different kind. + /// and the second argument is the Interval of a different kind. /// We construct another function (example: addIntervals) and call it if constexpr (!is_plus && !is_minus) From 8ebe01d9cf1807fcfe782de6efb61142a81e6d5b Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Fri, 14 Oct 2022 11:33:56 +0800 Subject: [PATCH 039/151] system.detached_parts add column bytes_on_disk and path --- .../MergeTree/DataPartStorageOnDisk.cpp | 5 +++++ .../MergeTree/DataPartStorageOnDisk.h | 4 ++++ .../System/StorageSystemDetachedParts.cpp | 20 ++++++++++++------- 3 files changed, 22 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index e2a2f3f793f..e50cee4b654 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -121,6 +121,11 @@ static UInt64 calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & return res; } +UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from) +{ + return calculateTotalSizeOnDiskImpl(disk, from); +} + UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk() const { return calculateTotalSizeOnDiskImpl(volume->getDisk(), fs::path(root_path) / part_dir); diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index adf1b78cdfb..0ad4e8f5239 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -112,6 +112,9 @@ public: void changeRootPath(const std::string & from_root, const std::string & to_root) override; DataPartStorageBuilderPtr getBuilder() const override; + + static UInt64 calculateTotalSizeOnDisk(const DiskPtr &disk, const String &from); + private: VolumePtr volume; std::string root_path; @@ -127,6 +130,7 @@ private: MergeTreeDataPartState state, Poco::Logger * log, bool is_projection) const; + }; class DataPartStorageBuilderOnDisk final : public IDataPartStorageBuilder diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 574ce4f44c2..9220a053e04 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -16,15 +17,17 @@ StorageSystemDetachedParts::StorageSystemDetachedParts(const StorageID & table_i { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription{{ - {"database", std::make_shared()}, - {"table", std::make_shared()}, - {"partition_id", std::make_shared(std::make_shared())}, - {"name", std::make_shared()}, - {"disk", std::make_shared()}, - {"reason", std::make_shared(std::make_shared())}, + {"database", std::make_shared()}, + {"table", std::make_shared()}, + {"partition_id", std::make_shared(std::make_shared())}, + {"name", std::make_shared()}, + {"bytes_on_disk", std::make_shared()}, + {"disk", std::make_shared()}, + {"path", std::make_shared()}, + {"reason", std::make_shared(std::make_shared())}, {"min_block_number", std::make_shared(std::make_shared())}, {"max_block_number", std::make_shared(std::make_shared())}, - {"level", std::make_shared(std::make_shared())} + {"level", std::make_shared(std::make_shared())} }}); setInMemoryMetadata(storage_metadata); } @@ -50,11 +53,14 @@ Pipe StorageSystemDetachedParts::read( for (const auto & p : parts) { size_t i = 0; + String detached_part_path = fs::path(MergeTreeData::DETACHED_DIR_NAME) / p.dir_name; new_columns[i++]->insert(info.database); new_columns[i++]->insert(info.table); new_columns[i++]->insert(p.valid_name ? p.partition_id : Field()); new_columns[i++]->insert(p.dir_name); + new_columns[i++]->insert(DataPartStorageOnDisk::calculateTotalSizeOnDisk(p.disk, fs::path(info.data->getRelativeDataPath()) / detached_part_path)); new_columns[i++]->insert(p.disk->getName()); + new_columns[i++]->insert((fs::path(info.data->getFullPathOnDisk(p.disk)) / detached_part_path).string()); new_columns[i++]->insert(p.valid_name ? p.prefix : Field()); new_columns[i++]->insert(p.valid_name ? p.min_block : Field()); new_columns[i++]->insert(p.valid_name ? p.max_block : Field()); From 555d2759202eaa1d08ba3c54dcc99507e188c069 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 14 Oct 2022 15:18:53 +0200 Subject: [PATCH 040/151] Update src/Storages/MergeTree/DataPartStorageOnDisk.h --- src/Storages/MergeTree/DataPartStorageOnDisk.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index 0ad4e8f5239..79988e9baab 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -113,7 +113,7 @@ public: DataPartStorageBuilderPtr getBuilder() const override; - static UInt64 calculateTotalSizeOnDisk(const DiskPtr &disk, const String &from); + static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from); private: VolumePtr volume; From c11e4bfbbfbc36999814e5983e19e41b1eecd2ea Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 14 Oct 2022 15:13:26 +0000 Subject: [PATCH 041/151] Fix build --- src/Functions/vectorFunctions.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index 007875a8b81..78f35342d60 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -555,7 +555,7 @@ public: { const auto & cur_types = tuple->getElements(); - for (auto & type : cur_types) + for (const auto & type : cur_types) if (!isInterval(type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of Tuple element of first argument of function {}, must be Interval", @@ -607,7 +607,7 @@ public: { const auto & cur_types = first_tuple->getElements(); - for (auto & type : cur_types) + for (const auto & type : cur_types) if (!isInterval(type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of Tuple element of first argument of function {}, must be Interval", From 65a0b9fcc60d0254e5f2d2b020af73c1b03b3b41 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Fri, 14 Oct 2022 23:32:13 +0800 Subject: [PATCH 042/151] fix test --- .../00502_custom_partitioning_local.reference | 2 +- .../00502_custom_partitioning_local.sql | 2 +- .../02117_show_create_table_system.reference | 133 +----------------- 3 files changed, 4 insertions(+), 133 deletions(-) diff --git a/tests/queries/0_stateless/00502_custom_partitioning_local.reference b/tests/queries/0_stateless/00502_custom_partitioning_local.reference index fff28819e74..be02d3eeefc 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_local.reference +++ b/tests/queries/0_stateless/00502_custom_partitioning_local.reference @@ -9,7 +9,7 @@ Sum before DETACH PARTITION: Sum after DETACH PARTITION: 0 system.detached_parts after DETACH PARTITION: -default not_partitioned all all_1_2_1 1 2 1 +default not_partitioned all all_1_2_1 324 default 1 2 1 default *** Partitioned by week *** Parts before OPTIMIZE: 1999-12-27 19991227_1_1_0 diff --git a/tests/queries/0_stateless/00502_custom_partitioning_local.sql b/tests/queries/0_stateless/00502_custom_partitioning_local.sql index c85a978af68..df4785af90d 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_local.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_local.sql @@ -18,7 +18,7 @@ ALTER TABLE not_partitioned DETACH PARTITION ID 'all'; SELECT 'Sum after DETACH PARTITION:'; SELECT sum(x) FROM not_partitioned; SELECT 'system.detached_parts after DETACH PARTITION:'; -SELECT system.detached_parts.* EXCEPT disk FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; +SELECT system.detached_parts.* EXCEPT `path`, disk FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; DROP TABLE not_partitioned; diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index ad27b86c6f5..ee2ccc8b7d1 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -139,7 +139,9 @@ CREATE TABLE system.detached_parts `table` String, `partition_id` Nullable(String), `name` String, + `bytes_on_disk` UInt64, `disk` String, + `path` String, `reason` Nullable(String), `min_block_number` Nullable(Int64), `max_block_number` Nullable(Int64), @@ -974,134 +976,3 @@ CREATE TABLE system.settings_profiles ) ENGINE = SystemSettingsProfiles COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.stack_trace -( - `thread_name` String, - `thread_id` UInt64, - `query_id` String, - `trace` Array(UInt64) -) -ENGINE = SystemStackTrace -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.storage_policies -( - `policy_name` String, - `volume_name` String, - `volume_priority` UInt64, - `disks` Array(String), - `volume_type` String, - `max_data_part_size` UInt64, - `move_factor` Float32, - `prefer_not_to_merge` UInt8 -) -ENGINE = SystemStoragePolicies -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.table_engines -( - `name` String, - `supports_settings` UInt8, - `supports_skipping_indices` UInt8, - `supports_projections` UInt8, - `supports_sort_order` UInt8, - `supports_ttl` UInt8, - `supports_replication` UInt8, - `supports_deduplication` UInt8, - `supports_parallel_insert` UInt8 -) -ENGINE = SystemTableEngines -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.table_functions -( - `name` String, - `description` String -) -ENGINE = SystemTableFunctions -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.tables -( - `database` String, - `name` String, - `uuid` UUID, - `engine` String, - `is_temporary` UInt8, - `data_paths` Array(String), - `metadata_path` String, - `metadata_modification_time` DateTime, - `dependencies_database` Array(String), - `dependencies_table` Array(String), - `create_table_query` String, - `engine_full` String, - `as_select` String, - `partition_key` String, - `sorting_key` String, - `primary_key` String, - `sampling_key` String, - `storage_policy` String, - `total_rows` Nullable(UInt64), - `total_bytes` Nullable(UInt64), - `lifetime_rows` Nullable(UInt64), - `lifetime_bytes` Nullable(UInt64), - `comment` String, - `has_own_data` UInt8, - `loading_dependencies_database` Array(String), - `loading_dependencies_table` Array(String), - `loading_dependent_database` Array(String), - `loading_dependent_table` Array(String), - `table` String -) -ENGINE = SystemTables -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.time_zones -( - `time_zone` String -) -ENGINE = SystemTimeZones -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.user_directories -( - `name` String, - `type` String, - `params` String, - `precedence` UInt64 -) -ENGINE = SystemUserDirectories -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.users -( - `name` String, - `id` UUID, - `storage` String, - `auth_type` Enum8('no_password' = 0, 'plaintext_password' = 1, 'sha256_password' = 2, 'double_sha1_password' = 3, 'ldap' = 4, 'kerberos' = 5, 'ssl_certificate' = 6), - `auth_params` String, - `host_ip` Array(String), - `host_names` Array(String), - `host_names_regexp` Array(String), - `host_names_like` Array(String), - `default_roles_all` UInt8, - `default_roles_list` Array(String), - `default_roles_except` Array(String), - `grantees_any` UInt8, - `grantees_list` Array(String), - `grantees_except` Array(String), - `default_database` String -) -ENGINE = SystemUsers -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.warnings -( - `message` String -) -ENGINE = SystemWarnings -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.zeros -( - `zero` UInt8 -) -ENGINE = SystemZeros -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.zeros_mt -( - `zero` UInt8 -) -ENGINE = SystemZeros -COMMENT 'SYSTEM TABLE is built on the fly.' From 66f6c0a68322f182c80fb86ad0ad9fc646a25ae5 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Sat, 15 Oct 2022 22:43:55 +0800 Subject: [PATCH 043/151] fix test --- .../02117_show_create_table_system.reference | 131 ++++++++++++++++++ 1 file changed, 131 insertions(+) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index ee2ccc8b7d1..4d705e4b0d1 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -976,3 +976,134 @@ CREATE TABLE system.settings_profiles ) ENGINE = SystemSettingsProfiles COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.stack_trace +( + `thread_name` String, + `thread_id` UInt64, + `query_id` String, + `trace` Array(UInt64) +) +ENGINE = SystemStackTrace +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.storage_policies +( + `policy_name` String, + `volume_name` String, + `volume_priority` UInt64, + `disks` Array(String), + `volume_type` String, + `max_data_part_size` UInt64, + `move_factor` Float32, + `prefer_not_to_merge` UInt8 +) +ENGINE = SystemStoragePolicies +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.table_engines +( + `name` String, + `supports_settings` UInt8, + `supports_skipping_indices` UInt8, + `supports_projections` UInt8, + `supports_sort_order` UInt8, + `supports_ttl` UInt8, + `supports_replication` UInt8, + `supports_deduplication` UInt8, + `supports_parallel_insert` UInt8 +) +ENGINE = SystemTableEngines +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.table_functions +( + `name` String, + `description` String +) +ENGINE = SystemTableFunctions +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.tables +( + `database` String, + `name` String, + `uuid` UUID, + `engine` String, + `is_temporary` UInt8, + `data_paths` Array(String), + `metadata_path` String, + `metadata_modification_time` DateTime, + `dependencies_database` Array(String), + `dependencies_table` Array(String), + `create_table_query` String, + `engine_full` String, + `as_select` String, + `partition_key` String, + `sorting_key` String, + `primary_key` String, + `sampling_key` String, + `storage_policy` String, + `total_rows` Nullable(UInt64), + `total_bytes` Nullable(UInt64), + `lifetime_rows` Nullable(UInt64), + `lifetime_bytes` Nullable(UInt64), + `comment` String, + `has_own_data` UInt8, + `loading_dependencies_database` Array(String), + `loading_dependencies_table` Array(String), + `loading_dependent_database` Array(String), + `loading_dependent_table` Array(String), + `table` String +) +ENGINE = SystemTables +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.time_zones +( + `time_zone` String +) +ENGINE = SystemTimeZones +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.user_directories +( + `name` String, + `type` String, + `params` String, + `precedence` UInt64 +) +ENGINE = SystemUserDirectories +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.users +( + `name` String, + `id` UUID, + `storage` String, + `auth_type` Enum8('no_password' = 0, 'plaintext_password' = 1, 'sha256_password' = 2, 'double_sha1_password' = 3, 'ldap' = 4, 'kerberos' = 5, 'ssl_certificate' = 6), + `auth_params` String, + `host_ip` Array(String), + `host_names` Array(String), + `host_names_regexp` Array(String), + `host_names_like` Array(String), + `default_roles_all` UInt8, + `default_roles_list` Array(String), + `default_roles_except` Array(String), + `grantees_any` UInt8, + `grantees_list` Array(String), + `grantees_except` Array(String), + `default_database` String +) +ENGINE = SystemUsers +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.warnings +( + `message` String +) +ENGINE = SystemWarnings +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.zeros +( + `zero` UInt8 +) +ENGINE = SystemZeros +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.zeros_mt +( + `zero` UInt8 +) +ENGINE = SystemZeros +COMMENT 'SYSTEM TABLE is built on the fly.' From 2a356715da9a44f9273daab8fc3e6e71c9a52d08 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Sun, 16 Oct 2022 16:46:00 +0800 Subject: [PATCH 044/151] fix test --- src/Storages/MergeTree/DataPartStorageOnDisk.cpp | 13 ++++++++++++- tests/integration/test_partition/test.py | 2 +- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index e50cee4b654..8ceebae333c 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -112,7 +112,18 @@ UInt32 DataPartStorageOnDisk::getRefCount(const String & file_name) const static UInt64 calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & from) { if (disk->isFile(from)) - return disk->getFileSize(from); + { + try + { + return disk->getFileSize(from); + } + catch (...) + { + /// Files of detached part may be not exist, and then set file size is 0. + return 0; + } + } + std::vector files; disk->listFiles(from, files); UInt64 res = 0; diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index f3df66631a5..18e59becdd4 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -379,7 +379,7 @@ def test_system_detached_parts(drop_detached_parts_table): ) res = q( - "select * from system.detached_parts where table like 'sdp_%' order by table, name" + "select system.detached_parts.* except `path`, disk from system.detached_parts where table like 'sdp_%' order by table, name" ) assert ( res == "default\tsdp_0\tall\tall_1_1_0\tdefault\t\t1\t1\t0\n" From 9ec206f8f74a6e6fb94a581dee30dc87b7bcd7d8 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Sun, 16 Oct 2022 19:52:53 +0800 Subject: [PATCH 045/151] fix test --- .../MergeTree/DataPartStorageOnDisk.cpp | 25 +++++++++---------- .../00502_custom_partitioning_local.reference | 2 +- .../00502_custom_partitioning_local.sql | 2 +- 3 files changed, 14 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 8ceebae333c..600544b0b20 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -112,18 +112,7 @@ UInt32 DataPartStorageOnDisk::getRefCount(const String & file_name) const static UInt64 calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & from) { if (disk->isFile(from)) - { - try - { - return disk->getFileSize(from); - } - catch (...) - { - /// Files of detached part may be not exist, and then set file size is 0. - return 0; - } - } - + return disk->getFileSize(from); std::vector files; disk->listFiles(from, files); UInt64 res = 0; @@ -134,7 +123,17 @@ static UInt64 calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from) { - return calculateTotalSizeOnDiskImpl(disk, from); + try + { + /// Files of detached part may be not exist, and then set file size is 0. + if (!disk->exists(from)) + return 0; + return calculateTotalSizeOnDiskImpl(disk, from); + } + catch (...) + { + return 0; + } } UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk() const diff --git a/tests/queries/0_stateless/00502_custom_partitioning_local.reference b/tests/queries/0_stateless/00502_custom_partitioning_local.reference index be02d3eeefc..226a39f628e 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_local.reference +++ b/tests/queries/0_stateless/00502_custom_partitioning_local.reference @@ -9,7 +9,7 @@ Sum before DETACH PARTITION: Sum after DETACH PARTITION: 0 system.detached_parts after DETACH PARTITION: -default not_partitioned all all_1_2_1 324 default 1 2 1 default +default not_partitioned all all_1_2_1 324 1 2 1 *** Partitioned by week *** Parts before OPTIMIZE: 1999-12-27 19991227_1_1_0 diff --git a/tests/queries/0_stateless/00502_custom_partitioning_local.sql b/tests/queries/0_stateless/00502_custom_partitioning_local.sql index df4785af90d..87a6331402b 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_local.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_local.sql @@ -18,7 +18,7 @@ ALTER TABLE not_partitioned DETACH PARTITION ID 'all'; SELECT 'Sum after DETACH PARTITION:'; SELECT sum(x) FROM not_partitioned; SELECT 'system.detached_parts after DETACH PARTITION:'; -SELECT system.detached_parts.* EXCEPT `path`, disk FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; +SELECT system.detached_parts.* EXCEPT (`path`, disk) FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; DROP TABLE not_partitioned; From 9e62570d61a3ece8e32468a2479b0ac047051602 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Mon, 17 Oct 2022 22:57:11 +0800 Subject: [PATCH 046/151] fix test --- src/Storages/MergeTree/DataPartStorageOnDisk.cpp | 12 +----------- src/Storages/System/StorageSystemDetachedParts.cpp | 13 ++++++++++++- .../00502_custom_partitioning_local.reference | 2 +- .../0_stateless/00502_custom_partitioning_local.sql | 2 +- 4 files changed, 15 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 600544b0b20..e50cee4b654 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -123,17 +123,7 @@ static UInt64 calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from) { - try - { - /// Files of detached part may be not exist, and then set file size is 0. - if (!disk->exists(from)) - return 0; - return calculateTotalSizeOnDiskImpl(disk, from); - } - catch (...) - { - return 0; - } + return calculateTotalSizeOnDiskImpl(disk, from); } UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk() const diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 9220a053e04..1bc55c922c1 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -58,7 +58,18 @@ Pipe StorageSystemDetachedParts::read( new_columns[i++]->insert(info.table); new_columns[i++]->insert(p.valid_name ? p.partition_id : Field()); new_columns[i++]->insert(p.dir_name); - new_columns[i++]->insert(DataPartStorageOnDisk::calculateTotalSizeOnDisk(p.disk, fs::path(info.data->getRelativeDataPath()) / detached_part_path)); + + size_t bytes_on_disk = 0; + try + { + /// Files of detached part may be not exist, and then set file size is 0. + bytes_on_disk = DataPartStorageOnDisk::calculateTotalSizeOnDisk( + p.disk, fs::path(info.data->getRelativeDataPath()) / detached_part_path); + } + catch (...) + {} + + new_columns[i++]->insert(bytes_on_disk); new_columns[i++]->insert(p.disk->getName()); new_columns[i++]->insert((fs::path(info.data->getFullPathOnDisk(p.disk)) / detached_part_path).string()); new_columns[i++]->insert(p.valid_name ? p.prefix : Field()); diff --git a/tests/queries/0_stateless/00502_custom_partitioning_local.reference b/tests/queries/0_stateless/00502_custom_partitioning_local.reference index 226a39f628e..fff28819e74 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_local.reference +++ b/tests/queries/0_stateless/00502_custom_partitioning_local.reference @@ -9,7 +9,7 @@ Sum before DETACH PARTITION: Sum after DETACH PARTITION: 0 system.detached_parts after DETACH PARTITION: -default not_partitioned all all_1_2_1 324 1 2 1 +default not_partitioned all all_1_2_1 1 2 1 *** Partitioned by week *** Parts before OPTIMIZE: 1999-12-27 19991227_1_1_0 diff --git a/tests/queries/0_stateless/00502_custom_partitioning_local.sql b/tests/queries/0_stateless/00502_custom_partitioning_local.sql index 87a6331402b..3d5f71429fe 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_local.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_local.sql @@ -18,7 +18,7 @@ ALTER TABLE not_partitioned DETACH PARTITION ID 'all'; SELECT 'Sum after DETACH PARTITION:'; SELECT sum(x) FROM not_partitioned; SELECT 'system.detached_parts after DETACH PARTITION:'; -SELECT system.detached_parts.* EXCEPT (`path`, disk) FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; +SELECT system.detached_parts.* EXCEPT (bytes_on_disk, `path`, disk) FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; DROP TABLE not_partitioned; From b26361c6476ec8ae4081da877b8d5b3f54c4f6e3 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Mon, 17 Oct 2022 23:02:39 +0800 Subject: [PATCH 047/151] fix test --- src/Storages/System/StorageSystemDetachedParts.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 1bc55c922c1..53526a02459 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -59,7 +59,7 @@ Pipe StorageSystemDetachedParts::read( new_columns[i++]->insert(p.valid_name ? p.partition_id : Field()); new_columns[i++]->insert(p.dir_name); - size_t bytes_on_disk = 0; + UInt64 bytes_on_disk = 0; try { /// Files of detached part may be not exist, and then set file size is 0. From 10acf43db4d863e8991c5195dfc37bb069d78c56 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Tue, 18 Oct 2022 09:20:59 +0800 Subject: [PATCH 048/151] fix test --- tests/integration/test_partition/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index 18e59becdd4..f333011fa37 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -379,7 +379,7 @@ def test_system_detached_parts(drop_detached_parts_table): ) res = q( - "select system.detached_parts.* except `path`, disk from system.detached_parts where table like 'sdp_%' order by table, name" + "select system.detached_parts.* except (bytes_on_disk, `path`, disk) from system.detached_parts where table like 'sdp_%' order by table, name" ) assert ( res == "default\tsdp_0\tall\tall_1_1_0\tdefault\t\t1\t1\t0\n" From 163b3c9f5263e810aba2998487af45839d5432e6 Mon Sep 17 00:00:00 2001 From: chen Date: Tue, 18 Oct 2022 14:53:21 +0800 Subject: [PATCH 049/151] fix test fix test --- tests/integration/test_partition/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index f333011fa37..03c4ffbbd36 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -379,7 +379,7 @@ def test_system_detached_parts(drop_detached_parts_table): ) res = q( - "select system.detached_parts.* except (bytes_on_disk, `path`, disk) from system.detached_parts where table like 'sdp_%' order by table, name" + "select system.detached_parts.* except (bytes_on_disk, `path`) from system.detached_parts where table like 'sdp_%' order by table, name" ) assert ( res == "default\tsdp_0\tall\tall_1_1_0\tdefault\t\t1\t1\t0\n" From a50c0a7f85f7cef911a946d4ba9a54b61023ad97 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 18 Oct 2022 07:23:00 +0000 Subject: [PATCH 050/151] Better test --- .../02457_tuple_of_intervals.reference | 21 +++++++--- .../0_stateless/02457_tuple_of_intervals.sql | 42 ++++++++++++++----- 2 files changed, 48 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.reference b/tests/queries/0_stateless/02457_tuple_of_intervals.reference index dd190dce891..d5ffbc33dc0 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.reference +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.reference @@ -1,16 +1,27 @@ SELECT (toIntervalSecond(-1), toIntervalMinute(2), toIntervalMonth(-3), toIntervalYear(1)) --- +-1 +2022-10-12 +2022-10-10 +(2) +(0) +2022-10-12 +2022-10-10 +2022-10-12 +(2) Tuple(IntervalSecond) +(0) Tuple(IntervalSecond) +--- 3 IntervalSecond (1,2) Tuple(IntervalHour, IntervalSecond) (1,1,1) Tuple(IntervalSecond, IntervalHour, IntervalSecond) (2,1) Tuple(IntervalSecond, IntervalHour) ---- -3 IntervalSecond (-1,-2) Tuple(IntervalHour, IntervalSecond) (-1,-1,-1) Tuple(IntervalSecond, IntervalHour, IntervalSecond) (-2,-1) Tuple(IntervalSecond, IntervalHour) --- -1 -1 -1 -1 +1 2022-03-01 +1 2022-02-28 +1 2023-07-11 00:01:59 +1 2021-07-31 23:00:00 +1 2021-06-10 23:59:59.000 diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.sql b/tests/queries/0_stateless/02457_tuple_of_intervals.sql index d4065ab98f8..494914d4d4f 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.sql +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.sql @@ -2,13 +2,29 @@ EXPLAIN SYNTAX SELECT INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; SELECT '---'; +SELECT negate(INTERVAL 1 SECOND); +SELECT addTupleOfIntervals('2022-10-11'::Date, tuple(INTERVAL 1 DAY)); +SELECT subtractTupleOfIntervals('2022-10-11'::Date, tuple(INTERVAL 1 DAY)); +SELECT addInterval(tuple(INTERVAL 1 SECOND), INTERVAL 1 SECOND); +SELECT subtractInterval(tuple(INTERVAL 1 SECOND), INTERVAL 1 SECOND); + +SELECT '2022-10-11'::Date + tuple(INTERVAL 1 DAY); +SELECT '2022-10-11'::Date - tuple(INTERVAL 1 DAY); +SELECT tuple(INTERVAL 1 DAY) + '2022-10-11'::Date; +SELECT tuple(INTERVAL 1 DAY) - '2022-10-11'::Date; -- { serverError 43 } + +WITH tuple(INTERVAL 1 SECOND) + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH tuple(INTERVAL 1 SECOND) - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 SECOND + tuple(INTERVAL 1 SECOND) as expr SELECT expr, toTypeName(expr); -- { serverError 43 } +WITH INTERVAL 1 SECOND - tuple(INTERVAL 1 SECOND) as expr SELECT expr, toTypeName(expr); -- { serverError 43 } + +SELECT '---'; + WITH INTERVAL 1 SECOND + INTERVAL 1 SECOND + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); WITH INTERVAL 1 HOUR + INTERVAL 1 SECOND + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); WITH INTERVAL 1 SECOND + INTERVAL 1 HOUR + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); WITH INTERVAL 1 SECOND + INTERVAL 1 SECOND + INTERVAL 1 HOUR as expr SELECT expr, toTypeName(expr); -SELECT '---'; - WITH - INTERVAL 1 SECOND - INTERVAL 1 SECOND - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); WITH - INTERVAL 1 HOUR - INTERVAL 1 SECOND - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); WITH - INTERVAL 1 SECOND - INTERVAL 1 HOUR - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); @@ -16,27 +32,33 @@ WITH - INTERVAL 1 SECOND - INTERVAL 1 SECOND - INTERVAL 1 HOUR as expr SELECT ex SELECT '---'; -WITH '2022-10-11'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH AS e1, - '2022-10-11'::Date + (INTERVAL 1 DAY + INTERVAL 1 MONTH) AS e2, - '2022-10-11'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH) AS e3, - '2022-10-11'::Date + INTERVAL '1 DAY 1 MONTH' AS e4 -SELECT e1 == e2 AND e2 == e3 AND e3 == e4; +WITH '2022-01-30'::Date + INTERVAL 1 MONTH + INTERVAL 1 DAY AS e1, + '2022-01-30'::Date + (INTERVAL 1 MONTH + INTERVAL 1 DAY) AS e2, + '2022-01-30'::Date + (INTERVAL 1 MONTH, INTERVAL 1 DAY) AS e3, + '2022-01-30'::Date + INTERVAL '1 MONTH 1 DAY' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; + +WITH '2022-01-30'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH AS e1, + '2022-01-30'::Date + (INTERVAL 1 DAY + INTERVAL 1 MONTH) AS e2, + '2022-01-30'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH) AS e3, + '2022-01-30'::Date + INTERVAL '1 DAY 1 MONTH' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; WITH '2022-10-11'::Date + INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR AS e1, '2022-10-11'::Date + (INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR) AS e2, '2022-10-11'::Date + (INTERVAL -1 SECOND, INTERVAL 2 MINUTE, INTERVAL -3 MONTH, INTERVAL 1 YEAR) AS e3, '2022-10-11'::Date + INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR' AS e4 -SELECT e1 == e2 AND e2 == e3 AND e3 == e4; +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; WITH '2022-10-11'::DateTime - INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR AS e1, '2022-10-11'::DateTime + (- INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR) AS e2, '2022-10-11'::DateTime - (INTERVAL 1 QUARTER, INTERVAL -3 WEEK, INTERVAL 1 YEAR, INTERVAL 1 HOUR) AS e3, '2022-10-11'::DateTime - INTERVAL '1 QUARTER -3 WEEK 1 YEAR 1 HOUR' AS e4 -SELECT e1 == e2 AND e2 == e3 AND e3 == e4; +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; WITH '2022-10-11'::DateTime64 - INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND AS e1, '2022-10-11'::DateTime64 + (- INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND) AS e2, '2022-10-11'::DateTime64 - (INTERVAL 1 YEAR, INTERVAL 4 MONTH, INTERVAL 1 SECOND) AS e3, '2022-10-11'::DateTime64 - INTERVAL '1 YEAR 4 MONTH 1 SECOND' AS e4 -SELECT e1 == e2 AND e2 == e3 AND e3 == e4; \ No newline at end of file +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; \ No newline at end of file From 32ad28cbe507aadea9f16df70d01129b1ef15d0b Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Wed, 19 Oct 2022 17:37:26 +0800 Subject: [PATCH 051/151] optimize code --- .../MergeTree/DataPartStorageOnDisk.cpp | 5 -- .../MergeTree/DataPartStorageOnDisk.h | 2 - .../System/StorageSystemDetachedParts.cpp | 47 ++++++++++++++----- 3 files changed, 35 insertions(+), 19 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index e50cee4b654..e2a2f3f793f 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -121,11 +121,6 @@ static UInt64 calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & return res; } -UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from) -{ - return calculateTotalSizeOnDiskImpl(disk, from); -} - UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk() const { return calculateTotalSizeOnDiskImpl(volume->getDisk(), fs::path(root_path) / part_dir); diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index 79988e9baab..91ed8bbfb43 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -113,8 +113,6 @@ public: DataPartStorageBuilderPtr getBuilder() const override; - static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from); - private: VolumePtr volume; std::string root_path; diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 53526a02459..980529032c4 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -31,6 +31,40 @@ StorageSystemDetachedParts::StorageSystemDetachedParts(const StorageID & table_i }}); setInMemoryMetadata(storage_metadata); } +static void calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & from, UInt64 & total_size) +{ + /// Files or directories of detached part may not exist. Only count the size of existing files. + if (disk->isFile(from)) + { + try + { + total_size += disk->getFileSize(from); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + return; + } + std::vector files; + try + { + disk->listFiles(from, files); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + for (const auto & file : files) + calculateTotalSizeOnDiskImpl(disk, fs::path(from) / file, total_size); +} + +static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from) +{ + UInt64 total_size = 0; + calculateTotalSizeOnDiskImpl(disk, from, total_size); + return total_size; +} Pipe StorageSystemDetachedParts::read( const Names & /* column_names */, @@ -58,18 +92,7 @@ Pipe StorageSystemDetachedParts::read( new_columns[i++]->insert(info.table); new_columns[i++]->insert(p.valid_name ? p.partition_id : Field()); new_columns[i++]->insert(p.dir_name); - - UInt64 bytes_on_disk = 0; - try - { - /// Files of detached part may be not exist, and then set file size is 0. - bytes_on_disk = DataPartStorageOnDisk::calculateTotalSizeOnDisk( - p.disk, fs::path(info.data->getRelativeDataPath()) / detached_part_path); - } - catch (...) - {} - - new_columns[i++]->insert(bytes_on_disk); + new_columns[i++]->insert(calculateTotalSizeOnDisk(p.disk, fs::path(info.data->getRelativeDataPath()) / detached_part_path)); new_columns[i++]->insert(p.disk->getName()); new_columns[i++]->insert((fs::path(info.data->getFullPathOnDisk(p.disk)) / detached_part_path).string()); new_columns[i++]->insert(p.valid_name ? p.prefix : Field()); From 330777a0ccf787eeff887a72892cf163ff0efb1c Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Wed, 19 Oct 2022 17:44:36 +0800 Subject: [PATCH 052/151] fix --- src/Storages/MergeTree/DataPartStorageOnDisk.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index 91ed8bbfb43..adf1b78cdfb 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -112,7 +112,6 @@ public: void changeRootPath(const std::string & from_root, const std::string & to_root) override; DataPartStorageBuilderPtr getBuilder() const override; - private: VolumePtr volume; std::string root_path; @@ -128,7 +127,6 @@ private: MergeTreeDataPartState state, Poco::Logger * log, bool is_projection) const; - }; class DataPartStorageBuilderOnDisk final : public IDataPartStorageBuilder From 39c88c74e84b1109015ad5b80e164bf57799e3ba Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Sat, 22 Oct 2022 22:31:17 +0800 Subject: [PATCH 053/151] check whether last manually created snapshot is done --- contrib/NuRaft | 2 +- docs/en/operations/clickhouse-keeper.md | 8 +++++++- src/Coordination/FourLetterCommand.cpp | 7 ++++++- src/Coordination/FourLetterCommand.h | 14 ++++++++++++++ src/Coordination/KeeperDispatcher.h | 6 ++++++ src/Coordination/KeeperServer.cpp | 16 +++++++++++++++- src/Coordination/KeeperServer.h | 6 ++++++ .../test_keeper_four_word_command/test.py | 9 ++++++--- 8 files changed, 61 insertions(+), 7 deletions(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 1be805e7cb2..e4e746a24eb 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 1be805e7cb2494aa8170015493474379b0362dfc +Subproject commit e4e746a24eb56861a86f3672771e3308d8c40722 diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 03eddd4f6ed..66b4685bff5 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -309,12 +309,18 @@ Sessions with Ephemerals (1): /clickhouse/task_queue/ddl ``` -- `csnp`: Schedule a snapshot creation task. Return `"Snapshot creation scheduled."` if successfully scheduled or Fail to scheduled snapshot creation.` if failed. +- `csnp`: Schedule a snapshot creation task. Return `Snapshot creation scheduled.` if successfully scheduled or `Fail to scheduled snapshot creation.` if failed. ``` Snapshot creation scheduled. ``` +- `snpd`: Whether the last successfully scheduled snapshot creation is done. Return `Yes` if true or `No` if false. + +``` +Yes +``` + ## [experimental] Migration from ZooKeeper {#migration-from-zookeeper} Seamlessly migration from ZooKeeper to ClickHouse Keeper is impossible you have to stop your ZooKeeper cluster, convert data and start ClickHouse Keeper. `clickhouse-keeper-converter` tool allows converting ZooKeeper logs and snapshots to ClickHouse Keeper snapshot. It works only with ZooKeeper > 3.4. Steps for migration: diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 70009703c5a..3d1077ea84c 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -477,7 +477,12 @@ String ApiVersionCommand::run() String CreateSnapshotCommand::run() { - return keeper_dispatcher.createSnapshot() ? "Snapshot creation scheduled." : "Fail to scheduled snapshot creation."; + return keeper_dispatcher.createSnapshot() ? "Snapshot creation scheduled." : "Fail to scheduled snapshot creation task."; +} + +String CheckSnapshotDoneCommand::run() +{ + return keeper_dispatcher.snapshotDone() ? "Snapshot creation done." : "Fail to scheduled snapshot creation task."; } } diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index 5001a750d66..28f1d7f153f 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -340,4 +340,18 @@ struct CreateSnapshotCommand : public IFourLetterCommand String run() override; ~CreateSnapshotCommand() override = default; }; + +/// Check whether last manual snapshot done +struct CheckSnapshotDoneCommand : public IFourLetterCommand +{ + explicit CheckSnapshotDoneCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } + + String name() override { return "snpd"; } + String run() override; + ~CheckSnapshotDoneCommand() override = default; +}; + } diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 79212ea3040..48681957c13 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -209,6 +209,12 @@ public: { return server->createSnapshot(); } + + /// Whether the last manually created snapshot is done + bool snapshotDone() + { + return server->snapshotDone(); + } }; } diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index e0186927b54..87ebea0b4ab 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -114,6 +114,7 @@ KeeperServer::KeeperServer( , is_recovering(config.getBool("keeper_server.force_recovery", false)) , keeper_context{std::make_shared()} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) + , last_manual_snapshot_log_idx(0) { if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); @@ -908,7 +909,20 @@ Keeper4LWInfo KeeperServer::getPartiallyFilled4LWInfo() const bool KeeperServer::createSnapshot() { - return raft_instance->create_snapshot(); + std::lock_guard lock(snapshot_mutex); + if (raft_instance->create_snapshot()) + { + last_manual_snapshot_log_idx = raft_instance->get_last_snapshot_idx(); + LOG_INFO(log, "Successfully schedule a keeper snapshot creation task at log index {}", last_manual_snapshot_log_idx); + return true; + } + return false; +} + +bool KeeperServer::snapshotDone() +{ + std::lock_guard lock(snapshot_mutex); + return last_manual_snapshot_log_idx != 0 && last_manual_snapshot_log_idx == raft_instance->get_last_snapshot_idx(); } } diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index ec832199387..11e3b75d127 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -66,6 +66,10 @@ private: const bool create_snapshot_on_exit; + /// Used to check whether the previous manually created snapshot complete. + uint64_t last_manual_snapshot_log_idx; + std::mutex snapshot_mutex; + public: KeeperServer( const KeeperConfigurationAndSettingsPtr & settings_, @@ -133,6 +137,8 @@ public: bool waitConfigurationUpdate(const ConfigUpdateAction & task); bool createSnapshot(); + + bool snapshotDone(); }; } diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 2b2343757bb..bfe0b2a96e4 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -598,7 +598,7 @@ def test_cmd_wchp(started_cluster): destroy_zk_client(zk) -def test_cmd_csnp(started_cluster): +def test_cmd_snapshot(started_cluster): zk = None try: wait_nodes() @@ -607,7 +607,10 @@ def test_cmd_csnp(started_cluster): zk = get_fake_zk(node1.name, timeout=30.0) - data = send_4lw_cmd(cmd="csnp") - assert data == "Snapshot creation scheduled." + create = send_4lw_cmd(cmd="csnp") + assert create == "Snapshot creation scheduled." + + check = send_4lw_cmd(cmd="snpd") + assert (check == "Yes" or check == "No") finally: destroy_zk_client(zk) From 42e391a0191ef046a499fab5c245e5714475cbe7 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Sat, 22 Oct 2022 22:47:03 +0800 Subject: [PATCH 054/151] fix test --- tests/integration/test_keeper_four_word_command/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index bfe0b2a96e4..4949d6f70de 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -611,6 +611,6 @@ def test_cmd_snapshot(started_cluster): assert create == "Snapshot creation scheduled." check = send_4lw_cmd(cmd="snpd") - assert (check == "Yes" or check == "No") + assert check == "Yes" or check == "No" finally: destroy_zk_client(zk) From 6242e93c81c22a1306c4dd35c263f65e22048fe9 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 24 Oct 2022 02:11:08 +0000 Subject: [PATCH 055/151] Fixes & better tests --- src/Functions/vectorFunctions.cpp | 8 ++++---- .../0_stateless/02457_tuple_of_intervals.reference | 5 +++++ tests/queries/0_stateless/02457_tuple_of_intervals.sql | 7 +++++++ 3 files changed, 16 insertions(+), 4 deletions(-) diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index 78f35342d60..20835f59cc1 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -434,7 +434,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) + if (!isDateOrDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. Should be a date or a date with time", arguments[0].type->getName(), getName()}; @@ -545,7 +545,7 @@ public: if (!isInterval(arguments[1])) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}, must be Interval", - arguments[0]->getName(), getName()); + arguments[1]->getName(), getName()); DataTypes types; @@ -559,7 +559,7 @@ public: if (!isInterval(type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of Tuple element of first argument of function {}, must be Interval", - types.back()->getName(), getName()); + type->getName(), getName()); types = cur_types; } @@ -582,7 +582,7 @@ public: if (!isInterval(arguments[1].type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}, must be Interval", - arguments[0].type->getName(), getName()); + arguments[1].type->getName(), getName()); Columns tuple_columns; diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.reference b/tests/queries/0_stateless/02457_tuple_of_intervals.reference index d5ffbc33dc0..e635aec1163 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.reference +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.reference @@ -5,6 +5,11 @@ SELECT (toIntervalSecond(-1), toIntervalMinute(2), toIntervalMonth(-3), toInterv 2022-10-10 (2) (0) +2022-11-12 +2022-09-10 +(1,2) +(1,0) +--- 2022-10-12 2022-10-10 2022-10-12 diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.sql b/tests/queries/0_stateless/02457_tuple_of_intervals.sql index 494914d4d4f..be9ccb50d92 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.sql +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.sql @@ -8,6 +8,13 @@ SELECT subtractTupleOfIntervals('2022-10-11'::Date, tuple(INTERVAL 1 DAY)); SELECT addInterval(tuple(INTERVAL 1 SECOND), INTERVAL 1 SECOND); SELECT subtractInterval(tuple(INTERVAL 1 SECOND), INTERVAL 1 SECOND); +SELECT addTupleOfIntervals('2022-10-11'::Date, (INTERVAL 1 DAY, INTERVAL 1 MONTH)); +SELECT subtractTupleOfIntervals('2022-10-11'::Date, (INTERVAL 1 DAY, INTERVAL 1 MONTH)); +SELECT addInterval((INTERVAL 1 DAY, INTERVAL 1 SECOND), INTERVAL 1 SECOND); +SELECT subtractInterval(tuple(INTERVAL 1 DAY, INTERVAL 1 SECOND), INTERVAL 1 SECOND); + +SELECT '---'; + SELECT '2022-10-11'::Date + tuple(INTERVAL 1 DAY); SELECT '2022-10-11'::Date - tuple(INTERVAL 1 DAY); SELECT tuple(INTERVAL 1 DAY) + '2022-10-11'::Date; From 61563d4f19b2d358c94e4044f48267ab94160574 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Mon, 24 Oct 2022 11:52:20 +0800 Subject: [PATCH 056/151] better --- .../System/StorageSystemDetachedParts.cpp | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 980529032c4..11c4f7843ac 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -44,19 +44,22 @@ static void calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & fr { tryLogCurrentException(__PRETTY_FUNCTION__); } - return; } - std::vector files; - try + else { - disk->listFiles(from, files); + DirectoryIteratorPtr it; + try + { + it = disk->iterateDirectory(from); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + for (; it->isValid(); it->next()) + calculateTotalSizeOnDiskImpl(disk, fs::path(from) / it->name(), total_size); } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - for (const auto & file : files) - calculateTotalSizeOnDiskImpl(disk, fs::path(from) / file, total_size); } static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from) From 2f30c817bfb51ae47bb7dde8c15f3f4999d0d924 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 24 Oct 2022 17:23:47 +0800 Subject: [PATCH 057/151] little fix --- src/Coordination/KeeperServer.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 87ebea0b4ab..042ab35d709 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -910,10 +910,11 @@ Keeper4LWInfo KeeperServer::getPartiallyFilled4LWInfo() const bool KeeperServer::createSnapshot() { std::lock_guard lock(snapshot_mutex); - if (raft_instance->create_snapshot()) + uint64_t log_idx = raft_instance->create_snapshot(); + if (log_idx != 0) { - last_manual_snapshot_log_idx = raft_instance->get_last_snapshot_idx(); - LOG_INFO(log, "Successfully schedule a keeper snapshot creation task at log index {}", last_manual_snapshot_log_idx); + last_manual_snapshot_log_idx = log_idx; + LOG_INFO(log, "Successfully schedule a keeper snapshot creation task at log index {}", log_idx); return true; } return false; From b5d1c4e6574fbc4916056a3c78ea87aa7c74a3f6 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 24 Oct 2022 20:08:58 +0800 Subject: [PATCH 058/151] replace snpd with lgif --- docs/en/operations/clickhouse-keeper.md | 13 +++++++++---- src/Coordination/FourLetterCommand.cpp | 19 ++++++++++++++++--- src/Coordination/FourLetterCommand.h | 17 ++++++++++++----- src/Coordination/Keeper4LWInfo.h | 22 ++++++++++++++++++++++ src/Coordination/KeeperDispatcher.h | 8 ++++---- src/Coordination/KeeperServer.cpp | 24 +++++++++++++----------- src/Coordination/KeeperServer.h | 8 ++------ 7 files changed, 78 insertions(+), 33 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 66b4685bff5..2ab76e1a1ea 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -309,16 +309,21 @@ Sessions with Ephemerals (1): /clickhouse/task_queue/ddl ``` -- `csnp`: Schedule a snapshot creation task. Return `Snapshot creation scheduled.` if successfully scheduled or `Fail to scheduled snapshot creation.` if failed. +- `csnp`: Schedule a snapshot creation task. Return `Snapshot creation scheduled with last committed log index xxx.` if successfully scheduled or `Fail to scheduled snapshot creation task.` if failed. ``` -Snapshot creation scheduled. +Snapshot creation scheduled with last committed log index 100. ``` -- `snpd`: Whether the last successfully scheduled snapshot creation is done. Return `Yes` if true or `No` if false. +- `lgif`: Keeper log information. `last_log_idx` : my last log index in log store; `last_log_term` : my last log term; `last_committed_log_idx` : my last committed log index in state machine; `leader_committed_log_idx` : leader's committed log index from my perspective; `target_committed_log_idx` : target log index should be committed to; `last_snapshot_idx` : the largest committed log index in last snapshot. ``` -Yes +last_log_idx : 101 +last_log_term : 1 +last_committed_log_idx : 100 +leader_committed_log_idx : 101 +target_committed_log_idx : 101 +last_snapshot_idx : 50 ``` ## [experimental] Migration from ZooKeeper {#migration-from-zookeeper} diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 3d1077ea84c..c5841ce3404 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -139,6 +139,9 @@ void FourLetterCommandFactory::registerCommands(KeeperDispatcher & keeper_dispat FourLetterCommandPtr create_snapshot_command = std::make_shared(keeper_dispatcher); factory.registerCommand(create_snapshot_command); + FourLetterCommandPtr log_info_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(log_info_command); + factory.initializeAllowList(keeper_dispatcher); factory.setInitialize(true); } @@ -477,12 +480,22 @@ String ApiVersionCommand::run() String CreateSnapshotCommand::run() { - return keeper_dispatcher.createSnapshot() ? "Snapshot creation scheduled." : "Fail to scheduled snapshot creation task."; + auto log_index = keeper_dispatcher.createSnapshot(); + return log_index > 0 ? "Snapshot creation scheduled with last committed log index " + std::to_string(log_index) + "." + : "Fail to scheduled snapshot creation task."; } -String CheckSnapshotDoneCommand::run() +String LogInfoCommand::run() { - return keeper_dispatcher.snapshotDone() ? "Snapshot creation done." : "Fail to scheduled snapshot creation task."; + KeeperLogInfo log_info = keeper_dispatcher.getKeeperLogInfo(); + StringBuffer ret; + print(ret, "last_log_idx", log_info.last_log_idx); + print(ret, "last_log_term", log_info.last_log_term); + print(ret, "last_committed_log_idx", log_info.last_committed_log_idx); + print(ret, "leader_committed_log_idx", log_info.leader_committed_log_idx); + print(ret, "target_committed_log_idx", log_info.target_committed_log_idx); + print(ret, "last_snapshot_idx", log_info.last_snapshot_idx); + return ret.str(); } } diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index 28f1d7f153f..99005bab987 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -341,17 +341,24 @@ struct CreateSnapshotCommand : public IFourLetterCommand ~CreateSnapshotCommand() override = default; }; -/// Check whether last manual snapshot done -struct CheckSnapshotDoneCommand : public IFourLetterCommand +/** Raft log information: + * last_log_idx : 101 + * last_log_term : 1 + * last_committed_idx : 100 + * leader_committed_log_idx : 101 + * target_committed_log_idx : 101 + * last_snapshot_idx : 50 + */ +struct LogInfoCommand : public IFourLetterCommand { - explicit CheckSnapshotDoneCommand(KeeperDispatcher & keeper_dispatcher_) + explicit LogInfoCommand(KeeperDispatcher & keeper_dispatcher_) : IFourLetterCommand(keeper_dispatcher_) { } - String name() override { return "snpd"; } + String name() override { return "lgif"; } String run() override; - ~CheckSnapshotDoneCommand() override = default; + ~LogInfoCommand() override = default; }; } diff --git a/src/Coordination/Keeper4LWInfo.h b/src/Coordination/Keeper4LWInfo.h index 7d90152611e..dbddadaefbf 100644 --- a/src/Coordination/Keeper4LWInfo.h +++ b/src/Coordination/Keeper4LWInfo.h @@ -47,4 +47,26 @@ struct Keeper4LWInfo } }; +/// Keeper log information for 4lw commands +struct KeeperLogInfo +{ + /// My last log index in log store. + uint64_t last_log_idx; + + /// My last log term. + uint64_t last_log_term; + + /// My last committed log index in state machine. + uint64_t last_committed_log_idx; + + /// Leader's committed log index from my perspective. + uint64_t leader_committed_log_idx; + + /// Target log index should be committed to. + uint64_t target_committed_log_idx; + + /// The largest committed log index in last snapshot. + uint64_t last_snapshot_idx; +}; + } diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 48681957c13..0126bf8a1e5 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -204,16 +204,16 @@ public: keeper_stats.reset(); } - /// Create snapshot manually - bool createSnapshot() + /// Create snapshot manually, return the last committed log index in the snapshot + uint64_t createSnapshot() { return server->createSnapshot(); } /// Whether the last manually created snapshot is done - bool snapshotDone() + KeeperLogInfo getKeeperLogInfo() { - return server->snapshotDone(); + return server->getKeeperLogInfo(); } }; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 042ab35d709..38070938fc5 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -907,23 +907,25 @@ Keeper4LWInfo KeeperServer::getPartiallyFilled4LWInfo() const return result; } -bool KeeperServer::createSnapshot() +uint64_t KeeperServer::createSnapshot() { - std::lock_guard lock(snapshot_mutex); uint64_t log_idx = raft_instance->create_snapshot(); if (log_idx != 0) - { - last_manual_snapshot_log_idx = log_idx; - LOG_INFO(log, "Successfully schedule a keeper snapshot creation task at log index {}", log_idx); - return true; - } - return false; + LOG_INFO(log, "Snapshot creation scheduled with last committed log index {}.", log_idx); + else + LOG_WARNING(log, "Fail to scheduled snapshot creation task."); + return log_idx; } -bool KeeperServer::snapshotDone() +KeeperLogInfo KeeperServer::getKeeperLogInfo() { - std::lock_guard lock(snapshot_mutex); - return last_manual_snapshot_log_idx != 0 && last_manual_snapshot_log_idx == raft_instance->get_last_snapshot_idx(); + KeeperLogInfo log_info; + log_info.last_log_idx = raft_instance->get_last_log_idx(); + log_info.last_log_term = raft_instance->get_last_log_term(); + log_info.leader_committed_log_idx = raft_instance->get_leader_committed_log_idx(); + log_info.target_committed_log_idx = raft_instance->get_target_committed_log_idx(); + log_info.last_snapshot_idx = raft_instance->get_last_snapshot_idx(); + return log_info; } } diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 11e3b75d127..192c8f470b1 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -66,10 +66,6 @@ private: const bool create_snapshot_on_exit; - /// Used to check whether the previous manually created snapshot complete. - uint64_t last_manual_snapshot_log_idx; - std::mutex snapshot_mutex; - public: KeeperServer( const KeeperConfigurationAndSettingsPtr & settings_, @@ -136,9 +132,9 @@ public: /// Return true if update was successfully received. bool waitConfigurationUpdate(const ConfigUpdateAction & task); - bool createSnapshot(); + uint64_t createSnapshot(); - bool snapshotDone(); + KeeperLogInfo getKeeperLogInfo(); }; } From 163001382b07844b36f559f2c4373bcdce6d6836 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Tue, 25 Oct 2022 09:43:29 +0800 Subject: [PATCH 059/151] Fix no progress indication on INSERT FROM INFILE --- src/Client/ClientBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5d7de8ec799..9a5002b17af 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1173,7 +1173,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des bool have_data_in_stdin = !is_interactive && !stdin_is_a_tty && !std_in.eof(); - if (need_render_progress && have_data_in_stdin) + if (need_render_progress) { /// Set total_bytes_to_read for current fd. FileProgress file_progress(0, std_in.getFileSize()); From 9a36a509fe1272b53698eb0707249e3593fc3088 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 25 Oct 2022 17:15:49 +0800 Subject: [PATCH 060/151] fix test --- docs/en/operations/clickhouse-keeper.md | 20 +++++----- src/Coordination/FourLetterCommand.cpp | 25 ++++++++---- src/Coordination/FourLetterCommand.h | 15 ++++--- src/Coordination/Keeper4LWInfo.h | 6 +++ src/Coordination/KeeperServer.cpp | 4 +- .../test_keeper_four_word_command/test.py | 39 ++++++++++++++++--- 6 files changed, 79 insertions(+), 30 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 2ab76e1a1ea..8eee97ed275 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -309,21 +309,23 @@ Sessions with Ephemerals (1): /clickhouse/task_queue/ddl ``` -- `csnp`: Schedule a snapshot creation task. Return `Snapshot creation scheduled with last committed log index xxx.` if successfully scheduled or `Fail to scheduled snapshot creation task.` if failed. +- `csnp`: Schedule a snapshot creation task. Return the last committed log index of the scheduled snapshot if successfully scheduled or `Fail to scheduled snapshot creation task.` if failed. ``` -Snapshot creation scheduled with last committed log index 100. +100 ``` -- `lgif`: Keeper log information. `last_log_idx` : my last log index in log store; `last_log_term` : my last log term; `last_committed_log_idx` : my last committed log index in state machine; `leader_committed_log_idx` : leader's committed log index from my perspective; `target_committed_log_idx` : target log index should be committed to; `last_snapshot_idx` : the largest committed log index in last snapshot. +- `lgif`: Keeper log information. `first_log_idx` : my first log index in log store; `first_log_term` : my first log term; `last_log_idx` : my last log index in log store; `last_log_term` : my last log term; `last_committed_log_idx` : my last committed log index in state machine; `leader_committed_log_idx` : leader's committed log index from my perspective; `target_committed_log_idx` : target log index should be committed to; `last_snapshot_idx` : the largest committed log index in last snapshot. ``` -last_log_idx : 101 -last_log_term : 1 -last_committed_log_idx : 100 -leader_committed_log_idx : 101 -target_committed_log_idx : 101 -last_snapshot_idx : 50 +first_log_idx 1 +first_log_term 1 +last_log_idx 101 +last_log_term 1 +last_committed_log_idx 100 +leader_committed_log_idx 101 +target_committed_log_idx 101 +last_snapshot_idx 50 ``` ## [experimental] Migration from ZooKeeper {#migration-from-zookeeper} diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index c5841ce3404..402270640d2 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -481,20 +481,29 @@ String ApiVersionCommand::run() String CreateSnapshotCommand::run() { auto log_index = keeper_dispatcher.createSnapshot(); - return log_index > 0 ? "Snapshot creation scheduled with last committed log index " + std::to_string(log_index) + "." - : "Fail to scheduled snapshot creation task."; + return log_index > 0 ? std::to_string(log_index) : "Fail to scheduled snapshot creation task."; } String LogInfoCommand::run() { KeeperLogInfo log_info = keeper_dispatcher.getKeeperLogInfo(); StringBuffer ret; - print(ret, "last_log_idx", log_info.last_log_idx); - print(ret, "last_log_term", log_info.last_log_term); - print(ret, "last_committed_log_idx", log_info.last_committed_log_idx); - print(ret, "leader_committed_log_idx", log_info.leader_committed_log_idx); - print(ret, "target_committed_log_idx", log_info.target_committed_log_idx); - print(ret, "last_snapshot_idx", log_info.last_snapshot_idx); + + auto append = [&ret] (String key, uint64_t value) -> void + { + writeText(key, ret); + writeText('\t', ret); + writeText(std::to_string(value), ret); + writeText('\n', ret); + }; + append("first_log_idx", log_info.first_log_idx); + append("first_log_term", log_info.first_log_idx); + append("last_log_idx", log_info.last_log_idx); + append("last_log_term", log_info.last_log_term); + append("last_committed_log_idx", log_info.last_committed_log_idx); + append("leader_committed_log_idx", log_info.leader_committed_log_idx); + append("target_committed_log_idx", log_info.target_committed_log_idx); + append("last_snapshot_idx", log_info.last_snapshot_idx); return ret.str(); } diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index 99005bab987..a8801474bb0 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -17,6 +17,7 @@ using FourLetterCommandPtr = std::shared_ptr; /// Just like zookeeper Four Letter Words commands, CH Keeper responds to a small set of commands. /// Each command is composed of four letters, these commands are useful to monitor and issue system problems. /// The feature is based on Zookeeper 3.5.9, details is in https://zookeeper.apache.org/doc/r3.5.9/zookeeperAdmin.html#sc_zkCommands. +/// Also we add some additional commands such as csnp, lgif etc. struct IFourLetterCommand { public: @@ -342,12 +343,14 @@ struct CreateSnapshotCommand : public IFourLetterCommand }; /** Raft log information: - * last_log_idx : 101 - * last_log_term : 1 - * last_committed_idx : 100 - * leader_committed_log_idx : 101 - * target_committed_log_idx : 101 - * last_snapshot_idx : 50 + * first_log_idx 1 + * first_log_term 1 + * last_log_idx 101 + * last_log_term 1 + * last_committed_idx 100 + * leader_committed_log_idx 101 + * target_committed_log_idx 101 + * last_snapshot_idx 50 */ struct LogInfoCommand : public IFourLetterCommand { diff --git a/src/Coordination/Keeper4LWInfo.h b/src/Coordination/Keeper4LWInfo.h index dbddadaefbf..105478457cc 100644 --- a/src/Coordination/Keeper4LWInfo.h +++ b/src/Coordination/Keeper4LWInfo.h @@ -50,6 +50,12 @@ struct Keeper4LWInfo /// Keeper log information for 4lw commands struct KeeperLogInfo { + /// My first log index in log store. + uint64_t first_log_idx; + + /// My first log term. + uint64_t first_log_term; + /// My last log index in log store. uint64_t last_log_idx; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 38070938fc5..bea69ea0ba8 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -114,7 +114,6 @@ KeeperServer::KeeperServer( , is_recovering(config.getBool("keeper_server.force_recovery", false)) , keeper_context{std::make_shared()} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) - , last_manual_snapshot_log_idx(0) { if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); @@ -920,8 +919,11 @@ uint64_t KeeperServer::createSnapshot() KeeperLogInfo KeeperServer::getKeeperLogInfo() { KeeperLogInfo log_info; + log_info.first_log_idx = state_manager->load_log_store()->start_index(); + log_info.first_log_term = state_manager->load_log_store()->term_at(log_info.first_log_idx); log_info.last_log_idx = raft_instance->get_last_log_idx(); log_info.last_log_term = raft_instance->get_last_log_term(); + log_info.last_committed_log_idx = raft_instance->get_committed_log_idx(); log_info.leader_committed_log_idx = raft_instance->get_leader_committed_log_idx(); log_info.target_committed_log_idx = raft_instance->get_target_committed_log_idx(); log_info.last_snapshot_idx = raft_instance->get_last_snapshot_idx(); diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 4949d6f70de..4559904f8b7 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -598,19 +598,46 @@ def test_cmd_wchp(started_cluster): destroy_zk_client(zk) -def test_cmd_snapshot(started_cluster): +def test_cmd_csnp(started_cluster): + zk = None + try: + wait_nodes() + zk = get_fake_zk(node1.name, timeout=30.0) + data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="csnp") + try: + int(data) + assert True + except ValueError: + assert False + finally: + destroy_zk_client(zk) + + +def test_cmd_lgif(started_cluster): zk = None try: wait_nodes() clear_znodes() - reset_node_stats() zk = get_fake_zk(node1.name, timeout=30.0) + do_some_action(zk, create_cnt=100) - create = send_4lw_cmd(cmd="csnp") - assert create == "Snapshot creation scheduled." + data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="lgif") + print(data) + reader = csv.reader(data.split("\n"), delimiter="\t") + result = {} - check = send_4lw_cmd(cmd="snpd") - assert check == "Yes" or check == "No" + for row in reader: + if len(row) != 0: + result[row[0]] = row[1] + + assert int(result["first_log_idx"]) == 1 + assert int(result["first_log_term"]) == 1 + assert int(result["last_log_idx"]) >= 1 + assert int(result["last_log_term"]) == 1 + assert int(result["last_committed_log_idx"]) >= 1 + assert int(result["leader_committed_log_idx"]) >= 1 + assert int(result["target_committed_log_idx"]) >= 1 + assert int(result["last_snapshot_idx"]) >= 1 finally: destroy_zk_client(zk) From c7a0ebeb05ba81f4259b2c5910ac88c10520cafc Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 25 Oct 2022 17:46:24 +0800 Subject: [PATCH 061/151] little fix --- docs/en/operations/clickhouse-keeper.md | 2 +- src/Coordination/KeeperDispatcher.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 8eee97ed275..269e18023df 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -309,7 +309,7 @@ Sessions with Ephemerals (1): /clickhouse/task_queue/ddl ``` -- `csnp`: Schedule a snapshot creation task. Return the last committed log index of the scheduled snapshot if successfully scheduled or `Fail to scheduled snapshot creation task.` if failed. +- `csnp`: Schedule a snapshot creation task. Return the last committed log index of the scheduled snapshot if success or `Fail to scheduled snapshot creation task.` if failed. ``` 100 diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 0126bf8a1e5..84345ca1ff5 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -210,7 +210,7 @@ public: return server->createSnapshot(); } - /// Whether the last manually created snapshot is done + /// Get Raft information KeeperLogInfo getKeeperLogInfo() { return server->getKeeperLogInfo(); From 611c2e2bd75614e8eeb9d10933d76d6c5cd9f89b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 25 Oct 2022 13:34:34 +0000 Subject: [PATCH 062/151] Support for optimizing old parts for entire partition only --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 27 ++++++++++++++++++- src/Storages/MergeTree/MergeTreeSettings.h | 1 + .../test.py | 24 ++++++++++++----- 3 files changed, 44 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index b0ef1522685..27000796343 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -214,6 +214,14 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( /// Previous part only in boundaries of partition frame const MergeTreeData::DataPartPtr * prev_part = nullptr; + /// collect min_age for each partition while iterating parts + struct PartitionInfo + { + time_t min_age{std::numeric_limits::max()}; + }; + + std::unordered_map partitions_info; + size_t parts_selected_precondition = 0; for (const MergeTreeData::DataPartPtr & part : data_parts) { @@ -277,6 +285,9 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( part_info.compression_codec_desc = part->default_codec->getFullCodecDesc(); part_info.shall_participate_in_merges = has_volumes_with_disabled_merges ? part->shallParticipateInMerges(storage_policy) : true; + auto & partition_info = partitions_info[partition_id]; + partition_info.min_age = std::min(partition_info.min_age, part_info.age); + ++parts_selected_precondition; parts_ranges.back().emplace_back(part_info); @@ -333,7 +344,8 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( SimpleMergeSelector::Settings merge_settings; /// Override value from table settings merge_settings.max_parts_to_merge_at_once = data_settings->max_parts_to_merge_at_once; - merge_settings.min_age_to_force_merge = data_settings->min_age_to_force_merge_seconds; + if (!data_settings->min_age_to_force_merge_on_partition_only) + merge_settings.min_age_to_force_merge = data_settings->min_age_to_force_merge_seconds; if (aggressive) merge_settings.base = 1; @@ -347,6 +359,19 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( if (parts_to_merge.empty()) { + if (data_settings->min_age_to_force_merge_on_partition_only && data_settings->min_age_to_force_merge_seconds) + { + auto best_partition_it = std::max_element( + partitions_info.begin(), + partitions_info.end(), + [](const auto & e1, const auto & e2) { return e1.second.min_age > e2.second.min_age; }); + + if (best_partition_it != partitions_info.end() + && static_cast(best_partition_it->second.min_age) >= data_settings->min_age_to_force_merge_seconds) + return selectAllPartsToMergeWithinPartition( + future_part, can_merge_callback, best_partition_it->first, true, metadata_snapshot, txn, out_disable_reason); + } + if (out_disable_reason) *out_disable_reason = "There is no need to merge parts according to merge selector algorithm"; return SelectPartsDecision::CANNOT_SELECT; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 3fecb85f484..844c1ddbfe5 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -63,6 +63,7 @@ struct Settings; M(UInt64, merge_tree_clear_old_parts_interval_seconds, 1, "The period of executing the clear old parts operation in background.", 0) \ M(UInt64, merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds, 1ULL * 3600 * 24 * 30, "Remove old broken detached parts in the background if they remained intouched for a specified by this setting period of time.", 0) \ M(UInt64, min_age_to_force_merge_seconds, 0, "If all parts in a certain range are older than this value, range will be always eligible for merging. Set to 0 to disable.", 0) \ + M(Bool, min_age_to_force_merge_on_partition_only, false, "Whether min_age_to_force_merge_seconds should be applied only on the entire partition and not on subset.", false) \ M(UInt64, merge_tree_enable_clear_old_broken_detached, false, "Enable clearing old broken detached parts operation in background.", 0) \ M(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 0) \ \ diff --git a/tests/integration/test_merge_tree_optimize_old_parts/test.py b/tests/integration/test_merge_tree_optimize_old_parts/test.py index 7b386eba2c4..87e0ecd8108 100644 --- a/tests/integration/test_merge_tree_optimize_old_parts/test.py +++ b/tests/integration/test_merge_tree_optimize_old_parts/test.py @@ -13,7 +13,7 @@ node = cluster.add_instance( @pytest.fixture(scope="module") -def start_cluster(): +def started_cluster(): try: cluster.start() @@ -42,7 +42,7 @@ def check_expected_part_number(seconds, table_name, expected): assert ok -def test_without_force_merge_old_parts(start_cluster): +def test_without_force_merge_old_parts(started_cluster): node.query( "CREATE TABLE test_without_merge (i Int64) ENGINE = MergeTree ORDER BY i;" ) @@ -60,13 +60,18 @@ def test_without_force_merge_old_parts(start_cluster): node.query("DROP TABLE test_without_merge;") -def test_force_merge_old_parts(start_cluster): +@pytest.mark.parametrize("partition_only", ["True", "False"]) +def test_force_merge_old_parts(started_cluster, partition_only): node.query( - "CREATE TABLE test_with_merge (i Int64) ENGINE = MergeTree ORDER BY i SETTINGS min_age_to_force_merge_seconds=5;" + "CREATE TABLE test_with_merge (i Int64) " + "ENGINE = MergeTree " + "ORDER BY i " + f"SETTINGS min_age_to_force_merge_seconds=5, min_age_to_force_merge_on_partition_only={partition_only};" ) node.query("INSERT INTO test_with_merge SELECT 1") node.query("INSERT INTO test_with_merge SELECT 2") node.query("INSERT INTO test_with_merge SELECT 3") + assert get_part_number("test_with_merge") == TSV("""3\n""") expected = TSV("""1\n""") check_expected_part_number(10, "test_with_merge", expected) @@ -74,15 +79,20 @@ def test_force_merge_old_parts(start_cluster): node.query("DROP TABLE test_with_merge;") -def test_force_merge_old_parts_replicated_merge_tree(start_cluster): +@pytest.mark.parametrize("partition_only", ["True", "False"]) +def test_force_merge_old_parts_replicated_merge_tree(started_cluster, partition_only): node.query( - "CREATE TABLE test_replicated (i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/testing/test', 'node') ORDER BY i SETTINGS min_age_to_force_merge_seconds=5;" + "CREATE TABLE test_replicated (i Int64) " + "ENGINE = ReplicatedMergeTree('/clickhouse/testing/test', 'node') " + "ORDER BY i " + f"SETTINGS min_age_to_force_merge_seconds=5, min_age_to_force_merge_on_partition_only={partition_only};" ) node.query("INSERT INTO test_replicated SELECT 1") node.query("INSERT INTO test_replicated SELECT 2") node.query("INSERT INTO test_replicated SELECT 3") + assert get_part_number("test_replicated") == TSV("""3\n""") expected = TSV("""1\n""") check_expected_part_number(10, "test_replicated", expected) - node.query("DROP TABLE test_replicated;") + node.query("DROP TABLE test_replicated SYNC;") From a9a799ac89816b7a8a82c227aeb5533892168d0e Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 26 Oct 2022 05:20:34 +0000 Subject: [PATCH 063/151] Add test --- .../0_stateless/02473_infile_progress.py | 42 +++++++++++++++++++ .../02473_infile_progress.reference | 0 2 files changed, 42 insertions(+) create mode 100755 tests/queries/0_stateless/02473_infile_progress.py create mode 100644 tests/queries/0_stateless/02473_infile_progress.reference diff --git a/tests/queries/0_stateless/02473_infile_progress.py b/tests/queries/0_stateless/02473_infile_progress.py new file mode 100755 index 00000000000..2d6493fe4a5 --- /dev/null +++ b/tests/queries/0_stateless/02473_infile_progress.py @@ -0,0 +1,42 @@ +#!/usr/bin/env python3 +# Tags: no-replicated-database, no-parallel, no-fasttest + +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +# log=sys.stdout + +with client(name="client>", log=log) as client1: + filename = os.environ["CLICKHOUSE_TMP"] + "/infile_progress.tsv" + + client1.expect(prompt) + client1.send("DROP TABLE IF EXISTS test.infile_progress") + client1.expect(prompt) + client1.send(f"SELECT number FROM numbers(5) INTO OUTFILE '{filename}'") + client1.expect(prompt) + client1.send( + "CREATE TABLE test.infile_progress (a Int32) Engine=MergeTree order by tuple()" + ) + client1.expect(prompt) + client1.send(f"INSERT INTO test.infile_progress FROM INFILE '{filename}'") + client1.expect("Progress: 5.00 rows.*\)") + client1.expect(prompt) + + # send Ctrl-C + client1.send("\x03", eol="") + match = client1.expect("(%s)|([#\$] )" % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) + client1.send("DROP TABLE test.infile_progress") + client1.expect(prompt) + + os.remove(filename) diff --git a/tests/queries/0_stateless/02473_infile_progress.reference b/tests/queries/0_stateless/02473_infile_progress.reference new file mode 100644 index 00000000000..e69de29bb2d From 2254bef74a105ec91389d1c84a6805119f6b4fca Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 25 Oct 2022 12:07:07 +0800 Subject: [PATCH 064/151] implement function ascii --- .../functions/string-functions.md | 10 +++ src/Functions/ascii.cpp | 74 +++++++++++++++++++ .../queries/0_stateless/02353_ascii.reference | 2 + tests/queries/0_stateless/02353_ascii.sql | 2 + 4 files changed, 88 insertions(+) create mode 100644 src/Functions/ascii.cpp create mode 100644 tests/queries/0_stateless/02353_ascii.reference create mode 100644 tests/queries/0_stateless/02353_ascii.sql diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index a8ba4843279..982ba05f494 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1150,3 +1150,13 @@ A text with tags . The content within CDATA Do Nothing for 2 Minutes 2:00   ``` + +## ascii(s) {#ascii} + +Returns the ASCII code point of the first character of str. The result type is Int32. + +If s is empty, the result is 0. If the first character is not an ASCII character or part of the Latin-1 Supplement range of UTF-16, the result is undefined. + + + + diff --git a/src/Functions/ascii.cpp b/src/Functions/ascii.cpp new file mode 100644 index 00000000000..a8a6b9f7226 --- /dev/null +++ b/src/Functions/ascii.cpp @@ -0,0 +1,74 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NOT_IMPLEMENTED; +} + +struct AsciiName +{ + static constexpr auto name = "ascii"; +}; + + +struct AsciiImpl +{ + static constexpr auto is_fixed_to_constant = false; + using ReturnType = Int32; + + + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, PaddedPODArray & res) + { + size_t size = offsets.size(); + + ColumnString::Offset prev_offset = 0; + for (size_t i = 0; i < size; ++i) + { + res[i] = doAscii(data, prev_offset, offsets[i] - prev_offset - 1); + prev_offset = offsets[i]; + } + } + + [[noreturn]] static void vectorFixedToConstant(const ColumnString::Chars & /*data*/, size_t /*n*/, Int32 & /*res*/) + { + throw Exception("vectorFixedToConstant not implemented for function " + std::string(AsciiName::name), ErrorCodes::NOT_IMPLEMENTED); + } + + static void vectorFixedToVector(const ColumnString::Chars & data, size_t n, PaddedPODArray & res) + { + size_t size = data.size() / n; + + for (size_t i = 0; i < size; ++i) + { + res[i] = doAscii(data, i * n, n); + } + } + + [[noreturn]] static void array(const ColumnString::Offsets & /*offsets*/, PaddedPODArray & /*res*/) + { + throw Exception("Cannot apply function " + std::string(AsciiName::name) + " to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + [[noreturn]] static void uuid(const ColumnUUID::Container & /*offsets*/, size_t /*n*/, PaddedPODArray & /*res*/) + { + throw Exception("Cannot apply function " + std::string(AsciiName::name) + " to UUID argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + +private: + static Int32 doAscii(const ColumnString::Chars & buf, size_t offset, size_t size) { return size ? static_cast(buf[offset]) : 0; } +}; + +using FunctionAscii = FunctionStringOrArrayToT; + +REGISTER_FUNCTION(Ascii) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/tests/queries/0_stateless/02353_ascii.reference b/tests/queries/0_stateless/02353_ascii.reference new file mode 100644 index 00000000000..d44c5c7d87e --- /dev/null +++ b/tests/queries/0_stateless/02353_ascii.reference @@ -0,0 +1,2 @@ +50 +0 diff --git a/tests/queries/0_stateless/02353_ascii.sql b/tests/queries/0_stateless/02353_ascii.sql new file mode 100644 index 00000000000..c1c5d60c447 --- /dev/null +++ b/tests/queries/0_stateless/02353_ascii.sql @@ -0,0 +1,2 @@ +SELECT ascii('234'); +SELECT ascii(''); From add5360a1b17d794760d4f0eac9834c931a10fac Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 25 Oct 2022 12:07:07 +0800 Subject: [PATCH 065/151] implement function ascii --- .../functions/string-functions.md | 10 +++ src/Functions/ascii.cpp | 74 +++++++++++++++++++ .../queries/0_stateless/02353_ascii.reference | 2 + tests/queries/0_stateless/02353_ascii.sql | 2 + 4 files changed, 88 insertions(+) create mode 100644 src/Functions/ascii.cpp create mode 100644 tests/queries/0_stateless/02353_ascii.reference create mode 100644 tests/queries/0_stateless/02353_ascii.sql diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index a8ba4843279..982ba05f494 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1150,3 +1150,13 @@ A text with tags . The content within CDATA Do Nothing for 2 Minutes 2:00   ``` + +## ascii(s) {#ascii} + +Returns the ASCII code point of the first character of str. The result type is Int32. + +If s is empty, the result is 0. If the first character is not an ASCII character or part of the Latin-1 Supplement range of UTF-16, the result is undefined. + + + + diff --git a/src/Functions/ascii.cpp b/src/Functions/ascii.cpp new file mode 100644 index 00000000000..a8a6b9f7226 --- /dev/null +++ b/src/Functions/ascii.cpp @@ -0,0 +1,74 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NOT_IMPLEMENTED; +} + +struct AsciiName +{ + static constexpr auto name = "ascii"; +}; + + +struct AsciiImpl +{ + static constexpr auto is_fixed_to_constant = false; + using ReturnType = Int32; + + + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, PaddedPODArray & res) + { + size_t size = offsets.size(); + + ColumnString::Offset prev_offset = 0; + for (size_t i = 0; i < size; ++i) + { + res[i] = doAscii(data, prev_offset, offsets[i] - prev_offset - 1); + prev_offset = offsets[i]; + } + } + + [[noreturn]] static void vectorFixedToConstant(const ColumnString::Chars & /*data*/, size_t /*n*/, Int32 & /*res*/) + { + throw Exception("vectorFixedToConstant not implemented for function " + std::string(AsciiName::name), ErrorCodes::NOT_IMPLEMENTED); + } + + static void vectorFixedToVector(const ColumnString::Chars & data, size_t n, PaddedPODArray & res) + { + size_t size = data.size() / n; + + for (size_t i = 0; i < size; ++i) + { + res[i] = doAscii(data, i * n, n); + } + } + + [[noreturn]] static void array(const ColumnString::Offsets & /*offsets*/, PaddedPODArray & /*res*/) + { + throw Exception("Cannot apply function " + std::string(AsciiName::name) + " to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + [[noreturn]] static void uuid(const ColumnUUID::Container & /*offsets*/, size_t /*n*/, PaddedPODArray & /*res*/) + { + throw Exception("Cannot apply function " + std::string(AsciiName::name) + " to UUID argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + +private: + static Int32 doAscii(const ColumnString::Chars & buf, size_t offset, size_t size) { return size ? static_cast(buf[offset]) : 0; } +}; + +using FunctionAscii = FunctionStringOrArrayToT; + +REGISTER_FUNCTION(Ascii) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/tests/queries/0_stateless/02353_ascii.reference b/tests/queries/0_stateless/02353_ascii.reference new file mode 100644 index 00000000000..d44c5c7d87e --- /dev/null +++ b/tests/queries/0_stateless/02353_ascii.reference @@ -0,0 +1,2 @@ +50 +0 diff --git a/tests/queries/0_stateless/02353_ascii.sql b/tests/queries/0_stateless/02353_ascii.sql new file mode 100644 index 00000000000..c1c5d60c447 --- /dev/null +++ b/tests/queries/0_stateless/02353_ascii.sql @@ -0,0 +1,2 @@ +SELECT ascii('234'); +SELECT ascii(''); From 97aaebfa1808e9b90258a78de2927c97bae05feb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 26 Oct 2022 10:06:56 +0000 Subject: [PATCH 066/151] Address PR comments --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 5 +- .../__init__.py | 0 .../configs/zookeeper_config.xml | 8 -- .../test.py | 98 ------------------- .../02473_optimize_old_parts.reference | 10 ++ .../0_stateless/02473_optimize_old_parts.sql | 67 +++++++++++++ 6 files changed, 80 insertions(+), 108 deletions(-) delete mode 100644 tests/integration/test_merge_tree_optimize_old_parts/__init__.py delete mode 100644 tests/integration/test_merge_tree_optimize_old_parts/configs/zookeeper_config.xml delete mode 100644 tests/integration/test_merge_tree_optimize_old_parts/test.py create mode 100644 tests/queries/0_stateless/02473_optimize_old_parts.reference create mode 100644 tests/queries/0_stateless/02473_optimize_old_parts.sql diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 27000796343..0f44d1a7da3 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -366,8 +366,9 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( partitions_info.end(), [](const auto & e1, const auto & e2) { return e1.second.min_age > e2.second.min_age; }); - if (best_partition_it != partitions_info.end() - && static_cast(best_partition_it->second.min_age) >= data_settings->min_age_to_force_merge_seconds) + assert(best_partition_it != partitions_info.end()); + + if (static_cast(best_partition_it->second.min_age) >= data_settings->min_age_to_force_merge_seconds) return selectAllPartsToMergeWithinPartition( future_part, can_merge_callback, best_partition_it->first, true, metadata_snapshot, txn, out_disable_reason); } diff --git a/tests/integration/test_merge_tree_optimize_old_parts/__init__.py b/tests/integration/test_merge_tree_optimize_old_parts/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_merge_tree_optimize_old_parts/configs/zookeeper_config.xml b/tests/integration/test_merge_tree_optimize_old_parts/configs/zookeeper_config.xml deleted file mode 100644 index 18412349228..00000000000 --- a/tests/integration/test_merge_tree_optimize_old_parts/configs/zookeeper_config.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - - zoo1 - 2181 - - - diff --git a/tests/integration/test_merge_tree_optimize_old_parts/test.py b/tests/integration/test_merge_tree_optimize_old_parts/test.py deleted file mode 100644 index 87e0ecd8108..00000000000 --- a/tests/integration/test_merge_tree_optimize_old_parts/test.py +++ /dev/null @@ -1,98 +0,0 @@ -import pytest -import time -from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import TSV - -cluster = ClickHouseCluster(__file__) -node = cluster.add_instance( - "node", - main_configs=["configs/zookeeper_config.xml"], - with_zookeeper=True, -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - yield cluster - finally: - cluster.shutdown() - - -def get_part_number(table_name): - return TSV( - node.query( - f"SELECT count(*) FROM system.parts where table='{table_name}' and active=1" - ) - ) - - -def check_expected_part_number(seconds, table_name, expected): - ok = False - for i in range(int(seconds) * 2): - result = get_part_number(table_name) - if result == expected: - ok = True - break - else: - time.sleep(1) - assert ok - - -def test_without_force_merge_old_parts(started_cluster): - node.query( - "CREATE TABLE test_without_merge (i Int64) ENGINE = MergeTree ORDER BY i;" - ) - node.query("INSERT INTO test_without_merge SELECT 1") - node.query("INSERT INTO test_without_merge SELECT 2") - node.query("INSERT INTO test_without_merge SELECT 3") - - expected = TSV("""3\n""") - # verify that the parts don't get merged - for i in range(10): - if get_part_number("test_without_merge") != expected: - assert False - time.sleep(1) - - node.query("DROP TABLE test_without_merge;") - - -@pytest.mark.parametrize("partition_only", ["True", "False"]) -def test_force_merge_old_parts(started_cluster, partition_only): - node.query( - "CREATE TABLE test_with_merge (i Int64) " - "ENGINE = MergeTree " - "ORDER BY i " - f"SETTINGS min_age_to_force_merge_seconds=5, min_age_to_force_merge_on_partition_only={partition_only};" - ) - node.query("INSERT INTO test_with_merge SELECT 1") - node.query("INSERT INTO test_with_merge SELECT 2") - node.query("INSERT INTO test_with_merge SELECT 3") - assert get_part_number("test_with_merge") == TSV("""3\n""") - - expected = TSV("""1\n""") - check_expected_part_number(10, "test_with_merge", expected) - - node.query("DROP TABLE test_with_merge;") - - -@pytest.mark.parametrize("partition_only", ["True", "False"]) -def test_force_merge_old_parts_replicated_merge_tree(started_cluster, partition_only): - node.query( - "CREATE TABLE test_replicated (i Int64) " - "ENGINE = ReplicatedMergeTree('/clickhouse/testing/test', 'node') " - "ORDER BY i " - f"SETTINGS min_age_to_force_merge_seconds=5, min_age_to_force_merge_on_partition_only={partition_only};" - ) - node.query("INSERT INTO test_replicated SELECT 1") - node.query("INSERT INTO test_replicated SELECT 2") - node.query("INSERT INTO test_replicated SELECT 3") - assert get_part_number("test_replicated") == TSV("""3\n""") - - expected = TSV("""1\n""") - check_expected_part_number(10, "test_replicated", expected) - - node.query("DROP TABLE test_replicated SYNC;") diff --git a/tests/queries/0_stateless/02473_optimize_old_parts.reference b/tests/queries/0_stateless/02473_optimize_old_parts.reference new file mode 100644 index 00000000000..6767887ba86 --- /dev/null +++ b/tests/queries/0_stateless/02473_optimize_old_parts.reference @@ -0,0 +1,10 @@ +Without merge +6 +With merge any part range +1 +With merge partition only +1 +With merge replicated any part range +1 +With merge replicated partition only +1 diff --git a/tests/queries/0_stateless/02473_optimize_old_parts.sql b/tests/queries/0_stateless/02473_optimize_old_parts.sql new file mode 100644 index 00000000000..545bd58dddc --- /dev/null +++ b/tests/queries/0_stateless/02473_optimize_old_parts.sql @@ -0,0 +1,67 @@ +DROP TABLE IF EXISTS test_without_merge; +DROP TABLE IF EXISTS test_with_merge; +DROP TABLE IF EXISTS test_replicated; + +SELECT 'Without merge'; + +CREATE TABLE test_without_merge (i Int64) ENGINE = MergeTree ORDER BY i; +INSERT INTO test_without_merge SELECT 1; +INSERT INTO test_without_merge SELECT 2; +INSERT INTO test_without_merge SELECT 3; + +SELECT sleepEachRow(1) FROM numbers(6) FORMAT Null; +SELECT count(*) FROM system.parts where table='test_without_merge' and active=1; + +DROP TABLE test_without_merge; + +SELECT 'With merge any part range'; + +CREATE TABLE test_with_merge (i Int64) ENGINE = MergeTree ORDER BY i +SETTINGS min_age_to_force_merge_seconds=3, min_age_to_force_merge_on_partition_only=false; +INSERT INTO test_with_merge SELECT 1; +INSERT INTO test_with_merge SELECT 2; +INSERT INTO test_with_merge SELECT 3; + +SELECT sleepEachRow(1) FROM numbers(6) FORMAT Null; +SELECT count(*) FROM system.parts where table='test_with_merge' and active=1; + +DROP TABLE test_with_merge; + +SELECT 'With merge partition only'; + +CREATE TABLE test_with_merge (i Int64) ENGINE = MergeTree ORDER BY i +SETTINGS min_age_to_force_merge_seconds=3, min_age_to_force_merge_on_partition_only=true; +INSERT INTO test_with_merge SELECT 1; +INSERT INTO test_with_merge SELECT 2; +INSERT INTO test_with_merge SELECT 3; + +SELECT sleepEachRow(1) FROM numbers(6) FORMAT Null; +SELECT count(*) FROM system.parts where table='test_with_merge' and active=1; + +DROP TABLE test_with_merge; + +SELECT 'With merge replicated any part range'; + +CREATE TABLE test_replicated (i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test02473', 'node') ORDER BY i +SETTINGS min_age_to_force_merge_seconds=3, min_age_to_force_merge_on_partition_only=false; +INSERT INTO test_replicated SELECT 1; +INSERT INTO test_replicated SELECT 2; +INSERT INTO test_replicated SELECT 3; + +SELECT sleepEachRow(1) FROM numbers(6) FORMAT Null; +SELECT count(*) FROM system.parts where table='test_replicated' and active=1; + +DROP TABLE test_replicated; + +SELECT 'With merge replicated partition only'; + +CREATE TABLE test_replicated (i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test02473_partition_only', 'node') ORDER BY i +SETTINGS min_age_to_force_merge_seconds=3, min_age_to_force_merge_on_partition_only=true; +INSERT INTO test_replicated SELECT 1; +INSERT INTO test_replicated SELECT 2; +INSERT INTO test_replicated SELECT 3; + +SELECT sleepEachRow(1) FROM numbers(6) FORMAT Null; +SELECT count(*) FROM system.parts where table='test_replicated' and active=1; + +DROP TABLE test_replicated; From 48c37c52e6c2e60423e399f0f457c6fd0bf8dbcb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Wed, 26 Oct 2022 18:18:04 +0800 Subject: [PATCH 067/151] Update src/Functions/ascii.cpp Co-authored-by: Vladimir C --- src/Functions/ascii.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Functions/ascii.cpp b/src/Functions/ascii.cpp index a8a6b9f7226..592cbe5f1c4 100644 --- a/src/Functions/ascii.cpp +++ b/src/Functions/ascii.cpp @@ -68,7 +68,16 @@ using FunctionAscii = FunctionStringOrArrayToT({}, FunctionFactory::CaseInsensitive); + factory.registerFunction( + { + R"( +Returns the ASCII code point of the first character of str. The result type is Int32. + +If s is empty, the result is 0. If the first character is not an ASCII character or part of the Latin-1 Supplement range of UTF-16, the result is undefined) + )", + Documentation::Examples{{"ascii", "SELECT ascii('234')"}}, + Documentation::Categories{"String"} + }, FunctionFactory::CaseInsensitive); } } From c7e5eb756ba0ca32d66ee50db80cfdf3cd765cfc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Wed, 26 Oct 2022 18:18:15 +0800 Subject: [PATCH 068/151] Update src/Functions/ascii.cpp Co-authored-by: Vladimir C --- src/Functions/ascii.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Functions/ascii.cpp b/src/Functions/ascii.cpp index 592cbe5f1c4..38ba7f0bbac 100644 --- a/src/Functions/ascii.cpp +++ b/src/Functions/ascii.cpp @@ -61,7 +61,10 @@ struct AsciiImpl } private: - static Int32 doAscii(const ColumnString::Chars & buf, size_t offset, size_t size) { return size ? static_cast(buf[offset]) : 0; } + static Int32 doAscii(const ColumnString::Chars & buf, size_t offset, size_t size) + { + return size ? static_cast(buf[offset]) : 0; + } }; using FunctionAscii = FunctionStringOrArrayToT; From 6a8fa2d4a5e922fe9dadd322429e5a88e4df1d30 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 26 Oct 2022 18:45:23 +0200 Subject: [PATCH 069/151] Added new field allow_readonly in system.table_functions to allow using table functions in readonly mode. Implementation: * Added a new field allow_readonly to table system.table_functions. * Updated to use new field allow_readonly to allow using table functions in readonly mode. * Added TableFunctionProperties similar to AggregateFunctionProperties. * The functions allowed in readonly mode are in this set table_functions_allowed_in_readonly_mode. Testing: * Added a test for filesystem tests/queries/0_stateless/02473_functions_in_readonly_mode.sh Documentation: * Updated the english documentation for Table Functions. --- .../en/sql-reference/table-functions/index.md | 4 ++ .../System/StorageSystemTableFunctions.cpp | 27 ++++++++++--- src/TableFunctions/ITableFunction.cpp | 5 ++- src/TableFunctions/ITableFunction.h | 9 +++++ src/TableFunctions/TableFunctionFactory.cpp | 40 ++++++++++++++----- src/TableFunctions/TableFunctionFactory.h | 24 +++++++++-- ...02473_functions_in_readonly_mode.reference | 3 ++ .../02473_functions_in_readonly_mode.sql | 5 +++ 8 files changed, 97 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/02473_functions_in_readonly_mode.reference create mode 100644 tests/queries/0_stateless/02473_functions_in_readonly_mode.sql diff --git a/docs/en/sql-reference/table-functions/index.md b/docs/en/sql-reference/table-functions/index.md index d09adcd13d6..94b23bc695c 100644 --- a/docs/en/sql-reference/table-functions/index.md +++ b/docs/en/sql-reference/table-functions/index.md @@ -39,3 +39,7 @@ You can’t use table functions if the [allow_ddl](../../operations/settings/per | [s3](../../sql-reference/table-functions/s3.md) | Creates a [S3](../../engines/table-engines/integrations/s3.md)-engine table. | | [sqlite](../../sql-reference/table-functions/sqlite.md) | Creates a [sqlite](../../engines/table-engines/integrations/sqlite.md)-engine table. | +:::note +Only these table functions are enabled in readonly mode : +null, view, viewIfPermitted, numbers, numbers_mt, generateRandom, values, cluster, clusterAllReplicas +::: \ No newline at end of file diff --git a/src/Storages/System/StorageSystemTableFunctions.cpp b/src/Storages/System/StorageSystemTableFunctions.cpp index 308cbc5686d..07a504edc5e 100644 --- a/src/Storages/System/StorageSystemTableFunctions.cpp +++ b/src/Storages/System/StorageSystemTableFunctions.cpp @@ -1,16 +1,23 @@ #include - #include +#include + namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_FUNCTION; +} + NamesAndTypesList StorageSystemTableFunctions::getNamesAndTypes() { return - { - {"name", std::make_shared()}, - {"description", std::make_shared()} - }; + { + {"name", std::make_shared()}, + {"description", std::make_shared()}, + {"allow_readonly", std::make_shared()} + }; } void StorageSystemTableFunctions::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const @@ -20,7 +27,15 @@ void StorageSystemTableFunctions::fillData(MutableColumns & res_columns, Context for (const auto & function_name : functions_names) { res_columns[0]->insert(function_name); - res_columns[1]->insert(factory.getDocumentation(function_name).description); + + auto properties = factory.tryGetProperties(function_name); + if (properties) + { + res_columns[1]->insert(properties->documentation.description); + res_columns[2]->insert(properties->allow_readonly); + } + else + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", function_name); } } diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index 82b6230dc30..da0de7e47f6 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace ProfileEvents @@ -25,8 +26,8 @@ StoragePtr ITableFunction::execute(const ASTPtr & ast_function, ContextPtr conte ProfileEvents::increment(ProfileEvents::TableFunctionExecute); AccessFlags required_access = getSourceAccessType(); - String function_name = getName(); - if ((function_name != "null") && (function_name != "view") && (function_name != "viewIfPermitted")) + auto table_function_properties = TableFunctionFactory::instance().tryGetProperties(getName()); + if (!(table_function_properties && table_function_properties->allow_readonly)) required_access |= AccessType::CREATE_TEMPORARY_TABLE; context->checkAccess(required_access); diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index 4b9a87b93f1..a05edcd32c8 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -79,6 +80,14 @@ private: virtual const char * getStorageTypeName() const = 0; }; +/// Properties of table function that are independent of argument types and parameters. +struct TableFunctionProperties +{ + Documentation documentation; + bool allow_readonly = false; +}; + + using TableFunctionPtr = std::shared_ptr; diff --git a/src/TableFunctions/TableFunctionFactory.cpp b/src/TableFunctions/TableFunctionFactory.cpp index 5ed22e39300..79802d2ec77 100644 --- a/src/TableFunctions/TableFunctionFactory.cpp +++ b/src/TableFunctions/TableFunctionFactory.cpp @@ -16,16 +16,22 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +static const NameSet table_functions_allowed_in_readonly_mode +{ + "null", "view", "viewIfPermitted", "numbers", "numbers_mt", "generateRandom", "values", "cluster", "clusterAllReplicas" +}; void TableFunctionFactory::registerFunction( const std::string & name, TableFunctionCreator creator, Documentation doc, CaseSensitiveness case_sensitiveness) { - if (!table_functions.emplace(name, TableFunctionFactoryData{creator, doc}).second) + bool allowed_in_readonly_mode = table_functions_allowed_in_readonly_mode.contains(name); + + if (!table_functions.emplace(name, TableFunctionFactoryData{creator, {doc,allowed_in_readonly_mode}}).second) throw Exception("TableFunctionFactory: the table function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); if (case_sensitiveness == CaseInsensitive - && !case_insensitive_table_functions.emplace(Poco::toLower(name), TableFunctionFactoryData{creator, doc}).second) + && !case_insensitive_table_functions.emplace(Poco::toLower(name), TableFunctionFactoryData{creator, {doc,allowed_in_readonly_mode}}).second) throw Exception("TableFunctionFactory: the case insensitive table function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); } @@ -59,13 +65,13 @@ TableFunctionPtr TableFunctionFactory::tryGet( auto it = table_functions.find(name); if (table_functions.end() != it) { - res = it->second.first(); + res = it->second.creator(); } else { it = case_insensitive_table_functions.find(Poco::toLower(name)); if (case_insensitive_table_functions.end() != it) - res = it->second.first(); + res = it->second.creator(); } if (!res) @@ -86,13 +92,29 @@ bool TableFunctionFactory::isTableFunctionName(const std::string & name) const return table_functions.contains(name); } -Documentation TableFunctionFactory::getDocumentation(const std::string & name) const +std::optional TableFunctionFactory::tryGetProperties(const String & name) const { - auto it = table_functions.find(name); - if (it == table_functions.end()) - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", name); + return tryGetPropertiesImpl(name); +} - return it->second.second; +std::optional TableFunctionFactory::tryGetPropertiesImpl(const String & name_param) const +{ + String name = getAliasToOrName(name_param); + Value found; + + /// Find by exact match. + if (auto it = table_functions.find(name); it != table_functions.end()) + { + found = it->second; + } + + if (auto jt = case_insensitive_table_functions.find(Poco::toLower(name)); jt != case_insensitive_table_functions.end()) + found = jt->second; + + if (found.creator) + return found.properties; + + return {}; } TableFunctionFactory & TableFunctionFactory::instance() diff --git a/src/TableFunctions/TableFunctionFactory.h b/src/TableFunctions/TableFunctionFactory.h index 8ff352ff9ac..24b46d40de0 100644 --- a/src/TableFunctions/TableFunctionFactory.h +++ b/src/TableFunctions/TableFunctionFactory.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include @@ -18,7 +17,24 @@ namespace DB class Context; using TableFunctionCreator = std::function; -using TableFunctionFactoryData = std::pair; + +struct TableFunctionFactoryData +{ + TableFunctionCreator creator; + TableFunctionProperties properties; + + TableFunctionFactoryData() = default; + TableFunctionFactoryData(const TableFunctionFactoryData &) = default; + TableFunctionFactoryData & operator = (const TableFunctionFactoryData &) = default; + + template + requires (!std::is_same_v) + TableFunctionFactoryData(Creator creator_, TableFunctionProperties properties_ = {}) /// NOLINT + : creator(std::forward(creator_)), properties(std::move(properties_)) + { + } +}; + /** Lets you get a table function by its name. */ @@ -48,7 +64,7 @@ public: /// Returns nullptr if not found. TableFunctionPtr tryGet(const std::string & name, ContextPtr context) const; - Documentation getDocumentation(const std::string & name) const; + std::optional tryGetProperties(const String & name) const; bool isTableFunctionName(const std::string & name) const; @@ -61,6 +77,8 @@ private: String getFactoryName() const override { return "TableFunctionFactory"; } + std::optional tryGetPropertiesImpl(const String & name) const; + TableFunctions table_functions; TableFunctions case_insensitive_table_functions; }; diff --git a/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference b/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference new file mode 100644 index 00000000000..4977168f515 --- /dev/null +++ b/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference @@ -0,0 +1,3 @@ +0 +(123,'str') +0 diff --git a/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql b/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql new file mode 100644 index 00000000000..7cf55ad714c --- /dev/null +++ b/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql @@ -0,0 +1,5 @@ +SELECT * from numbers(1); +select * from format(JSONEachRow, '{"x" : [123, "str"]}'); +SELECT * from numbers(1) SETTINGS readonly=1; +select * from format(JSONEachRow, '{"x" : [123, "str"]}') SETTINGS readonly=1; -- { serverError READONLY } +set readonly=0; \ No newline at end of file From 6d8e2db082bc6226afc26ea9145b503356ff011d Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 26 Oct 2022 18:49:14 +0200 Subject: [PATCH 070/151] Removed extra line from test - 42414 Enable table functions in readonly mode --- tests/queries/0_stateless/02473_functions_in_readonly_mode.sql | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql b/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql index 7cf55ad714c..eb2631168a8 100644 --- a/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql +++ b/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql @@ -1,5 +1,4 @@ SELECT * from numbers(1); select * from format(JSONEachRow, '{"x" : [123, "str"]}'); SELECT * from numbers(1) SETTINGS readonly=1; -select * from format(JSONEachRow, '{"x" : [123, "str"]}') SETTINGS readonly=1; -- { serverError READONLY } -set readonly=0; \ No newline at end of file +select * from format(JSONEachRow, '{"x" : [123, "str"]}') SETTINGS readonly=1; -- { serverError READONLY } \ No newline at end of file From 598b45f1ec2194183ec20418d7b7caf7192429be Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 27 Oct 2022 08:06:39 +0000 Subject: [PATCH 071/151] Add test for partition only and new parts --- .../02473_optimize_old_parts.reference | 2 ++ .../0_stateless/02473_optimize_old_parts.sql | 18 ++++++++++++++++++ 2 files changed, 20 insertions(+) diff --git a/tests/queries/0_stateless/02473_optimize_old_parts.reference b/tests/queries/0_stateless/02473_optimize_old_parts.reference index e80812bddcd..9002d73ff27 100644 --- a/tests/queries/0_stateless/02473_optimize_old_parts.reference +++ b/tests/queries/0_stateless/02473_optimize_old_parts.reference @@ -8,3 +8,5 @@ With merge replicated any part range 1 With merge replicated partition only 1 +With merge partition only and new parts +3 diff --git a/tests/queries/0_stateless/02473_optimize_old_parts.sql b/tests/queries/0_stateless/02473_optimize_old_parts.sql index d673ef22f67..76c1ba73097 100644 --- a/tests/queries/0_stateless/02473_optimize_old_parts.sql +++ b/tests/queries/0_stateless/02473_optimize_old_parts.sql @@ -65,3 +65,21 @@ SELECT sleepEachRow(1) FROM numbers(6) FORMAT Null; SELECT count(*) FROM system.parts WHERE database = currentDatabase() AND table='test_replicated' AND active; DROP TABLE test_replicated; + +SELECT 'With merge partition only and new parts'; + +CREATE TABLE test_with_merge (i Int64) ENGINE = MergeTree ORDER BY i +SETTINGS min_age_to_force_merge_seconds=3, min_age_to_force_merge_on_partition_only=true; +SYSTEM STOP MERGES test_with_merge; +-- These three parts will have min_age=6 at the time of merge +INSERT INTO test_with_merge SELECT 1; +INSERT INTO test_with_merge SELECT 2; +SELECT sleepEachRow(1) FROM numbers(6) FORMAT Null; +-- These three parts will have min_age=0 at the time of merge +-- and so, nothing will be merged. +INSERT INTO test_with_merge SELECT 3; +SYSTEM START MERGES test_with_merge; + +SELECT count(*) FROM system.parts WHERE database = currentDatabase() AND table='test_with_merge' AND active; + +DROP TABLE test_with_merge; From 41dc5b30b4d1a16a4a397bc8fc034c7091adf18f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 27 Oct 2022 10:52:59 +0200 Subject: [PATCH 072/151] Updated test to use TSV in format, and fixed test 02117_show_create_table_system - 42414 Enable table functions in readonly mode --- .../0_stateless/02117_show_create_table_system.reference | 3 ++- .../0_stateless/02473_functions_in_readonly_mode.reference | 2 +- .../queries/0_stateless/02473_functions_in_readonly_mode.sql | 4 ++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index ad27b86c6f5..65362e9f35d 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -1013,7 +1013,8 @@ COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.table_functions ( `name` String, - `description` String + `description` String, + `allow_readonly` UInt8 ) ENGINE = SystemTableFunctions COMMENT 'SYSTEM TABLE is built on the fly.' diff --git a/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference b/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference index 4977168f515..500004a06b2 100644 --- a/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference +++ b/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference @@ -1,3 +1,3 @@ 0 -(123,'str') +123 0 diff --git a/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql b/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql index eb2631168a8..c5c82d2e2bf 100644 --- a/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql +++ b/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql @@ -1,4 +1,4 @@ SELECT * from numbers(1); -select * from format(JSONEachRow, '{"x" : [123, "str"]}'); +SELECT * from format('TSV', '123'); SELECT * from numbers(1) SETTINGS readonly=1; -select * from format(JSONEachRow, '{"x" : [123, "str"]}') SETTINGS readonly=1; -- { serverError READONLY } \ No newline at end of file +SELECT * from format('TSV', '123') SETTINGS readonly=1; -- { serverError READONLY } \ No newline at end of file From c0809643915b2840a02e7962b946d05bca7619a4 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 28 Oct 2022 13:39:01 +0800 Subject: [PATCH 073/151] correct 'notLike' in key condition --- src/Storages/MergeTree/KeyCondition.cpp | 6 ++++-- src/Storages/MergeTree/KeyCondition.h | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 9b579512522..9bd9f3c8853 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -55,7 +55,7 @@ String Range::toString() const /// Example: for `Hello\_World% ...` string it returns `Hello_World`, and for `%test%` returns an empty string. -String extractFixedPrefixFromLikePattern(const String & like_pattern) +String extractFixedPrefixFromLikePattern(const String & like_pattern, bool perfect_prefix_match) { String fixed_prefix; @@ -68,6 +68,8 @@ String extractFixedPrefixFromLikePattern(const String & like_pattern) case '%': [[fallthrough]]; case '_': + if (perfect_prefix_match && std::find_if(pos+1, end, [](const char c) { return c != '%' && c != '_'; }) != end) + return ""; return fixed_prefix; case '\\': @@ -567,7 +569,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = extractFixedPrefixFromLikePattern(value.get()); + String prefix = extractFixedPrefixFromLikePattern(value.get(), true); if (prefix.empty()) return false; diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index d00a25a1077..da1a74af90d 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -481,6 +481,6 @@ private: bool strict; }; -String extractFixedPrefixFromLikePattern(const String & like_pattern); +String extractFixedPrefixFromLikePattern(const String & like_pattern, bool perfect_prefix_match = false); } From 9c860f0e0623a19b77a8f4f474df680836406ef4 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 28 Oct 2022 13:42:39 +0800 Subject: [PATCH 074/151] fix test style check --- .../0_stateless/002458_key_condition_not_like_prefix.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql b/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql index 211fa5662e7..2c1402df27e 100644 --- a/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql +++ b/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql @@ -2,4 +2,4 @@ CREATE TABLE data (str String) ENGINE=MergeTree ORDER BY str; INSERT INTO data (str) SELECT 'aa' FROM numbers(100000); INSERT INTO data (str) SELECT 'ba' FROM numbers(100000); INSERT INTO data (str) SELECT 'ca' FROM numbers(100000); -SELECT count()FROM data WHERE str NOT LIKE 'a%' SETTINGS force_primary_key=1; \ No newline at end of file +SELECT count()FROM data WHERE str NOT LIKE 'a%' SETTINGS force_primary_key=1; From 9ea9039b0f80b22f229f98c036437814b41bfb73 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 28 Oct 2022 13:45:08 +0800 Subject: [PATCH 075/151] fix test name --- ...ix.reference => 02458_key_condition_not_like_prefix.reference} | 0 ...ot_like_prefix.sql => 02458_key_condition_not_like_prefix.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{002458_key_condition_not_like_prefix.reference => 02458_key_condition_not_like_prefix.reference} (100%) rename tests/queries/0_stateless/{002458_key_condition_not_like_prefix.sql => 02458_key_condition_not_like_prefix.sql} (100%) diff --git a/tests/queries/0_stateless/002458_key_condition_not_like_prefix.reference b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.reference similarity index 100% rename from tests/queries/0_stateless/002458_key_condition_not_like_prefix.reference rename to tests/queries/0_stateless/02458_key_condition_not_like_prefix.reference diff --git a/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.sql similarity index 100% rename from tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql rename to tests/queries/0_stateless/02458_key_condition_not_like_prefix.sql From 0fe0aa44d0ec318d3e9c35aa5f5af964fa28dc5e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 28 Oct 2022 07:38:57 +0000 Subject: [PATCH 076/151] Increase wait time --- .../queries/0_stateless/02473_optimize_old_parts.sql | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02473_optimize_old_parts.sql b/tests/queries/0_stateless/02473_optimize_old_parts.sql index 76c1ba73097..106175ab6f5 100644 --- a/tests/queries/0_stateless/02473_optimize_old_parts.sql +++ b/tests/queries/0_stateless/02473_optimize_old_parts.sql @@ -9,7 +9,7 @@ INSERT INTO test_without_merge SELECT 1; INSERT INTO test_without_merge SELECT 2; INSERT INTO test_without_merge SELECT 3; -SELECT sleepEachRow(1) FROM numbers(6) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(9) FORMAT Null; SELECT count(*) FROM system.parts WHERE database = currentDatabase() AND table='test_without_merge' AND active; DROP TABLE test_without_merge; @@ -22,7 +22,7 @@ INSERT INTO test_with_merge SELECT 1; INSERT INTO test_with_merge SELECT 2; INSERT INTO test_with_merge SELECT 3; -SELECT sleepEachRow(1) FROM numbers(6) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(9) FORMAT Null; SELECT count(*) FROM system.parts WHERE database = currentDatabase() AND table='test_with_merge' AND active; DROP TABLE test_with_merge; @@ -35,7 +35,7 @@ INSERT INTO test_with_merge SELECT 1; INSERT INTO test_with_merge SELECT 2; INSERT INTO test_with_merge SELECT 3; -SELECT sleepEachRow(1) FROM numbers(6) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(9) FORMAT Null; SELECT count(*) FROM system.parts WHERE database = currentDatabase() AND table='test_with_merge' AND active; DROP TABLE test_with_merge; @@ -48,7 +48,7 @@ INSERT INTO test_replicated SELECT 1; INSERT INTO test_replicated SELECT 2; INSERT INTO test_replicated SELECT 3; -SELECT sleepEachRow(1) FROM numbers(6) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(9) FORMAT Null; SELECT count(*) FROM system.parts WHERE database = currentDatabase() AND table='test_replicated' AND active; DROP TABLE test_replicated; @@ -61,7 +61,7 @@ INSERT INTO test_replicated SELECT 1; INSERT INTO test_replicated SELECT 2; INSERT INTO test_replicated SELECT 3; -SELECT sleepEachRow(1) FROM numbers(6) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(9) FORMAT Null; SELECT count(*) FROM system.parts WHERE database = currentDatabase() AND table='test_replicated' AND active; DROP TABLE test_replicated; @@ -74,7 +74,7 @@ SYSTEM STOP MERGES test_with_merge; -- These three parts will have min_age=6 at the time of merge INSERT INTO test_with_merge SELECT 1; INSERT INTO test_with_merge SELECT 2; -SELECT sleepEachRow(1) FROM numbers(6) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(9) FORMAT Null; -- These three parts will have min_age=0 at the time of merge -- and so, nothing will be merged. INSERT INTO test_with_merge SELECT 3; From e4786a611ffc8fc5426c409c4f1e8749af446a34 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 28 Oct 2022 11:45:18 +0000 Subject: [PATCH 077/151] Add long tag --- tests/queries/0_stateless/02473_optimize_old_parts.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02473_optimize_old_parts.sql b/tests/queries/0_stateless/02473_optimize_old_parts.sql index 106175ab6f5..c2bd37033c1 100644 --- a/tests/queries/0_stateless/02473_optimize_old_parts.sql +++ b/tests/queries/0_stateless/02473_optimize_old_parts.sql @@ -1,3 +1,5 @@ +-- Tags: long + DROP TABLE IF EXISTS test_without_merge; DROP TABLE IF EXISTS test_with_merge; DROP TABLE IF EXISTS test_replicated; From fb1623a5f84eb46a3144969a7d8c6c098c8ad377 Mon Sep 17 00:00:00 2001 From: Miel Donkers Date: Fri, 28 Oct 2022 21:04:20 +0200 Subject: [PATCH 078/151] Add SSL_CERTIFICATE auth method to all places missing --- docs/en/operations/system-tables/session_log.md | 1 + docs/en/operations/system-tables/users.md | 2 +- docs/en/sql-reference/statements/alter/user.md | 2 +- docs/en/sql-reference/statements/create/user.md | 5 +++-- src/Interpreters/SessionLog.cpp | 1 + 5 files changed, 7 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/system-tables/session_log.md b/docs/en/operations/system-tables/session_log.md index 79c8ea184ce..cdf86b57ef6 100644 --- a/docs/en/operations/system-tables/session_log.md +++ b/docs/en/operations/system-tables/session_log.md @@ -24,6 +24,7 @@ Columns: - `DOUBLE_SHA1_PASSWORD` - `LDAP` - `KERBEROS` + - `SSL_CERTIFICATE` - `profiles` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — The list of profiles set for all roles and/or users. - `roles` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — The list of roles to which the profile is applied. - `settings` ([Array](../../sql-reference/data-types/array.md)([Tuple](../../sql-reference/data-types/tuple.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md), [String](../../sql-reference/data-types/string.md)))) — Settings that were changed when the client logged in/out. diff --git a/docs/en/operations/system-tables/users.md b/docs/en/operations/system-tables/users.md index eaeabab131b..6ef9b7b18a4 100644 --- a/docs/en/operations/system-tables/users.md +++ b/docs/en/operations/system-tables/users.md @@ -12,7 +12,7 @@ Columns: - `storage` ([String](../../sql-reference/data-types/string.md)) — Path to the storage of users. Configured in the `access_control_path` parameter. -- `auth_type` ([Enum8](../../sql-reference/data-types/enum.md)('no_password' = 0,'plaintext_password' = 1, 'sha256_password' = 2, 'double_sha1_password' = 3)) — Shows the authentication type. There are multiple ways of user identification: with no password, with plain text password, with [SHA256](https://ru.wikipedia.org/wiki/SHA-2)-encoded password or with [double SHA-1](https://ru.wikipedia.org/wiki/SHA-1)-encoded password. +- `auth_type` ([Enum8](../../sql-reference/data-types/enum.md)('no_password' = 0,'plaintext_password' = 1, 'sha256_password' = 2, 'double_sha1_password' = 3, 'ldap' = 4, 'kerberos' = 5, 'ssl_certificate' = 6)) — Shows the authentication type. There are multiple ways of user identification: with no password, with plain text password, with [SHA256](https://ru.wikipedia.org/wiki/SHA-2)-encoded password or with [double SHA-1](https://ru.wikipedia.org/wiki/SHA-1)-encoded password. - `auth_params` ([String](../../sql-reference/data-types/string.md)) — Authentication parameters in the JSON format depending on the `auth_type`. diff --git a/docs/en/sql-reference/statements/alter/user.md b/docs/en/sql-reference/statements/alter/user.md index 0a68885842a..31db89164d7 100644 --- a/docs/en/sql-reference/statements/alter/user.md +++ b/docs/en/sql-reference/statements/alter/user.md @@ -12,7 +12,7 @@ Syntax: ``` sql ALTER USER [IF EXISTS] name1 [ON CLUSTER cluster_name1] [RENAME TO new_name1] [, name2 [ON CLUSTER cluster_name2] [RENAME TO new_name2] ...] - [NOT IDENTIFIED | IDENTIFIED {[WITH {no_password | plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']}] + [NOT IDENTIFIED | IDENTIFIED {[WITH {no_password | plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name'}] [[ADD | DROP] HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] [DEFAULT ROLE role [,...] | ALL | ALL EXCEPT role [,...] ] [GRANTEES {user | role | ANY | NONE} [,...] [EXCEPT {user | role} [,...]]] diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md index 56a0560e57e..a756b3d4a0d 100644 --- a/docs/en/sql-reference/statements/create/user.md +++ b/docs/en/sql-reference/statements/create/user.md @@ -12,7 +12,7 @@ Syntax: ``` sql CREATE USER [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1] [, name2 [ON CLUSTER cluster_name2] ...] - [NOT IDENTIFIED | IDENTIFIED {[WITH {no_password | plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']}] + [NOT IDENTIFIED | IDENTIFIED {[WITH {no_password | plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name'}] [HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] [DEFAULT ROLE role [,...]] [DEFAULT DATABASE database | NONE] @@ -34,6 +34,7 @@ There are multiple ways of user identification: - `IDENTIFIED WITH double_sha1_hash BY 'hash'` - `IDENTIFIED WITH ldap SERVER 'server_name'` - `IDENTIFIED WITH kerberos` or `IDENTIFIED WITH kerberos REALM 'realm'` +- `IDENTIFIED WITH ssl_certificate CN 'mysite.com:user'` For identification with sha256_hash using `SALT` - hash must be calculated from concatination of 'password' and 'salt'. @@ -54,7 +55,7 @@ Another way of specifying host is to use `@` syntax following the username. Exam - `CREATE USER mira@'localhost'` — Equivalent to the `HOST LOCAL` syntax. - `CREATE USER mira@'192.168.%.%'` — Equivalent to the `HOST LIKE` syntax. -:::warning +:::warning ClickHouse treats `user_name@'address'` as a username as a whole. Thus, technically you can create multiple users with the same `user_name` and different constructions after `@`. However, we do not recommend to do so. ::: diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index 3edb84c046d..79aac63b40c 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -86,6 +86,7 @@ NamesAndTypesList SessionLogElement::getNamesAndTypes() AUTH_TYPE_NAME_AND_VALUE(AuthType::DOUBLE_SHA1_PASSWORD), AUTH_TYPE_NAME_AND_VALUE(AuthType::LDAP), AUTH_TYPE_NAME_AND_VALUE(AuthType::KERBEROS), + AUTH_TYPE_NAME_AND_VALUE(AuthType::SSL_CERTIFICATE), }); #undef AUTH_TYPE_NAME_AND_VALUE static_assert(static_cast(AuthenticationType::MAX) == 7); From 7419a3bd583ea476baa0b57ea8c66e647aa114a8 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Sat, 29 Oct 2022 18:58:11 +0800 Subject: [PATCH 079/151] fix Signed-off-by: Lloyd-Pottiger --- CMakeLists.txt | 4 ++-- cmake/tools.cmake | 11 ++++++++--- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 7a04f347b2d..d10bc63c15e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -202,7 +202,7 @@ option(ADD_GDB_INDEX_FOR_GOLD "Add .gdb-index to resulting binaries for gold lin if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") # Can be lld or ld-lld or lld-13 or /path/to/lld. - if (LINKER_NAME MATCHES "lld") + if (LINKER_NAME MATCHES "lld" AND OS_LINUX) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--gdb-index") set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--gdb-index") message (STATUS "Adding .gdb-index via --gdb-index linker option.") @@ -248,7 +248,7 @@ endif () # Create BuildID when using lld. For other linkers it is created by default. # (NOTE: LINKER_NAME can be either path or name, and in different variants) -if (LINKER_NAME MATCHES "lld") +if (LINKER_NAME MATCHES "lld" AND OS_LINUX) # SHA1 is not cryptographically secure but it is the best what lld is offering. set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--build-id=sha1") endif () diff --git a/cmake/tools.cmake b/cmake/tools.cmake index 8a17d97cf13..23f34bb24cd 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -57,14 +57,19 @@ if (NOT LINKER_NAME) if (COMPILER_GCC) find_program (LLD_PATH NAMES "ld.lld") find_program (GOLD_PATH NAMES "ld.gold") - elseif (COMPILER_CLANG) + # llvm lld is a generic driver. + # Invoke ld.lld (Unix), ld64.lld (macOS), lld-link (Windows), wasm-ld (WebAssembly) instead + elseif (COMPILER_CLANG AND OS_LINUX) find_program (LLD_PATH NAMES "ld.lld-${COMPILER_VERSION_MAJOR}" "lld-${COMPILER_VERSION_MAJOR}" "ld.lld" "lld") find_program (GOLD_PATH NAMES "ld.gold" "gold") + elseif (COMPILER_CLANG AND OS_DARWIN) + find_program (LLD_PATH NAMES "ld64.lld-${COMPILER_VERSION_MAJOR}" "lld-${COMPILER_VERSION_MAJOR}" "ld64.lld" "lld") + find_program (GOLD_PATH NAMES "ld.gold" "gold") endif () endif() -if (OS_LINUX AND NOT LINKER_NAME) - # prefer lld linker over gold or ld on linux +if ((OS_LINUX OR OS_DARWIN) AND NOT LINKER_NAME) + # prefer lld linker over gold or ld on linux and macos if (LLD_PATH) if (COMPILER_GCC) # GCC driver requires one of supported linker names like "lld". From e1e6dbe4aa92f39b6615990486cd136e4851a712 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Tue, 1 Nov 2022 03:00:11 +0100 Subject: [PATCH 080/151] Try to fix test --- tests/queries/0_stateless/02473_infile_progress.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02473_infile_progress.py b/tests/queries/0_stateless/02473_infile_progress.py index 2d6493fe4a5..c1b31eeecbb 100755 --- a/tests/queries/0_stateless/02473_infile_progress.py +++ b/tests/queries/0_stateless/02473_infile_progress.py @@ -27,7 +27,7 @@ with client(name="client>", log=log) as client1: ) client1.expect(prompt) client1.send(f"INSERT INTO test.infile_progress FROM INFILE '{filename}'") - client1.expect("Progress: 5.00 rows.*\)") + client1.expect("Progress: 0.00 rows, 10.00 B.*\)") client1.expect(prompt) # send Ctrl-C From b689f7ab70fd72473a0c73d3b1de739ff18caa95 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 1 Nov 2022 10:13:10 +0100 Subject: [PATCH 081/151] Updated registerFunction method to take TableFunctionFactoryData as input , moved table_functions_allowed_in_readonly_mode inside TableFunctionFactory- 42414 Enable table functions in readonly mode --- src/TableFunctions/TableFunctionFactory.cpp | 13 +++---------- src/TableFunctions/TableFunctionFactory.h | 11 ++++++++--- 2 files changed, 11 insertions(+), 13 deletions(-) diff --git a/src/TableFunctions/TableFunctionFactory.cpp b/src/TableFunctions/TableFunctionFactory.cpp index 79802d2ec77..3d2a72ddc9e 100644 --- a/src/TableFunctions/TableFunctionFactory.cpp +++ b/src/TableFunctions/TableFunctionFactory.cpp @@ -16,22 +16,15 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static const NameSet table_functions_allowed_in_readonly_mode -{ - "null", "view", "viewIfPermitted", "numbers", "numbers_mt", "generateRandom", "values", "cluster", "clusterAllReplicas" -}; - void TableFunctionFactory::registerFunction( - const std::string & name, TableFunctionCreator creator, Documentation doc, CaseSensitiveness case_sensitiveness) + const std::string & name, Value value, CaseSensitiveness case_sensitiveness) { - bool allowed_in_readonly_mode = table_functions_allowed_in_readonly_mode.contains(name); - - if (!table_functions.emplace(name, TableFunctionFactoryData{creator, {doc,allowed_in_readonly_mode}}).second) + if (!table_functions.emplace(name, value).second) throw Exception("TableFunctionFactory: the table function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); if (case_sensitiveness == CaseInsensitive - && !case_insensitive_table_functions.emplace(Poco::toLower(name), TableFunctionFactoryData{creator, {doc,allowed_in_readonly_mode}}).second) + && !case_insensitive_table_functions.emplace(Poco::toLower(name), value).second) throw Exception("TableFunctionFactory: the case insensitive table function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); } diff --git a/src/TableFunctions/TableFunctionFactory.h b/src/TableFunctions/TableFunctionFactory.h index 24b46d40de0..e2b8e29bd59 100644 --- a/src/TableFunctions/TableFunctionFactory.h +++ b/src/TableFunctions/TableFunctionFactory.h @@ -47,15 +47,16 @@ public: /// No locking, you must register all functions before usage of get. void registerFunction( const std::string & name, - TableFunctionCreator creator, - Documentation doc = {}, + Value value, CaseSensitiveness case_sensitiveness = CaseSensitive); template void registerFunction(Documentation doc = {}, CaseSensitiveness case_sensitiveness = CaseSensitive) { auto creator = []() -> TableFunctionPtr { return std::make_shared(); }; - registerFunction(Function::name, std::move(creator), std::move(doc), case_sensitiveness); + registerFunction(Function::name, + TableFunctionFactoryData{std::move(creator), {std::move(doc), table_functions_allowed_in_readonly_mode.contains(Function::name)}} , + case_sensitiveness); } /// Throws an exception if not found. @@ -81,6 +82,10 @@ private: TableFunctions table_functions; TableFunctions case_insensitive_table_functions; + + inline static const NameSet table_functions_allowed_in_readonly_mode = { + "null", "view", "viewIfPermitted", "numbers", "numbers_mt", "generateRandom", "values", "cluster", "clusterAllReplicas" + }; }; } From 97d361bce05f4dadcc5c7abdd56a677793461f49 Mon Sep 17 00:00:00 2001 From: SaltTan <20357526+SaltTan@users.noreply.github.com> Date: Tue, 1 Nov 2022 10:02:04 +0000 Subject: [PATCH 082/151] Update check-table.md --- docs/en/sql-reference/statements/check-table.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/check-table.md b/docs/en/sql-reference/statements/check-table.md index f9b428b74a1..9bcda724860 100644 --- a/docs/en/sql-reference/statements/check-table.md +++ b/docs/en/sql-reference/statements/check-table.md @@ -8,7 +8,7 @@ title: "CHECK TABLE Statement" Checks if the data in the table is corrupted. ``` sql -CHECK TABLE [db.]name +CHECK TABLE [db.]name PARTITION partition_expr ``` The `CHECK TABLE` query compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution. From 3afc688751a56be3e56cb2d44dd98d91d829ee48 Mon Sep 17 00:00:00 2001 From: SaltTan <20357526+SaltTan@users.noreply.github.com> Date: Tue, 1 Nov 2022 13:04:24 +0000 Subject: [PATCH 083/151] Update docs/en/sql-reference/statements/check-table.md Co-authored-by: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> --- docs/en/sql-reference/statements/check-table.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/check-table.md b/docs/en/sql-reference/statements/check-table.md index 9bcda724860..8c4b8ab90a2 100644 --- a/docs/en/sql-reference/statements/check-table.md +++ b/docs/en/sql-reference/statements/check-table.md @@ -8,7 +8,7 @@ title: "CHECK TABLE Statement" Checks if the data in the table is corrupted. ``` sql -CHECK TABLE [db.]name PARTITION partition_expr +CHECK TABLE [db.]name [PARTITION partition_expr] ``` The `CHECK TABLE` query compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution. From e58008b711acc08ba4f17b611d877ec14be95e00 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 1 Nov 2022 18:10:31 +0100 Subject: [PATCH 084/151] Fix rare possible hung on query cancellation Some queries can hung after cancelling (because enough rows had been read for Distributed queries) or after KILL. Example of such query is a Distributed query that uses DelayedSource that uses ExpandPipeline to add new processors, and in this case it is possible that all already existing processes was cancelled but new had been added and now PipelineExecutor will wait for them undefinitelly since nobody will read from them and nobody will cancel them either. Signed-off-by: Azat Khuzhin --- src/Processors/Executors/ExecutingGraph.cpp | 8 ++++++++ src/Processors/Executors/ExecutingGraph.h | 1 + 2 files changed, 9 insertions(+) diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index 4ab2c5b3802..cd94ca7ceae 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -109,6 +109,13 @@ bool ExecutingGraph::expandPipeline(std::stack & stack, uint64_t pid) { std::lock_guard guard(processors_mutex); + /// Do not add new processors to existing list, since the query was already cancelled. + if (cancelled) + { + for (auto & processor : new_processors) + processor->cancel(); + return false; + } processors->insert(processors->end(), new_processors.begin(), new_processors.end()); } @@ -388,6 +395,7 @@ void ExecutingGraph::cancel() std::lock_guard guard(processors_mutex); for (auto & processor : *processors) processor->cancel(); + cancelled = true; } } diff --git a/src/Processors/Executors/ExecutingGraph.h b/src/Processors/Executors/ExecutingGraph.h index b374f968122..834ef5d4d9d 100644 --- a/src/Processors/Executors/ExecutingGraph.h +++ b/src/Processors/Executors/ExecutingGraph.h @@ -157,6 +157,7 @@ private: UpgradableMutex nodes_mutex; const bool profile_processors; + bool cancelled = false; }; } From e06572cc27f22536805fae16cd1cb95bee70e4b6 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 1 Nov 2022 23:03:07 +0000 Subject: [PATCH 085/151] Validate Query Tree in debug --- src/Analyzer/IQueryTreePass.h | 2 +- src/Analyzer/QueryTreePassManager.cpp | 55 ++++++++++++++++++++------- 2 files changed, 43 insertions(+), 14 deletions(-) diff --git a/src/Analyzer/IQueryTreePass.h b/src/Analyzer/IQueryTreePass.h index 39b3d743ed3..4293934c32d 100644 --- a/src/Analyzer/IQueryTreePass.h +++ b/src/Analyzer/IQueryTreePass.h @@ -16,7 +16,7 @@ namespace DB * Dependencies between passes must be avoided. */ class IQueryTreePass; -using QueryTreePassPtr = std::shared_ptr; +using QueryTreePassPtr = std::unique_ptr; using QueryTreePasses = std::vector; class IQueryTreePass diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 853b4a23f38..1c4679e799c 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -18,6 +18,8 @@ #include #include +#include +#include namespace DB { @@ -27,6 +29,28 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace +{ + +#ifndef NDEBUG + +// This visitor checks if Query Tree structure is valid after each pass +// in debug build. +class ValidationChecker : public InDepthQueryTreeVisitor +{ +public: + void visitImpl(QueryTreeNodePtr & node) const + { + auto * column = node->as(); + if (!column) + return; + column->getColumnSource(); + } +}; +#endif + +} + /** ClickHouse query tree pass manager. * * TODO: Support _shard_num into shardNum() rewriting. @@ -61,7 +85,12 @@ void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node) size_t passes_size = passes.size(); for (size_t i = 0; i < passes_size; ++i) + { passes[i]->run(query_tree_node, current_context); +#ifndef NDEBUG + ValidationChecker().visit(query_tree_node); +#endif + } } void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node, size_t up_to_pass_index) @@ -114,38 +143,38 @@ void addQueryTreePasses(QueryTreePassManager & manager) auto context = manager.getContext(); const auto & settings = context->getSettingsRef(); - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_functions_to_subcolumns) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.count_distinct_optimization) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_rewrite_sum_if_to_count_if) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_normalize_count_variants) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_arithmetic_operations_in_aggregate_functions) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_injective_functions_inside_uniq) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_multiif_to_if) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_if_chain_to_multiif) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); - manager.addPass(std::make_shared()); - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); } } From ec382d7903fb64b5d8c85ce93792c52071299000 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 2 Nov 2022 09:50:41 +0800 Subject: [PATCH 086/151] add comment --- src/Storages/MergeTree/KeyCondition.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 15283eeb435..3bea261fd52 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -56,6 +56,7 @@ String Range::toString() const /// Example: for `Hello\_World% ...` string it returns `Hello_World`, and for `%test%` returns an empty string. +/// If perfect_prefix_match == true, only consider pattern in the format `prefix%_` String extractFixedPrefixFromLikePattern(const String & like_pattern, bool perfect_prefix_match) { String fixed_prefix; From c4b717b343cee23a329cfb7905d56aa1f2407b43 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 2 Nov 2022 08:24:54 +0100 Subject: [PATCH 087/151] Update src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp Co-authored-by: Sergei Trifonov --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 2b123cfbad9..fcc1b4cb3e2 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -364,7 +364,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( auto best_partition_it = std::max_element( partitions_info.begin(), partitions_info.end(), - [](const auto & e1, const auto & e2) { return e1.second.min_age > e2.second.min_age; }); + [](const auto & e1, const auto & e2) { return e1.second.min_age < e2.second.min_age; }); assert(best_partition_it != partitions_info.end()); From 550c776ffe493b1915d5c436734b6ff44ca492bf Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 2 Nov 2022 13:16:27 +0100 Subject: [PATCH 088/151] Wait for all files are in sync before archiving them --- tests/integration/ci-runner.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 4eab305358b..551466cf583 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -353,6 +353,11 @@ class ClickhouseIntegrationTestsRunner: ) def _compress_logs(self, dir, relpaths, result_path): + # We execute sync in advance to have all files written after containers + # are finished or killed + subprocess.check_call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL + "sync", shell=True + ) subprocess.check_call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL "tar czf {} -C {} {}".format(result_path, dir, " ".join(relpaths)), shell=True, From 204dac3d5d9d423192e171773952c2983574b29f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Nov 2022 17:14:20 +0300 Subject: [PATCH 089/151] Suggest using https://fiddle.clickhouse.com/ --- .github/ISSUE_TEMPLATE/85_bug-report.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/ISSUE_TEMPLATE/85_bug-report.md b/.github/ISSUE_TEMPLATE/85_bug-report.md index 3d2ed6148e3..08d03c284ca 100644 --- a/.github/ISSUE_TEMPLATE/85_bug-report.md +++ b/.github/ISSUE_TEMPLATE/85_bug-report.md @@ -13,6 +13,8 @@ assignees: '' > A clear and concise description of what works not as it is supposed to. +> A link to reproducer in [https://fiddle.clickhouse.com/](https://fiddle.clickhouse.com/). + **Does it reproduce on recent release?** [The list of releases](https://github.com/ClickHouse/ClickHouse/blob/master/utils/list-versions/version_date.tsv) From a860b252492cc7840f23bf374e930a4048dd4c61 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 2 Nov 2022 15:24:09 +0000 Subject: [PATCH 090/151] Improve ValidationChecker readability --- src/Analyzer/QueryTreePassManager.cpp | 23 +++++++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 1c4679e799c..d59e7457ed2 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -20,6 +20,7 @@ #include #include #include +#include namespace DB { @@ -34,17 +35,26 @@ namespace #ifndef NDEBUG -// This visitor checks if Query Tree structure is valid after each pass -// in debug build. +/** This visitor checks if Query Tree structure is valid after each pass + * in debug build. + */ class ValidationChecker : public InDepthQueryTreeVisitor { + String pass_name; public: + explicit ValidationChecker(String pass_name_) + : pass_name(std::move(pass_name_)) + {} + void visitImpl(QueryTreeNodePtr & node) const { auto * column = node->as(); if (!column) return; - column->getColumnSource(); + if (column->getColumnSourceOrNull() == nullptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Column {} {} query tree node does not have valid source node after running {} pass", + column->getColumnName(), column->getColumnType(), pass_name); } }; #endif @@ -88,7 +98,7 @@ void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node) { passes[i]->run(query_tree_node, current_context); #ifndef NDEBUG - ValidationChecker().visit(query_tree_node); + ValidationChecker(passes[i]->getName()).visit(query_tree_node); #endif } } @@ -104,7 +114,12 @@ void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node, size_t up_to_pa auto current_context = getContext(); for (size_t i = 0; i < up_to_pass_index; ++i) + { passes[i]->run(query_tree_node, current_context); +#ifndef NDEBUG + ValidationChecker(passes[i]->getName()).visit(query_tree_node); +#endif + } } void QueryTreePassManager::dump(WriteBuffer & buffer) From 67707230272d0b47659769edd9a8414795115e24 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 3 Nov 2022 12:02:31 +0800 Subject: [PATCH 091/151] modify as requested --- docs/en/sql-reference/functions/string-functions.md | 2 +- src/Functions/ascii.cpp | 6 +++--- tests/queries/0_stateless/02353_ascii.reference | 2 ++ tests/queries/0_stateless/02353_ascii.sql | 2 ++ 4 files changed, 8 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 982ba05f494..8b9f25e2f98 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1155,7 +1155,7 @@ Do Nothing for 2 Minutes 2:00   Returns the ASCII code point of the first character of str. The result type is Int32. -If s is empty, the result is 0. If the first character is not an ASCII character or part of the Latin-1 Supplement range of UTF-16, the result is undefined. +If s is empty, the result is 0. If the first character is not an ASCII character or not part of the Latin-1 Supplement range of UTF-16, the result is undefined. diff --git a/src/Functions/ascii.cpp b/src/Functions/ascii.cpp index 38ba7f0bbac..abf9078057e 100644 --- a/src/Functions/ascii.cpp +++ b/src/Functions/ascii.cpp @@ -37,7 +37,7 @@ struct AsciiImpl [[noreturn]] static void vectorFixedToConstant(const ColumnString::Chars & /*data*/, size_t /*n*/, Int32 & /*res*/) { - throw Exception("vectorFixedToConstant not implemented for function " + std::string(AsciiName::name), ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "vectorFixedToConstant not implemented for function {}", std::string(AsciiName::name)); } static void vectorFixedToVector(const ColumnString::Chars & data, size_t n, PaddedPODArray & res) @@ -52,12 +52,12 @@ struct AsciiImpl [[noreturn]] static void array(const ColumnString::Offsets & /*offsets*/, PaddedPODArray & /*res*/) { - throw Exception("Cannot apply function " + std::string(AsciiName::name) + " to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot apply function {} to Array argument", std::string(AsciiName::name)); } [[noreturn]] static void uuid(const ColumnUUID::Container & /*offsets*/, size_t /*n*/, PaddedPODArray & /*res*/) { - throw Exception("Cannot apply function " + std::string(AsciiName::name) + " to UUID argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot apply function {} to UUID argument", std::string(AsciiName::name)); } private: diff --git a/tests/queries/0_stateless/02353_ascii.reference b/tests/queries/0_stateless/02353_ascii.reference index b85ac563483..79588517e2a 100644 --- a/tests/queries/0_stateless/02353_ascii.reference +++ b/tests/queries/0_stateless/02353_ascii.reference @@ -1,5 +1,7 @@ 50 +0 50 +0 48 49 50 diff --git a/tests/queries/0_stateless/02353_ascii.sql b/tests/queries/0_stateless/02353_ascii.sql index 936b0d460db..5b7a20ad61c 100644 --- a/tests/queries/0_stateless/02353_ascii.sql +++ b/tests/queries/0_stateless/02353_ascii.sql @@ -1,3 +1,5 @@ SELECT ascii('234'); +SELECT ascii(''); SELECT ascii(materialize('234')); +SELECT ascii(materialize('')); SELECT ascii(toString(number) || 'abc') from numbers(10); From f39cb41d3e1f138180caf4b1f340954015a27396 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 3 Nov 2022 16:44:57 +0800 Subject: [PATCH 092/151] change as requested --- src/Functions/ascii.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/ascii.cpp b/src/Functions/ascii.cpp index abf9078057e..cb59be55cc1 100644 --- a/src/Functions/ascii.cpp +++ b/src/Functions/ascii.cpp @@ -37,7 +37,7 @@ struct AsciiImpl [[noreturn]] static void vectorFixedToConstant(const ColumnString::Chars & /*data*/, size_t /*n*/, Int32 & /*res*/) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "vectorFixedToConstant not implemented for function {}", std::string(AsciiName::name)); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "vectorFixedToConstant not implemented for function {}", AsciiName::name); } static void vectorFixedToVector(const ColumnString::Chars & data, size_t n, PaddedPODArray & res) @@ -52,12 +52,12 @@ struct AsciiImpl [[noreturn]] static void array(const ColumnString::Offsets & /*offsets*/, PaddedPODArray & /*res*/) { - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot apply function {} to Array argument", std::string(AsciiName::name)); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot apply function {} to Array argument", AsciiName::name); } [[noreturn]] static void uuid(const ColumnUUID::Container & /*offsets*/, size_t /*n*/, PaddedPODArray & /*res*/) { - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot apply function {} to UUID argument", std::string(AsciiName::name)); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot apply function {} to UUID argument", AsciiName::name); } private: @@ -76,7 +76,7 @@ REGISTER_FUNCTION(Ascii) R"( Returns the ASCII code point of the first character of str. The result type is Int32. -If s is empty, the result is 0. If the first character is not an ASCII character or part of the Latin-1 Supplement range of UTF-16, the result is undefined) +If s is empty, the result is 0. If the first character is not an ASCII character or not part of the Latin-1 Supplement range of UTF-16, the result is undefined) )", Documentation::Examples{{"ascii", "SELECT ascii('234')"}}, Documentation::Categories{"String"} From e6a387e7b2fb753f335ab130f5de2ea59633c8d5 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 3 Nov 2022 12:31:07 +0100 Subject: [PATCH 093/151] Updated registerFunction factory method and removed table_functions_allowed_in_readonly_mode - 42414 Enable table functions in readonly mode --- src/TableFunctions/TableFunctionExplain.cpp | 4 ++-- src/TableFunctions/TableFunctionFactory.h | 8 ++------ src/TableFunctions/TableFunctionGenerateRandom.cpp | 2 +- src/TableFunctions/TableFunctionNull.cpp | 2 +- src/TableFunctions/TableFunctionNumbers.cpp | 4 ++-- src/TableFunctions/TableFunctionRemote.cpp | 4 ++-- src/TableFunctions/TableFunctionValues.cpp | 2 +- src/TableFunctions/TableFunctionView.cpp | 2 +- src/TableFunctions/TableFunctionViewIfPermitted.cpp | 2 +- src/TableFunctions/TableFunctionZeros.cpp | 8 ++++---- 10 files changed, 17 insertions(+), 21 deletions(-) diff --git a/src/TableFunctions/TableFunctionExplain.cpp b/src/TableFunctions/TableFunctionExplain.cpp index 02493aa9b0e..02b9308ed22 100644 --- a/src/TableFunctions/TableFunctionExplain.cpp +++ b/src/TableFunctions/TableFunctionExplain.cpp @@ -91,7 +91,7 @@ InterpreterExplainQuery TableFunctionExplain::getInterpreter(ContextPtr context) void registerTableFunctionExplain(TableFunctionFactory & factory) { - factory.registerFunction({R"( + factory.registerFunction({.documentation = {R"( Returns result of EXPLAIN query. The function should not be called directly but can be invoked via `SELECT * FROM (EXPLAIN )`. @@ -103,7 +103,7 @@ Example: )", {{"1", "SELECT explain FROM (EXPLAIN AST SELECT * FROM system.numbers) WHERE explain LIKE '%Asterisk%'"}} -}); +}}); } diff --git a/src/TableFunctions/TableFunctionFactory.h b/src/TableFunctions/TableFunctionFactory.h index e2b8e29bd59..2cc648ba181 100644 --- a/src/TableFunctions/TableFunctionFactory.h +++ b/src/TableFunctions/TableFunctionFactory.h @@ -51,11 +51,11 @@ public: CaseSensitiveness case_sensitiveness = CaseSensitive); template - void registerFunction(Documentation doc = {}, CaseSensitiveness case_sensitiveness = CaseSensitive) + void registerFunction(TableFunctionProperties properties = {}, CaseSensitiveness case_sensitiveness = CaseSensitive) { auto creator = []() -> TableFunctionPtr { return std::make_shared(); }; registerFunction(Function::name, - TableFunctionFactoryData{std::move(creator), {std::move(doc), table_functions_allowed_in_readonly_mode.contains(Function::name)}} , + TableFunctionFactoryData{std::move(creator), {std::move(properties)}} , case_sensitiveness); } @@ -82,10 +82,6 @@ private: TableFunctions table_functions; TableFunctions case_insensitive_table_functions; - - inline static const NameSet table_functions_allowed_in_readonly_mode = { - "null", "view", "viewIfPermitted", "numbers", "numbers_mt", "generateRandom", "values", "cluster", "clusterAllReplicas" - }; }; } diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index 8d1c06c7c4d..1ddbb48962d 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -91,7 +91,7 @@ StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & /*ast_functio void registerTableFunctionGenerate(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction({.documentation = {}, .allow_readonly = true}); } } diff --git a/src/TableFunctions/TableFunctionNull.cpp b/src/TableFunctions/TableFunctionNull.cpp index 1a35830b06c..9ff07cc1946 100644 --- a/src/TableFunctions/TableFunctionNull.cpp +++ b/src/TableFunctions/TableFunctionNull.cpp @@ -52,6 +52,6 @@ StoragePtr TableFunctionNull::executeImpl(const ASTPtr & /*ast_function*/, Conte void registerTableFunctionNull(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction({.documentation = {}, .allow_readonly = true}); } } diff --git a/src/TableFunctions/TableFunctionNumbers.cpp b/src/TableFunctions/TableFunctionNumbers.cpp index 0ef1d3815d9..2056cd838f5 100644 --- a/src/TableFunctions/TableFunctionNumbers.cpp +++ b/src/TableFunctions/TableFunctionNumbers.cpp @@ -51,8 +51,8 @@ StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_f void registerTableFunctionNumbers(TableFunctionFactory & factory) { - factory.registerFunction>(); - factory.registerFunction>(); + factory.registerFunction>({.documentation = {}, .allow_readonly = true}); + factory.registerFunction>({.documentation = {}, .allow_readonly = true}); } template diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 097a239ccae..ff19485274c 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -323,8 +323,8 @@ void registerTableFunctionRemote(TableFunctionFactory & factory) { factory.registerFunction("remote", [] () -> TableFunctionPtr { return std::make_shared("remote"); }); factory.registerFunction("remoteSecure", [] () -> TableFunctionPtr { return std::make_shared("remote", /* secure = */ true); }); - factory.registerFunction("cluster", [] () -> TableFunctionPtr { return std::make_shared("cluster"); }); - factory.registerFunction("clusterAllReplicas", [] () -> TableFunctionPtr { return std::make_shared("clusterAllReplicas"); }); + factory.registerFunction("cluster", {[] () -> TableFunctionPtr { return std::make_shared("cluster"); }, {.documentation = {}, .allow_readonly = true}}); + factory.registerFunction("clusterAllReplicas", {[] () -> TableFunctionPtr { return std::make_shared("clusterAllReplicas"); }, {.documentation = {}, .allow_readonly = true}}); } } diff --git a/src/TableFunctions/TableFunctionValues.cpp b/src/TableFunctions/TableFunctionValues.cpp index 7b8de69a4f8..05574825275 100644 --- a/src/TableFunctions/TableFunctionValues.cpp +++ b/src/TableFunctions/TableFunctionValues.cpp @@ -147,7 +147,7 @@ StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, Context void registerTableFunctionValues(TableFunctionFactory & factory) { - factory.registerFunction({}, TableFunctionFactory::CaseInsensitive); + factory.registerFunction({.documentation = {}, .allow_readonly = true}, TableFunctionFactory::CaseInsensitive); } } diff --git a/src/TableFunctions/TableFunctionView.cpp b/src/TableFunctions/TableFunctionView.cpp index c53d26a794b..c999cba08e9 100644 --- a/src/TableFunctions/TableFunctionView.cpp +++ b/src/TableFunctions/TableFunctionView.cpp @@ -55,7 +55,7 @@ StoragePtr TableFunctionView::executeImpl( void registerTableFunctionView(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction({.documentation = {}, .allow_readonly = true}); } } diff --git a/src/TableFunctions/TableFunctionViewIfPermitted.cpp b/src/TableFunctions/TableFunctionViewIfPermitted.cpp index ba3d2cb9d16..6128fe0a36f 100644 --- a/src/TableFunctions/TableFunctionViewIfPermitted.cpp +++ b/src/TableFunctions/TableFunctionViewIfPermitted.cpp @@ -107,7 +107,7 @@ bool TableFunctionViewIfPermitted::isPermitted(const ContextPtr & context, const void registerTableFunctionViewIfPermitted(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction({.documentation = {}, .allow_readonly = true}); } } diff --git a/src/TableFunctions/TableFunctionZeros.cpp b/src/TableFunctions/TableFunctionZeros.cpp index 126166b547b..5874fca67e6 100644 --- a/src/TableFunctions/TableFunctionZeros.cpp +++ b/src/TableFunctions/TableFunctionZeros.cpp @@ -48,7 +48,7 @@ StoragePtr TableFunctionZeros::executeImpl(const ASTPtr & ast_fun void registerTableFunctionZeros(TableFunctionFactory & factory) { - factory.registerFunction>({R"( + factory.registerFunction>({.documentation = {R"( Generates a stream of zeros (a table with one column 'zero' of type 'UInt8') of specified size. This table function is used in performance tests, where you want to spend as little time as possible to data generation while testing some other parts of queries. @@ -62,9 +62,9 @@ This query will test the speed of `randomPrintableASCII` function using single t See also the `system.zeros` table. )", {{"1", "SELECT count() FROM zeros(100000000) WHERE NOT ignore(randomPrintableASCII(10))"}} -}); +}}); - factory.registerFunction>({R"( + factory.registerFunction>({.documentation = {R"( Generates a stream of zeros (a table with one column 'zero' of type 'UInt8') of specified size. This table function is used in performance tests, where you want to spend as little time as possible to data generation while testing some other parts of queries. @@ -78,7 +78,7 @@ This query will test the speed of `randomPrintableASCII` function using multiple See also the `system.zeros` table. )", {{"1", "SELECT count() FROM zeros_mt(1000000000) WHERE NOT ignore(randomPrintableASCII(10))"}} -}); +}}); } template From e68a3b0c7addd3bd2fb49d163c1599e91f322a7f Mon Sep 17 00:00:00 2001 From: Yakko Majuri <38760734+yakkomajuri@users.noreply.github.com> Date: Thu, 3 Nov 2022 11:25:07 -0300 Subject: [PATCH 094/151] fix(typo): Passible -> Possible --- src/Storages/Kafka/KafkaSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 51e5ee47077..e0da8031dd8 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -30,7 +30,7 @@ class ASTStorage; M(String, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine", 0) \ M(UInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block", 0) \ M(Bool, kafka_thread_per_consumer, false, "Provide independent thread for each consumer", 0) \ - M(HandleKafkaErrorMode, kafka_handle_error_mode, HandleKafkaErrorMode::DEFAULT, "How to handle errors for Kafka engine. Passible values: default, stream.", 0) \ + M(HandleKafkaErrorMode, kafka_handle_error_mode, HandleKafkaErrorMode::DEFAULT, "How to handle errors for Kafka engine. Possible values: default, stream.", 0) \ M(Bool, kafka_commit_on_select, false, "Commit messages when select query is made", 0) \ /** TODO: */ From 7186898ffaa86d909d8de838991f30d5f3a152bd Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 4 Nov 2022 00:29:17 +0800 Subject: [PATCH 095/151] Experiment --- src/Storages/MergeTree/MergeTreeData.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 83e87a0e462..c62d6337dd9 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5783,7 +5783,7 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg const auto & analysis_result = select.getAnalysisResult(); query_info.prepared_sets = select.getQueryAnalyzer()->getPreparedSets(); - query_info.prewhere_info = analysis_result.prewhere_info; + // query_info.prewhere_info = analysis_result.prewhere_info; const auto & before_where = analysis_result.before_where; const auto & where_column_name = analysis_result.where_column_name; @@ -6173,8 +6173,8 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg selected_candidate->aggregate_descriptions = select.getQueryAnalyzer()->aggregates(); } - /// Just in case, reset prewhere info calculated from projection. - query_info.prewhere_info.reset(); + // /// Just in case, reset prewhere info calculated from projection. + // query_info.prewhere_info.reset(); return *selected_candidate; } From a9f8948c8defc7775929b32bed349435ff279e46 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Thu, 3 Nov 2022 17:28:41 +0000 Subject: [PATCH 096/151] Fix build --- src/Storages/StorageDelta.cpp | 82 +++++++++++++++++++---------------- src/Storages/StorageDelta.h | 9 ++-- 2 files changed, 49 insertions(+), 42 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 027312385bb..222cbaa5377 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -12,6 +12,9 @@ # include # include +#include + +#include # include # include @@ -168,19 +171,18 @@ void JsonMetadataGetter::handleJSON(const JSON & json) } StorageDelta::StorageDelta( - const S3::URI & uri_, - const String & access_key_, - const String & secret_access_key_, + const StorageS3Configuration & configuration_, const StorageID & table_id_, - const String & format_name_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_) + ContextPtr context_, + std::optional format_settings_) : IStorage(table_id_) - , base_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) + , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} , log(&Poco::Logger::get("StorageDeltaLake (" + table_id_.table_name + ")")) - , table_path(uri_.key) + , table_path(base_configuration.uri.key) + , format_name(configuration_.format) { StorageInMemoryMetadata storage_metadata; StorageS3::updateS3Configuration(context_, base_configuration); @@ -193,32 +195,38 @@ StorageDelta::StorageDelta( LOG_DEBUG(log, "New uri: {}", new_uri); LOG_DEBUG(log, "Table path: {}", table_path); - auto s3_uri = S3::URI(Poco::URI(new_uri)); + + // set new url in configuration + StorageS3Configuration new_configuration; + new_configuration.url = new_uri; + new_configuration.auth_settings.access_key_id = configuration_.auth_settings.access_key_id; + new_configuration.auth_settings.secret_access_key= configuration_.auth_settings.secret_access_key; + new_configuration.format = configuration_.format; + if (columns_.empty()) { - columns_ = StorageS3::getTableStructureFromData(format_name_, s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + columns_ = StorageS3::getTableStructureFromData(new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr); storage_metadata.setColumns(columns_); } else storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); s3engine = std::make_shared( - s3_uri, - access_key_, - secret_access_key_, - table_id_, - format_name_, - base_configuration.rw_settings, - columns_, - constraints_, - comment, - context_, - std::nullopt); + new_configuration, + table_id_, + columns_, + constraints_, + comment, + context_, + format_settings_, + /* distributed_processing_ */false, + nullptr); } Pipe StorageDelta::read( @@ -253,29 +261,29 @@ void registerStorageDelta(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage DeltaLake requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); - - String table_url = checkAndGetLiteralArgument(engine_args[0], "url"); - String access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); - String secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); - - String format = "Parquet"; - if (engine_args.size() == 4) - { - format = checkAndGetLiteralArgument(engine_args[3], "format"); - } - - auto s3_uri = S3::URI(Poco::URI(table_url)); + StorageS3Configuration configuration; + + configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); + configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + + if (engine_args.size() == 4) + configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); + + if (configuration.format == "auto") + configuration.format = "Parquet"; + + //auto format_settings = getFormatSettings(args.getContext()); return std::make_shared( - s3_uri, - access_key_id, - secret_access_key, + configuration, args.table_id, - format, args.columns, args.constraints, args.comment, - args.getContext()); + args.getContext(), + std::nullopt); + }, { .supports_settings = true, diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index c5dc0b2fd07..ff5986ef505 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -64,15 +64,13 @@ class StorageDelta : public IStorage { public: StorageDelta( - const S3::URI & uri_, - const String & access_key_, - const String & secret_access_key_, + const StorageS3Configuration & configuration_, const StorageID & table_id_, - const String & format_name_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_); + ContextPtr context_, + std::optional format_settings_); String getName() const override { return "DeltaLake"; } @@ -93,6 +91,7 @@ private: std::shared_ptr s3engine; Poco::Logger * log; String table_path; + String format_name; }; } From 7b57e94d9f5687bc544d6ad2b0e1d3e692f85425 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Thu, 3 Nov 2022 18:14:10 +0000 Subject: [PATCH 097/151] Fix StorageHudi --- src/Storages/StorageHudi.cpp | 79 ++++++++++++++++++------------------ src/Storages/StorageHudi.h | 8 ++-- 2 files changed, 43 insertions(+), 44 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 4aff4ff3a43..d36f0c3c737 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -8,7 +8,7 @@ # include # include # include - +#include # include # include # include @@ -25,34 +25,37 @@ namespace ErrorCodes } StorageHudi::StorageHudi( - const S3::URI & uri_, - const String & access_key_, - const String & secret_access_key_, + const StorageS3Configuration & configuration_, const StorageID & table_id_, - const String & format_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_) + ContextPtr context_, + std::optional format_settings_) : IStorage(table_id_) - , base_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) + , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} , log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")")) - , table_path(uri_.key) + , table_path(base_configuration.uri.key) { StorageInMemoryMetadata storage_metadata; StorageS3::updateS3Configuration(context_, base_configuration); auto keys = getKeysFromS3(); - auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys), format_); + auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys), configuration_.format); LOG_DEBUG(log, "New uri: {}", new_uri); LOG_DEBUG(log, "Table path: {}", table_path); - auto s3_uri = S3::URI(Poco::URI(new_uri)); - + + StorageS3Configuration new_configuration; + new_configuration.url = new_uri; + new_configuration.auth_settings.access_key_id = configuration_.auth_settings.access_key_id; + new_configuration.auth_settings.secret_access_key= configuration_.auth_settings.secret_access_key; + new_configuration.format = configuration_.format; + if (columns_.empty()) { - columns_ = StorageS3::getTableStructureFromData(format_, s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + columns_ = StorageS3::getTableStructureFromData(new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr); storage_metadata.setColumns(columns_); } else @@ -63,17 +66,15 @@ StorageHudi::StorageHudi( setInMemoryMetadata(storage_metadata); s3engine = std::make_shared( - s3_uri, - access_key_, - secret_access_key_, - table_id_, - format_, - base_configuration.rw_settings, - columns_, - constraints_, - comment, - context_, - std::nullopt); + new_configuration, + table_id_, + columns_, + constraints_, + comment, + context_, + format_settings_, + /* distributed_processing_ */false, + nullptr); } Pipe StorageHudi::read( @@ -207,28 +208,28 @@ void registerStorageHudi(StorageFactory & factory) "Storage Hudi requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); - String table_url = checkAndGetLiteralArgument(engine_args[0], "url"); - String access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); - String secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); - - String format = "Parquet"; - if (engine_args.size() == 4) - { - format = checkAndGetLiteralArgument(engine_args[3], "format"); - } - - auto s3_uri = S3::URI(Poco::URI(table_url)); + StorageS3Configuration configuration; + + configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); + configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + + if (engine_args.size() == 4) + configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); + + if (configuration.format == "auto") + configuration.format = "Parquet"; + + auto format_settings = getFormatSettings(args.getContext()); return std::make_shared( - s3_uri, - access_key_id, - secret_access_key, + configuration, args.table_id, - format, args.columns, args.constraints, args.comment, - args.getContext()); + args.getContext(), + format_settings); }, { .supports_settings = true, diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index 7b647345441..0ed1935a36c 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -24,15 +24,13 @@ class StorageHudi : public IStorage { public: StorageHudi( - const S3::URI & uri_, - const String & access_key_, - const String & secret_access_key_, + const StorageS3Configuration & configuration_, const StorageID & table_id_, - const String & format_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_); + ContextPtr context_, + std::optional format_settings_); String getName() const override { return "Hudi"; } From a573d8aef76e6761d97d0befef9c3563adc08ff0 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Thu, 3 Nov 2022 18:17:39 +0000 Subject: [PATCH 098/151] Apply clang-format --- src/Storages/StorageDelta.cpp | 51 +++++++++++++++-------------------- src/Storages/StorageDelta.h | 1 - src/Storages/StorageHudi.cpp | 47 +++++++++++++++----------------- 3 files changed, 43 insertions(+), 56 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 222cbaa5377..01598996161 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -10,11 +10,11 @@ # include # include +# include # include # include -#include -#include +# include # include # include @@ -182,7 +182,6 @@ StorageDelta::StorageDelta( , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} , log(&Poco::Logger::get("StorageDeltaLake (" + table_id_.table_name + ")")) , table_path(base_configuration.uri.key) - , format_name(configuration_.format) { StorageInMemoryMetadata storage_metadata; StorageS3::updateS3Configuration(context_, base_configuration); @@ -195,38 +194,39 @@ StorageDelta::StorageDelta( LOG_DEBUG(log, "New uri: {}", new_uri); LOG_DEBUG(log, "Table path: {}", table_path); - + // set new url in configuration StorageS3Configuration new_configuration; new_configuration.url = new_uri; new_configuration.auth_settings.access_key_id = configuration_.auth_settings.access_key_id; - new_configuration.auth_settings.secret_access_key= configuration_.auth_settings.secret_access_key; + new_configuration.auth_settings.secret_access_key = configuration_.auth_settings.secret_access_key; new_configuration.format = configuration_.format; if (columns_.empty()) { - columns_ = StorageS3::getTableStructureFromData(new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr); + columns_ = StorageS3::getTableStructureFromData( + new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr); storage_metadata.setColumns(columns_); } else storage_metadata.setColumns(columns_); - + storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); s3engine = std::make_shared( - new_configuration, - table_id_, - columns_, - constraints_, - comment, - context_, - format_settings_, - /* distributed_processing_ */false, - nullptr); + new_configuration, + table_id_, + columns_, + constraints_, + comment, + context_, + format_settings_, + /* distributed_processing_ */ false, + nullptr); } Pipe StorageDelta::read( @@ -262,28 +262,21 @@ void registerStorageDelta(StorageFactory & factory) "Storage DeltaLake requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); StorageS3Configuration configuration; - + configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); - - if (engine_args.size() == 4) + + if (engine_args.size() == 4) configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); - + if (configuration.format == "auto") configuration.format = "Parquet"; - + //auto format_settings = getFormatSettings(args.getContext()); return std::make_shared( - configuration, - args.table_id, - args.columns, - args.constraints, - args.comment, - args.getContext(), - std::nullopt); - + configuration, args.table_id, args.columns, args.constraints, args.comment, args.getContext(), std::nullopt); }, { .supports_settings = true, diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index ff5986ef505..3f20a850526 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -91,7 +91,6 @@ private: std::shared_ptr s3engine; Poco::Logger * log; String table_path; - String format_name; }; } diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index d36f0c3c737..57e6461fb72 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -5,10 +5,10 @@ # include # include +# include # include # include # include -#include # include # include # include @@ -46,16 +46,17 @@ StorageHudi::StorageHudi( LOG_DEBUG(log, "New uri: {}", new_uri); LOG_DEBUG(log, "Table path: {}", table_path); - + StorageS3Configuration new_configuration; new_configuration.url = new_uri; new_configuration.auth_settings.access_key_id = configuration_.auth_settings.access_key_id; - new_configuration.auth_settings.secret_access_key= configuration_.auth_settings.secret_access_key; + new_configuration.auth_settings.secret_access_key = configuration_.auth_settings.secret_access_key; new_configuration.format = configuration_.format; - + if (columns_.empty()) { - columns_ = StorageS3::getTableStructureFromData(new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr); + columns_ = StorageS3::getTableStructureFromData( + new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr); storage_metadata.setColumns(columns_); } else @@ -66,15 +67,15 @@ StorageHudi::StorageHudi( setInMemoryMetadata(storage_metadata); s3engine = std::make_shared( - new_configuration, - table_id_, - columns_, - constraints_, - comment, - context_, - format_settings_, - /* distributed_processing_ */false, - nullptr); + new_configuration, + table_id_, + columns_, + constraints_, + comment, + context_, + format_settings_, + /* distributed_processing_ */ false, + nullptr); } Pipe StorageHudi::read( @@ -209,27 +210,21 @@ void registerStorageHudi(StorageFactory & factory) StorageS3Configuration configuration; - + configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); - - if (engine_args.size() == 4) + + if (engine_args.size() == 4) configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); - + if (configuration.format == "auto") configuration.format = "Parquet"; - + auto format_settings = getFormatSettings(args.getContext()); return std::make_shared( - configuration, - args.table_id, - args.columns, - args.constraints, - args.comment, - args.getContext(), - format_settings); + configuration, args.table_id, args.columns, args.constraints, args.comment, args.getContext(), format_settings); }, { .supports_settings = true, From cdd09fd72a4378cade1586394faa3a8465e72809 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Nov 2022 19:23:19 +0100 Subject: [PATCH 099/151] Improve messages for loading data parts (add part type) Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeData.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 83e87a0e462..bb12f3a62a6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1019,13 +1019,14 @@ void MergeTreeData::loadDataPartsFromDisk( if (!part_opt) return; - LOG_TRACE(log, "Loading part {} from disk {}", part_name, part_disk_ptr->getName()); const auto & part_info = *part_opt; auto single_disk_volume = std::make_shared("volume_" + part_name, part_disk_ptr, 0); auto data_part_storage = std::make_shared(single_disk_volume, relative_data_path, part_name); auto part = createPart(part_name, part_info, data_part_storage); bool broken = false; + LOG_TRACE(log, "Loading part {} ({}) from disk {}", part_name, part->getType().toString(), part_disk_ptr->getName()); + String part_path = fs::path(relative_data_path) / part_name; String marker_path = fs::path(part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; if (part_disk_ptr->exists(marker_path)) From 7032b2b98a48a876d7d8af1f465fc7331b4af293 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Nov 2022 19:35:16 +0100 Subject: [PATCH 100/151] Attach thread pool for loading parts to the query Detach should be done via CurrentThread::defaultThreadDeleter() Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeData.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bb12f3a62a6..51d8097c1db 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1131,8 +1131,11 @@ void MergeTreeData::loadDataPartsFromDisk( { for (size_t thread = 0; thread < num_threads; ++thread) { - pool.scheduleOrThrowOnError([&, thread] + pool.scheduleOrThrowOnError([&, thread, thread_group = CurrentThread::getGroup()] { + if (thread_group) + CurrentThread::attachToIfDetached(thread_group); + while (true) { std::pair thread_part; From d901ead1bcb375e45bc492d8af2925eb7f138287 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 4 Nov 2022 17:26:44 +0800 Subject: [PATCH 101/151] Done --- src/Storages/MergeTree/MergeTreeData.cpp | 7 ++++--- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 3 ++- src/Storages/MergeTree/MergeTreeDataSelectExecutor.h | 1 + .../0_stateless/01710_projection_in_index.reference | 1 + .../queries/0_stateless/01710_projection_in_index.sql | 10 ++++++++++ 5 files changed, 18 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c62d6337dd9..977133a8ad8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5426,6 +5426,7 @@ static void selectBestProjection( auto projection_result_ptr = reader.estimateNumMarksToRead( projection_parts, + candidate.prewhere_info, candidate.required_columns, storage_snapshot->metadata, candidate.desc->metadata, @@ -5449,6 +5450,7 @@ static void selectBestProjection( { auto normal_result_ptr = reader.estimateNumMarksToRead( normal_parts, + query_info.prewhere_info, required_columns, storage_snapshot->metadata, storage_snapshot->metadata, @@ -5783,7 +5785,6 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg const auto & analysis_result = select.getAnalysisResult(); query_info.prepared_sets = select.getQueryAnalyzer()->getPreparedSets(); - // query_info.prewhere_info = analysis_result.prewhere_info; const auto & before_where = analysis_result.before_where; const auto & where_column_name = analysis_result.where_column_name; @@ -6060,6 +6061,7 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg { auto normal_result_ptr = reader.estimateNumMarksToRead( normal_parts, + query_info.prewhere_info, analysis_result.required_columns, metadata_snapshot, metadata_snapshot, @@ -6092,6 +6094,7 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg { query_info.merge_tree_select_result_ptr = reader.estimateNumMarksToRead( parts, + query_info.prewhere_info, analysis_result.required_columns, metadata_snapshot, metadata_snapshot, @@ -6173,8 +6176,6 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg selected_candidate->aggregate_descriptions = select.getQueryAnalyzer()->aggregates(); } - // /// Just in case, reset prewhere info calculated from projection. - // query_info.prewhere_info.reset(); return *selected_candidate; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index afdd98b8e41..674e02b16ec 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1294,6 +1294,7 @@ static void selectColumnNames( MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMarksToRead( MergeTreeData::DataPartsVector parts, + const PrewhereInfoPtr & prewhere_info, const Names & column_names_to_return, const StorageMetadataPtr & metadata_snapshot_base, const StorageMetadataPtr & metadata_snapshot, @@ -1318,7 +1319,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar return ReadFromMergeTree::selectRangesToRead( std::move(parts), - query_info.prewhere_info, + prewhere_info, added_filter_nodes, metadata_snapshot_base, metadata_snapshot, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 541f6446674..e302663597d 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -56,6 +56,7 @@ public: /// This method is used to select best projection for table. MergeTreeDataSelectAnalysisResultPtr estimateNumMarksToRead( MergeTreeData::DataPartsVector parts, + const PrewhereInfoPtr & prewhere_info, const Names & column_names, const StorageMetadataPtr & metadata_snapshot_base, const StorageMetadataPtr & metadata_snapshot, diff --git a/tests/queries/0_stateless/01710_projection_in_index.reference b/tests/queries/0_stateless/01710_projection_in_index.reference index 73c1df53be4..4be49ff0513 100644 --- a/tests/queries/0_stateless/01710_projection_in_index.reference +++ b/tests/queries/0_stateless/01710_projection_in_index.reference @@ -1,2 +1,3 @@ 1 1 1 2 2 2 +1 diff --git a/tests/queries/0_stateless/01710_projection_in_index.sql b/tests/queries/0_stateless/01710_projection_in_index.sql index 2669d69dc9f..87f5e79e37e 100644 --- a/tests/queries/0_stateless/01710_projection_in_index.sql +++ b/tests/queries/0_stateless/01710_projection_in_index.sql @@ -9,3 +9,13 @@ set allow_experimental_projection_optimization = 1, max_rows_to_read = 3; select * from t where i < 5 and j in (1, 2); drop table t; + +drop table if exists test; + +create table test (name String, time Int64) engine MergeTree order by time; + +insert into test values ('hello world', 1662336000241); + +select count() from (select fromUnixTimestamp64Milli(time, 'UTC') time_fmt, name from test where time_fmt > '2022-09-05 00:00:00'); + +drop table test; From a89e8475145d5e4197079d319bf3f274aaa830d0 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 1 Nov 2022 22:33:52 +0800 Subject: [PATCH 102/151] Fix getauxval for sanitizer builds --- base/glibc-compatibility/musl/getauxval.c | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index 22886013d07..aaaca0465ea 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -8,6 +8,8 @@ #include // ElfW #include +#include "syscall.h" + #define ARRAY_SIZE(a) sizeof((a))/sizeof((a[0])) /// Suppress TSan since it is possible for this code to be called from multiple threads, @@ -39,7 +41,9 @@ ssize_t __retry_read(int fd, void * buf, size_t count) { for (;;) { - ssize_t ret = read(fd, buf, count); + // We cannot use the read syscall as it will be intercept by sanitizers, which aren't + // initialized yet. Emit syscall directly. + ssize_t ret = __syscall_ret(__syscall(SYS_read, fd, buf, count)); if (ret == -1) { if (errno == EINTR) From 4c2b3de93d1b0b2df78c9647a94c45d7bffb557e Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 4 Nov 2022 20:21:50 +0800 Subject: [PATCH 103/151] Fix msan error --- base/glibc-compatibility/musl/getauxval.c | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index aaaca0465ea..eba12604b4d 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -94,6 +94,11 @@ static unsigned long NO_SANITIZE_THREAD __auxv_init_procfs(unsigned long type) _Static_assert(sizeof(aux) < 4096, "Unexpected sizeof(aux)"); while (__retry_read(fd, &aux, sizeof(aux)) == sizeof(aux)) { +#if defined(__has_feature) +#if __has_feature(memory_sanitizer) + __msan_unpoison(&aux, sizeof(aux)); +#endif +#endif if (aux.a_type == AT_NULL) { break; From 9c066e964d2d3a2abd07e99d1406e8b2d975d909 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 4 Nov 2022 15:52:48 +0000 Subject: [PATCH 104/151] Less use of CH-specific bit_cast() Converted usage of CH-custom bit_cast to std::bit_cast if possible, i.e. when sizeof(From) == sizeof(To). (The CH-custom bit_cast is able to deal with sizeof(From) != sizeof(To).) Motivation for this came from #42847 where it is not clear how the internal bit_cast should behave on big endian systems, so we better avoid that situation as much as possible. --- programs/library-bridge/ExternalDictionaryLibraryHandler.cpp | 3 +-- programs/library-bridge/ExternalDictionaryLibraryUtils.h | 1 - programs/obfuscator/Obfuscator.cpp | 5 ++--- src/AggregateFunctions/QuantileBFloat16Histogram.h | 5 ++--- src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 1 - src/Functions/FunctionsRound.h | 1 - src/Functions/isFinite.cpp | 5 ++--- src/Functions/isInfinite.cpp | 5 ++--- src/Functions/padString.cpp | 5 ++--- src/Interpreters/BloomFilterHash.h | 1 - .../MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp | 1 - src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp | 1 - src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp | 1 - 13 files changed, 11 insertions(+), 24 deletions(-) diff --git a/programs/library-bridge/ExternalDictionaryLibraryHandler.cpp b/programs/library-bridge/ExternalDictionaryLibraryHandler.cpp index 14850da2ebf..c60d4a4e5cc 100644 --- a/programs/library-bridge/ExternalDictionaryLibraryHandler.cpp +++ b/programs/library-bridge/ExternalDictionaryLibraryHandler.cpp @@ -1,7 +1,6 @@ #include "ExternalDictionaryLibraryHandler.h" #include -#include #include #include @@ -113,7 +112,7 @@ Block ExternalDictionaryLibraryHandler::loadAll() Block ExternalDictionaryLibraryHandler::loadIds(const std::vector & ids) { - const ExternalDictionaryLibraryAPI::VectorUInt64 ids_data{bit_cast(ids.data()), ids.size()}; + const ExternalDictionaryLibraryAPI::VectorUInt64 ids_data{std::bit_cast(ids.data()), ids.size()}; auto columns_holder = std::make_unique(attributes_names.size()); ExternalDictionaryLibraryAPI::CStrings columns_pass{static_cast(columns_holder.get()), attributes_names.size()}; diff --git a/programs/library-bridge/ExternalDictionaryLibraryUtils.h b/programs/library-bridge/ExternalDictionaryLibraryUtils.h index e813efab2a6..c9d03d27f75 100644 --- a/programs/library-bridge/ExternalDictionaryLibraryUtils.h +++ b/programs/library-bridge/ExternalDictionaryLibraryUtils.h @@ -2,7 +2,6 @@ #include #include -#include #include #include "ExternalDictionaryLibraryAPI.h" diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 7fdc5a54d8a..b6952ad6cb0 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -32,7 +32,6 @@ #include #include #include -#include #include #include #include @@ -278,9 +277,9 @@ Float transformFloatMantissa(Float x, UInt64 seed) using UInt = std::conditional_t, UInt32, UInt64>; constexpr size_t mantissa_num_bits = std::is_same_v ? 23 : 52; - UInt x_uint = bit_cast(x); + UInt x_uint = std::bit_cast(x); x_uint = static_cast(feistelNetwork(x_uint, mantissa_num_bits, seed)); - return bit_cast(x_uint); + return std::bit_cast(x_uint); } diff --git a/src/AggregateFunctions/QuantileBFloat16Histogram.h b/src/AggregateFunctions/QuantileBFloat16Histogram.h index 8ec325e238d..de9f61e01a2 100644 --- a/src/AggregateFunctions/QuantileBFloat16Histogram.h +++ b/src/AggregateFunctions/QuantileBFloat16Histogram.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include @@ -104,13 +103,13 @@ private: /// Take the most significant 16 bits of the floating point number. BFloat16 toBFloat16(const Value & x) const { - return bit_cast(static_cast(x)) >> 16; + return std::bit_cast(static_cast(x)) >> 16; } /// Put the bits into most significant 16 bits of the floating point number and fill other bits with zeros. Float32 toFloat32(const BFloat16 & x) const { - return bit_cast(x << 16); + return std::bit_cast(x << 16); } using Pair = PairNoInit; diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index ed9199a359f..29f5719e3ed 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -24,7 +24,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 283f1ea5a43..2c7883cf471 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -16,7 +16,6 @@ #include #include #include -#include #include #include diff --git a/src/Functions/isFinite.cpp b/src/Functions/isFinite.cpp index 90185b64fff..612e7067bf5 100644 --- a/src/Functions/isFinite.cpp +++ b/src/Functions/isFinite.cpp @@ -1,6 +1,5 @@ #include #include -#include #include @@ -20,11 +19,11 @@ struct IsFiniteImpl static bool execute(const T t) { if constexpr (std::is_same_v) - return (bit_cast(t) + return (std::bit_cast(t) & 0b01111111100000000000000000000000) != 0b01111111100000000000000000000000; else if constexpr (std::is_same_v) - return (bit_cast(t) + return (std::bit_cast(t) & 0b0111111111110000000000000000000000000000000000000000000000000000) != 0b0111111111110000000000000000000000000000000000000000000000000000; else diff --git a/src/Functions/isInfinite.cpp b/src/Functions/isInfinite.cpp index e923e1461bc..ace2c334873 100644 --- a/src/Functions/isInfinite.cpp +++ b/src/Functions/isInfinite.cpp @@ -1,6 +1,5 @@ #include #include -#include #include @@ -16,11 +15,11 @@ struct IsInfiniteImpl static bool execute(const T t) { if constexpr (std::is_same_v) - return (bit_cast(t) + return (std::bit_cast(t) & 0b01111111111111111111111111111111) == 0b01111111100000000000000000000000; else if constexpr (std::is_same_v) - return (bit_cast(t) + return (std::bit_cast(t) & 0b0111111111111111111111111111111111111111111111111111111111111111) == 0b0111111111110000000000000000000000000000000000000000000000000000; else diff --git a/src/Functions/padString.cpp b/src/Functions/padString.cpp index c8ed920755c..486fa328fa0 100644 --- a/src/Functions/padString.cpp +++ b/src/Functions/padString.cpp @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB { @@ -59,10 +58,10 @@ namespace { if (num_chars <= step) { - writeSlice(StringSource::Slice{bit_cast(pad_string.data()), numCharsToNumBytes(num_chars)}, res_sink); + writeSlice(StringSource::Slice{std::bit_cast(pad_string.data()), numCharsToNumBytes(num_chars)}, res_sink); break; } - writeSlice(StringSource::Slice{bit_cast(pad_string.data()), numCharsToNumBytes(step)}, res_sink); + writeSlice(StringSource::Slice{std::bit_cast(pad_string.data()), numCharsToNumBytes(step)}, res_sink); num_chars -= step; } } diff --git a/src/Interpreters/BloomFilterHash.h b/src/Interpreters/BloomFilterHash.h index 31532cc888a..b95abbfd770 100644 --- a/src/Interpreters/BloomFilterHash.h +++ b/src/Interpreters/BloomFilterHash.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp index b19080b5097..9bc0e4e6dc0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index 33668b96a60..c2ed081ac00 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp index f80d40d2fa8..7efaf0866db 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include From d6bbd65ddea49d67ccea3dd187f5437bcf5ddafc Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Fri, 4 Nov 2022 16:03:12 +0000 Subject: [PATCH 105/151] Fix broken include --- src/Storages/StorageDelta.cpp | 4 ++-- src/Storages/StorageDelta.h | 4 ++-- src/Storages/StorageHudi.cpp | 4 ++-- src/Storages/StorageHudi.h | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 01598996161..7ab522860e8 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -1,4 +1,4 @@ -#include +#include "config.h" #if USE_AWS_S3 @@ -236,7 +236,7 @@ Pipe StorageDelta::read( ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { StorageS3::updateS3Configuration(context, base_configuration); diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index 3f20a850526..d0992dd4cb7 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -1,6 +1,6 @@ #pragma once -#include "config_core.h" +#include "config.h" #if USE_AWS_S3 @@ -81,7 +81,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; private: void Init(); diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 57e6461fb72..e2a46fad2d0 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -1,4 +1,4 @@ -#include +#include "config.h" #if USE_AWS_S3 @@ -85,7 +85,7 @@ Pipe StorageHudi::read( ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { StorageS3::updateS3Configuration(context, base_configuration); return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index 0ed1935a36c..91a77ec83ff 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -1,6 +1,6 @@ #pragma once -#include "config_core.h" +#include "config.h" #if USE_AWS_S3 @@ -41,7 +41,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; private: std::vector getKeysFromS3(); From 5cb69d8a22a9e4b70fc574eaea416041483e48e5 Mon Sep 17 00:00:00 2001 From: Aleksandr Musorin Date: Wed, 2 Nov 2022 11:04:58 +0100 Subject: [PATCH 106/151] changed type name for S3_Plain storage renamed a disk for S3PlainObjectStorage in system.disks table from s3 to s3_plain --- src/Disks/DiskType.h | 3 +++ src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 4 +++- .../test_backup_restore_s3/test.py | 19 +++++++++++++++++++ 3 files changed, 25 insertions(+), 1 deletion(-) diff --git a/src/Disks/DiskType.h b/src/Disks/DiskType.h index 037b65f8e07..4d099e33a7a 100644 --- a/src/Disks/DiskType.h +++ b/src/Disks/DiskType.h @@ -11,6 +11,7 @@ enum class DataSourceType Local, RAM, S3, + S3_Plain, HDFS, WebServer, AzureBlobStorage, @@ -26,6 +27,8 @@ inline String toString(DataSourceType data_source_type) return "memory"; case DataSourceType::S3: return "s3"; + case DataSourceType::S3_Plain: + return "s3_plain"; case DataSourceType::HDFS: return "hdfs"; case DataSourceType::WebServer: diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 6b1e8289b15..56f1c895924 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -213,7 +213,9 @@ public: template S3PlainObjectStorage(Args && ...args) : S3ObjectStorage("S3PlainObjectStorage", std::forward(args)...) - {} + { + data_source_description.type = DataSourceType::S3_Plain; + } }; } diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 7ddb1459ab9..ee9c458c044 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -50,6 +50,24 @@ def check_backup_and_restore(storage_policy, backup_destination, size=1000): ) +def check_system_tables(): + disks = [ + tuple(disk.split("\t")) + for disk in node.query("SELECT name, type FROM system.disks").split("\n") + if disk + ] + expected_disks = ( + ("default", "local"), + ("disk_s3", "s3"), + ("disk_s3_other_bucket", "s3"), + ("disk_s3_plain", "s3_plain"), + ) + assert len(expected_disks) == len(disks) + for expected_disk in expected_disks: + if expected_disk not in disks: + raise AssertionError(f"Missed {expected_disk} in {disks}") + + @pytest.mark.parametrize( "storage_policy, to_disk", [ @@ -93,6 +111,7 @@ def test_backup_to_s3(): f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" ) check_backup_and_restore(storage_policy, backup_destination) + check_system_tables() def test_backup_to_s3_named_collection(): From c98731a19b2e7af00480a768175ce111a9cceaa9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 4 Nov 2022 19:22:04 +0100 Subject: [PATCH 107/151] Remove some utils --- utils/CMakeLists.txt | 4 - utils/db-generator/CMakeLists.txt | 2 - utils/db-generator/README.md | 35 - utils/db-generator/query_db_generator.cpp | 1354 ----------------- utils/iotest/CMakeLists.txt | 9 - utils/iotest/iotest.cpp | 197 --- utils/iotest/iotest_aio.cpp | 203 --- utils/iotest/iotest_nonblock.cpp | 177 --- .../CMakeLists.txt | 3 - .../main.cpp | 286 ---- .../CMakeLists.txt | 2 - .../main.cpp | 47 - 12 files changed, 2319 deletions(-) delete mode 100644 utils/db-generator/CMakeLists.txt delete mode 100644 utils/db-generator/README.md delete mode 100644 utils/db-generator/query_db_generator.cpp delete mode 100644 utils/iotest/CMakeLists.txt delete mode 100644 utils/iotest/iotest.cpp delete mode 100644 utils/iotest/iotest_aio.cpp delete mode 100644 utils/iotest/iotest_nonblock.cpp delete mode 100644 utils/zookeeper-adjust-block-numbers-to-parts/CMakeLists.txt delete mode 100644 utils/zookeeper-adjust-block-numbers-to-parts/main.cpp delete mode 100644 utils/zookeeper-create-entry-to-download-part/CMakeLists.txt delete mode 100644 utils/zookeeper-create-entry-to-download-part/main.cpp diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 92a97a9c60e..70c32c67063 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -20,17 +20,13 @@ add_subdirectory (report) # Not used in package if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS) add_subdirectory (compressor) - add_subdirectory (iotest) add_subdirectory (corrector_utf8) add_subdirectory (zookeeper-cli) add_subdirectory (zookeeper-dump-tree) add_subdirectory (zookeeper-remove-by-list) - add_subdirectory (zookeeper-create-entry-to-download-part) - add_subdirectory (zookeeper-adjust-block-numbers-to-parts) add_subdirectory (wikistat-loader) add_subdirectory (check-marks) add_subdirectory (checksum-for-compressed-block) - add_subdirectory (db-generator) add_subdirectory (wal-dump) add_subdirectory (check-mysql-binlog) add_subdirectory (keeper-bench) diff --git a/utils/db-generator/CMakeLists.txt b/utils/db-generator/CMakeLists.txt deleted file mode 100644 index 45780717752..00000000000 --- a/utils/db-generator/CMakeLists.txt +++ /dev/null @@ -1,2 +0,0 @@ -clickhouse_add_executable (query_db_generator query_db_generator.cpp) -target_link_libraries(query_db_generator PRIVATE clickhouse_parsers boost::program_options) diff --git a/utils/db-generator/README.md b/utils/db-generator/README.md deleted file mode 100644 index 5596aac66e4..00000000000 --- a/utils/db-generator/README.md +++ /dev/null @@ -1,35 +0,0 @@ -# Clickhouse query analysis - -Here we will consider only `SELECT` queries, i.e. those queries that get data from the table. -The built-in Clickhouse parser accepts a string as input, which is a query. Among 14 main clauses of `SELECT` statement: `WITH`, `SELECT`, `TABLES`, `PREWHERE`, `WHERE`, `GROUP_BY`, `HAVING`, `ORDER_BY`, `LIMIT_BY_OFFSET`, `LIMIT_BY_LENGTH`, `LIMIT_BY`, `LIMIT_OFFSET`, `LIMIT_LENGTH`, `SETTINGS`, we will analyze the `SELECT`, `TABLES`, `WHERE`, `GROUP_BY`, `HAVING`, `ORDER_BY` clauses because the most of data is there. We need this data to analyze the structure and to identify values. The parser issues a tree structure after parsing a query, where each node is a specific query execution operation, a function over values, a constant, a designation, etc. Nodes also have subtrees where their arguments or suboperations are located. We will try to reveal the data we need by avoiding this tree. - -## Scheme analysis - -It is necessary to determine possible tables by a query. Having a query string, you can understand which parts of it represent the names of the tables, so you can determine their number in our database. -In the Clickhouse parser, `TABLES` (Figure 1) is a query subtree responsible for tables where we get data. It contains the main table where the columns come from, as well as the `JOIN` operations that are performed in the query. Avoiding all nodes in the subtree, we use the names of the tables and databases where they are located, as well as their alias, i.e. the shortened names chosen by the query author. We may need these names to determine the ownership of the column in the future. -Thus, we get a set of databases for the query, as well as tables and their aliases, with the help of them a query is made. - -Then we need to define the set of columns that are in the query and the tables they can refer to. The set of columns in each table is already known during the query execution. Therefore, the program automatically links the column and table at runtime. However, in our case, it is impossible to unambiguously interpret the belonging of a column to a specific table, for example, in the following query `SELECT column1, column2, column3 FROM table1 JOIN table2 on table1.column2 = table2.column3`. In this case, we can say which table `column2` and `column3` belong to. However, `column1` can belong to either the first or the second table. We will refer undefined columns to the main table, on which a query is made, for unambiguous interpretation of such cases. For example, in this case, it will be `table1`. -All columns in the tree are in `IDENTIFIER` type nodes, which are in the `SELECT`, `TABLES`, `WHERE`, `GROUP_BY`, `HAVING`, `ORDER_BY` subtrees. We form a set of all tables recursively avoiding the subtrees, then we split the column into constituents such as the table (if it is explicitly specified with a dot) and the name. Then, since the table can be an alias, we replace the alias with the original table name. We now have a list of all the columns and tables they belong to. We define the main query table for non-table columns. - -## Column analysis - -Then we need to exactly define data types for columns that have a value in the query. An example is the boolean `WHERE` clause where we test boolean expressions in its attributes. If the query specifies `column > 5`, then we can conclude that this column contains a numeric value, or if the `LIKE` expression is applied to the attribute, then the attribute has a string type. -In this part, you need to learn how to extract such expressions from a query and match data types for columns, where it is possible. At the same time, it is clear that it is not always possible to make an unambiguous decision about the type of a particular attribute from the available values. For example, `column > 5` can mean many numeric types such as `UINT8`, `UINT32`, `INT32`, `INT64`, etc. It is necessary to determine the interpretation of certain values since searching through all possible values ​​can be quite large and long. -It can take a long time to iterate over all possible values, so we use `INT64` and `FLOAT64` types for numeric values, `STRING` for strings, `DATE` and `DATETIME` for dates, and `ARRAY`. -We can determine column values ​​using boolean, arithmetic and other functions on the column values ​​that are specified in the query. Such functions are in the `SELECT` and `WHERE` subtrees. The function parameter can be a constant, a column or another function (Figure 2). Thus, the following parameters can help to understand the type of the column: -- The types of arguments that a function can take, for example, the `TOSTARTOFMINUTE` function (truncate time up to a multiple of 5 minutes down) can only accept `DATETIME`, so if the argument of this function is a column, then this column has `DATETIME` type. -- The types of the remaining arguments in this function. For example, the `EQUALS` function means equality of its argument types, so if a constant and a column are present in this function, then we can define the type of the column as the type of the constant. - -Thus, we define the possible argument types, the return type, the parameter for each function, and the function arguments of the identical type. The recursive function handler will determine the possible types of columns used in these functions by the values of the arguments, and then return the possible types of the function's result. -Now, for each column, we have many possible types of values. We will choose one specific type from this set to interpret the query unambiguously. - -## Column values definition - -At this stage, we already have a certain structure of the database tables, we need to fill this table with values. We should understand which columns depend on each other when executing the function (for example, the join is done according to two columns, which means that they must have the same values). We also need to understand what values ​​the columns must have to fulfill various conditions during execution. -We search for all comparison operations in our query to achieve the goal. If the arguments of the operation are two columns, then we consider them linked. If the arguments are the column and the value, then we assign that value to the possible column value and add the value with some noise. A random number is a noise for a numeric type, it is a random number of days for a date, etc. In this case, a handler for this operation is required for each comparison operation, which generates at least two values, one of them is the operation condition, and the other is not. For example, a value greater than 5 and less than or equal to 5 must be assigned for the operation `column1 > 5`, `column1`, for the operation `column2 LIKE some% string` the same is true. The satisfying and not satisfying expression must be assigned to `column2`. -Now we have many associated columns and many values. We know that the connectivity of columns is symmetric, but we need to add transitivity for a complete definition, because if `column1 = column2` and `column2 = column3`, then `column1 = column3`, but this does not follow from the construction. Accordingly, we need to extend the connectivity across all columns. We combine multiple values for each column with the values associated with it. If we have columns with no values, then we generate random values. - -## Generation - -We have a complete view of the database schema as well as many values ​​for each table now. We will generate data by cartesian product of the value set of each column for a specific table. Thus, we get a set for each table, consisting of sets of values for each column. We start generating queries that create this table and fill it with data. We generate the `CREATE QUERY` that creates this table based on the structure of the table and the types of its columns, and then we generate the `INSERT QUERY` over the set of values, which fills the table with data. diff --git a/utils/db-generator/query_db_generator.cpp b/utils/db-generator/query_db_generator.cpp deleted file mode 100644 index 00785af89f7..00000000000 --- a/utils/db-generator/query_db_generator.cpp +++ /dev/null @@ -1,1354 +0,0 @@ -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - - -namespace po = boost::program_options; - -using ColumnType = uint32_t; -using TableAndColumn = std::pair; -pcg64 rng; - -std::string randomString(size_t length) -{ - auto randchar = []() -> char - { - const char charset[] = "0123456789" "ABCDEFGHIJKLMNOPQRSTUVWXYZ" "abcdefghijklmnopqrstuvwxyz"; - const size_t max_index = (sizeof(charset) - 1); - return charset[rng() % max_index]; - }; - std::string str(length, 0); - std::generate_n(str.begin(), length, randchar); - return str; -} -std::string randomInteger(unsigned int min = 0, unsigned int max = 4294967295) -{ - int r = rng() % (max - min) + min; - return std::to_string(r); -} - -std::string randomFloat(unsigned int min = 0, unsigned int max = 4294967295) -{ - float r = static_cast(rng() % max) / (static_cast(rng() % 100)) + min; - return std::to_string(r); -} - -std::string randomDate() -{ - int32_t year = rng() % 136 + 1970; - int32_t month = rng() % 12 + 1; - int32_t day = rng() % 12 + 1; - char answer[13]; - size_t size = sprintf(answer, "'%04u-%02u-%02u'", year, month, day); - return std::string(answer, size); -} - -std::string randomDatetime() -{ - int32_t year = rng() % 136 + 1970; - int32_t month = rng() % 12 + 1; - int32_t day = rng() % 12 + 1; - int32_t hours = rng() % 24; - int32_t minutes = rng() % 60; - int32_t seconds = rng() % 60; - char answer[22]; - size_t size = sprintf( - answer, - "'%04u-%02u-%02u %02u:%02u:%02u'", - year, - month, - day, - hours, - minutes, - seconds); - return std::string(answer, size); -} -TableAndColumn get_table_a_column(const std::string & c) -{ - auto point_place = c.rfind('.'); - std::string db{}; - std::string column{}; - if (point_place != std::string::npos) - { - db = c.substr(0, point_place); - column = c.substr(point_place + 1); - } - else - { - column = c; - } - return { db, column }; -} - - -enum Type : ColumnType -{ - i = 1, - // int - f = 2, - // float - s = 4, - // string - d = 8, - // date - dt = 16, - // datetime - b = 32, - // bool - all = 63, - a = 64, - // array - t = 128, - // tuple -}; - - -std::map type_definition = -{ - {Type::i, "Int64"}, {Type::f, "Float64"}, {Type::s, "String"}, {Type::d, "Date"}, {Type::dt, "DateTime"}, {Type::b, "UInt8"} -}; - -ColumnType time_type(std::string value) -{ - if (value.length() == 12) - { - for (size_t i : {5, 8}) - { - if (value[i] != '-') - return Type::s; - } - for (size_t i : {1, 2, 3, 4, 6, 7, 9, 10}) - { - if (!isdigit(value[i])) - return Type::s; - } - return Type::d; - } - - if (value.length() == 21) - { - for (size_t i : {5, 8}) - { - if (value[i] != '-') - return Type::s; - } - for (size_t i : {14, 17}) - { - if (value[i] != '-') - return Type::s; - } - if (value[11] != '-') - return Type::s; - return Type::dt; - } - return Type::s; -} -// Casting inner clickhouse parser type to our type -ColumnType type_cast(int t) -{ - switch (t) - { - case 1: - case 2: - case 4: - case 5: - case 19: - case 20: - case 21: - return Type::i; - - case 3: - return Type::f; - - case 16: - return Type::s; - - case 17: - return Type::a | Type::all; - - case 18: - return Type::t | Type::all; - } - return Type::all; -} - - -class FuncRet -{ -public: - FuncRet() = default; - - FuncRet(ColumnType t, std::string v) - : value(v) - , type(t) {} - - FuncRet(ColumnType t, std::string v, bool is_a) - : value(v) - , type(t) - , is_array(is_a) {} - - std::string value{}; - ColumnType type = Type::all; - bool is_array = false; -}; - - -std::map func_to_return_type = { - {"divide", FuncRet(Type::f, "")}, {"e", FuncRet(Type::f, "e()")}, {"pi", FuncRet(Type::f, "pi()")}, {"exp", FuncRet(Type::f, "")}, - {"log", FuncRet(Type::f,"")}, {"exp2", FuncRet(Type::f, "")}, {"log2", FuncRet(Type::f, "")}, {"exp10", FuncRet(Type::f, "")}, - {"log10", FuncRet(Type::f, "")}, {"sqrt", FuncRet(Type::f, "")}, {"cbrt", FuncRet(Type::f, "")}, {"erf", FuncRet(Type::f, "")}, - {"erfc", FuncRet(Type::f, "")}, {"lgamma", FuncRet(Type::f, "")}, {"tgamma", FuncRet(Type::f, "")}, {"sin", FuncRet(Type::f, "")}, - {"cos", FuncRet(Type::f, "")}, {"tan", FuncRet(Type::f, "")}, {"asin", FuncRet(Type::f, "")}, {"acos", FuncRet(Type::f, "")}, - {"atan", FuncRet(Type::f, "")}, {"pow", FuncRet(Type::f, "")}, {"splitbystring", FuncRet(Type::s | Type::a,"")}, - {"splitbychar", FuncRet(Type::s | Type::a, "")}, {"alphatokens", FuncRet(Type::s | Type::a, "")}, {"toyear", FuncRet(Type::i, "")}, - {"tomonth", FuncRet(Type::i, "")}, {"todayofmonth", FuncRet(Type::i, "")}, {"tohour", FuncRet(Type::dt, "")}, {"tominute", FuncRet(Type::dt, "")}, - {"toseconds", FuncRet(Type::dt, "")}, {"tounixtimestamp", FuncRet(Type::i, "")}, {"tostartofyear", FuncRet(Type::dt | Type::d, "")}, - {"tostartofquater",FuncRet(Type::dt | Type::d, "")}, {"tostartofmonth", FuncRet(Type::dt | Type::d, "")}, {"tomonday", FuncRet(Type::dt | Type::d, "")}, - {"tostartoffiveminutes", FuncRet(Type::dt, "")}, {"tostartoftenminutes", FuncRet(Type::dt, "")}, {"tostartoffifteenminutes", FuncRet(Type::dt, "")}, - {"tostartofinterval", FuncRet(Type::dt, "")}, {"totime", FuncRet(Type::dt, "")}, {"torelativemonthnum", FuncRet(Type::i, "")}, - {"torelativeweeknum", FuncRet(Type::i, "")}, {"torelativedaynum", FuncRet(Type::i, "")}, {"torelativehournum", FuncRet(Type::i, "")}, - {"torelativeminutenum", FuncRet(Type::i, "")}, {"torelativesecondsnum", FuncRet(Type::i, "")}, {"datediff", FuncRet(Type::d | Type::dt, "")}, - {"formatdatetime", FuncRet(Type::s, "")}, {"now", FuncRet(Type::dt | Type::d, "now()")}, {"today", FuncRet(Type::d | Type::dt, "today()")}, - {"yesterday", FuncRet(Type::d | Type::dt, "yesterday()")}, {"tolastdayofmonth", FuncRet(Type::dt | Type::d, "")} -}; - -std::set func_args_same_types = { - "equals", "notequals", "less", "greater", "lessorequals", "greaterorequals", "multiply" -}; - -std::map func_to_param_type = { - {"tostartofminute", Type::dt}, {"plus", Type::i | Type::f | Type::d | Type::dt}, {"multiply", Type::i | Type::f}, - {"minus", Type::i | Type::f | Type::d | Type::dt}, {"negate", Type::i | Type::f}, {"divide", Type::i | Type::f}, - {"abs", Type::i | Type::f}, {"gcd", Type::i | Type::f}, {"lcm", Type::i | Type::f}, {"bitnot", Type::i}, {"bitshiftleft", Type::i}, - {"bitshiftright", Type::i}, {"bittest", Type::i}, {"exp", Type::i | Type::f}, {"log", Type::i | Type::f}, - {"exp2", Type::i | Type::f}, {"log2", Type::i | Type::f}, {"exp10", Type::i | Type::f}, {"log10", Type::i | Type::f}, - {"sqrt", Type::i | Type::f}, {"cbrt", Type::i | Type::f}, {"erf", Type::i | Type::f}, {"erfc", Type::i | Type::f}, - {"lgamma", Type::i | Type::f}, {"tgamma", Type::i | Type::f}, {"sin", Type::i | Type::f}, {"cos", Type::i | Type::f}, - {"tan", Type::i | Type::f}, {"asin", Type::i | Type::f}, {"acos", Type::i | Type::f}, {"atan", Type::i | Type::f}, - {"pow", Type::i | Type::f}, {"arrayjoin", Type::all | Type::a}, {"substring", Type::s}, {"splitbystring", Type::s}, {"splitbychar", Type::s}, - {"alphatokens", Type::s}, {"toyear", Type::d | Type::dt}, {"tomonth", Type::d | Type::dt}, {"todayofmonth", Type::d | Type::dt}, {"tohour", Type::dt}, - {"tominute", Type::dt}, {"tosecond", Type::dt}, {"touixtimestamp", Type::dt}, {"tostartofyear", Type::d | Type::dt}, - {"tostartofquarter", Type::d | Type::dt}, {"tostartofmonth", Type::d | Type::dt}, {"tomonday", Type::d | Type::dt}, - {"tostartoffiveminutes", Type::dt}, {"tostartoftenminutes", Type::dt}, {"tostartoffifteenminutes", Type::d | Type::dt}, - {"tostartofinterval", Type::d | Type::dt}, {"totime", Type::d | Type::dt}, {"torelativehonthnum", Type::d | Type::dt}, - {"torelativeweeknum", Type::d | Type::dt}, {"torelativedaynum", Type::d | Type::dt}, {"torelativehournum", Type::d | Type::dt}, - {"torelativeminutenum", Type::d | Type::dt}, {"torelativesecondnum", Type::d | Type::dt}, {"datediff", Type::d | Type::dt}, - {"formatdatetime", Type::dt}, {"tolastdayofmonth", Type::d | Type::dt} -}; - - -class Column -{ -public: - TableAndColumn name; - std::set equals; - std::set values; - ColumnType type = Type::all; - bool is_array = false; - - Column() = default; - - explicit Column(const std::string & column_name) - { - name = std::make_pair("", column_name); - type = Type::all; - } - - void merge(Column other) - { - if (name.second.empty()) - name = other.name; - equals.insert(other.equals.begin(), other.equals.end()); - values.insert(other.values.begin(), other.values.end()); - type &= other.type; - is_array |= other.is_array; - } - - void printType() const - { - if (type & Type::i) - std::cout << "I"; - if (type & Type::f) - std::cout << "F"; - if (type & Type::s) - std::cout << "S"; - if (type & Type::d) - std::cout << "D"; - if (type & Type::dt) - std::cout << "DT"; - if (is_array) - std::cout << "ARR"; - std::cout << "\n"; - } - - void print() - { - std::cout << name.first << "." << name.second << "\n"; - std::cout << "type: "; - printType(); - std::cout << "values:"; - for (const auto & val : values) - std::cout << " " << val; - std::cout << "\n"; - std::cout << "equal:"; - for (const auto & col : equals) - std::cout << " " << col.first << "." << col.second; - std::cout << "\n"; - } - - std::string generateOneValue() const - { - if (type & Type::i) - return randomInteger(); - - if (type & Type::f) - return randomFloat(); - - if (type & Type::d) - return randomDate(); - - if (type & Type::dt) - return randomDatetime(); - - if (type & Type::s) - return "'" + randomString(rng() % 40) + "'"; - - if (type & Type::b) - return "0"; - - return ""; - } - - bool generateValues(int amount = 0) - { - if (values.size() > 2 && amount == 0) - return false; - while (values.empty() or amount > 0) - { - amount -= 1; - if (is_array) - { - std::string v = "["; - for (unsigned int i = 0; i < static_cast(rng()) % 10 + 1; ++i) - { - if (i != 0) - v += ", "; - v += generateOneValue(); - } - v += "]"; - values.insert(v); - } - else - { - values.insert(generateOneValue()); - } - } - return true; - } - - void unifyType() - { - if (type & Type::i) - type = Type::i; - else if (type & Type::f) - type = Type::f; - else if (type & Type::d) - type = Type::d; - else if (type & Type::dt) - type = Type::dt; - else if (type & Type::s) - type = Type::s; - else if (type & Type::b) - type = Type::b; - else - throw std::runtime_error("Error in determination column type " + name.first + '.' + name.second); - } -}; - - -std::set> -decartMul( - std::set> & prev, - std::set & mul) -{ - std::set> result; - for (const auto & v : prev) - { - for (const auto & m : mul) - { - std::vector tmp = v; - tmp.push_back(m); - result.insert(tmp); - } - } - return result; -} - - -class Table -{ -public: - Table() = default; - - explicit Table(std::string table_name) - : name(table_name) {} - - std::string name; - std::set columns; - std::map column_description; - - bool columnExists(const std::string & column_name) const - { - return columns.contains(column_name); // || columns_maybe.contains(column_name); - } - - void addColumn(const std::string & column_name) - { - columns.insert(column_name); - } - - void setDescription(Column other) - { - column_description[other.name.second].merge(other); - } - - void print() - { - std::cout << "Table\n"; - std::cout << name << "\n"; - std::cout << "Columns:\n\n"; - for (const auto & column : columns) - { - std::cout << column << "\n"; - if (column_description.contains(column)) - column_description[column].print(); - std::cout << "\n"; - } - std::cout << "\n"; - } - - void merge(Table other) - { - name = other.name; - columns.insert(other.columns.begin(), other.columns.end()); - for (const auto & desc : other.column_description) - column_description[desc.first].merge(desc.second); - } - - std::string createQuery() - { - std::string create; - std::string db, _; - std::tie(db, _) = get_table_a_column(name); - create = "CREATE DATABASE IF NOT EXISTS " + db + ";\n\n"; - create += "CREATE TABLE IF NOT EXISTS " + name + " (\n"; - for (auto column = columns.begin(); column != columns.end(); ++column) - { - if (column != columns.begin()) - create += ", \n"; - create += *column + " "; - create += column_description[*column].is_array ? "Array(" : ""; - create += type_definition[column_description[*column].type]; - create += column_description[*column].is_array ? ")" : ""; - } - create += "\n) ENGINE = Log;\n\n"; - return create; - } - - std::string insertQuery() - { - std::string insert = "INSERT INTO " + name + "\n"; - insert += "("; - std::set> values = {std::vector(0)}; - for (auto column = columns.begin(); column != columns.end(); ++column) - { - if (column != columns.begin()) - insert += ", "; - insert += *column; - values = decartMul(values, column_description[*column].values); - } - insert += ") VALUES \n"; - for (auto val_set_iter = values.begin(); val_set_iter != values.end(); - ++val_set_iter) - { - if (val_set_iter != values.begin()) - insert += ",\n"; - auto val_set = *val_set_iter; - insert += "("; - for (auto val = val_set.begin(); val != val_set.end(); ++val) - { - if (val != val_set.begin()) - insert += ", "; - insert += *val; - } - insert += ")"; - } - insert += ";\n\n"; - return insert; - } -}; - - -class TableList -{ -public: - std::string main_table; - std::map aliases; - std::unordered_map tables; - std::set nested; - - bool tableExists(const std::string & table_name) const - { - return tables.contains(table_name); - } - - void addColumn(std::string full_column) - { - std::string table, column; - std::tie(table, column) = get_table_a_column(full_column); - if (!table.empty()) - { - if (tables.contains(table)) - { - tables[table].addColumn(column); - return; - } - if (aliases.contains(table)) - { - tables[aliases[table]].addColumn(column); - return; - } - nested.insert(table); - } - tables[main_table].addColumn(full_column); - } - - void addTable(std::string table_name) - { - if (tables.contains(table_name)) - return; - - tables[table_name] = Table(table_name); - if (main_table.empty()) - main_table = table_name; - } - - void addDescription(const Column & description) - { - std::string table = description.name.first; - if (tables.contains(table)) - tables[table].setDescription(description); - } - - TableAndColumn getTable(std::string full_column) const - { - std::string table, column; - std::tie(table, column) = get_table_a_column(full_column); - if (!table.empty()) - { - if (tables.contains(table)) - return std::make_pair(table, column); - - if (aliases.contains(table)) - { - table = aliases.find(table)->second; - return std::make_pair(table, column); - } - } - return std::make_pair(main_table, full_column); - } - - void print() - { - for (auto & table : tables) - { - table.second.print(); - std::cout << "\n"; - } - } - - void merge(TableList other) - { - for (const auto & table : other.tables) - tables[table.first].merge(table.second); - nested.insert(other.nested.begin(), other.nested.end()); - if (main_table.empty()) - main_table = other.main_table; - } -}; - -std::string getAlias(DB::ASTPtr ch) -{ - auto x = std::dynamic_pointer_cast(ch); - if (x) - return x->alias; - - for (const auto & child : (*ch).children) - { - auto alias = getAlias(child); - if (!alias.empty()) - return alias; - } - return ""; -} - -using FuncHandler = std::function &)>; -std::map handlers = {}; - -FuncRet arrayJoinFunc(DB::ASTPtr ch, std::map & columns) -{ - auto x = std::dynamic_pointer_cast(ch); - if (x) - { - std::set indents = {}; - for (auto & arg : x->arguments->children) - { - auto ident = std::dynamic_pointer_cast(arg); - if (ident) - indents.insert(ident->name()); - } - for (const auto & indent : indents) - { - auto c = Column(indent); - c.type = Type::all; - c.is_array = true; - if (columns.contains(indent)) - columns[indent].merge(c); - else - columns[indent] = c; - } - FuncRet r(Type::all, ""); - return r; - } - return FuncRet(); -} - -FuncRet inFunc(DB::ASTPtr ch, std::map & columns) -{ - auto x = std::dynamic_pointer_cast(ch); - if (x) - { - std::set indents{}; - std::set values{}; - ColumnType type_value = Type::all; - - for (auto & arg : x->arguments->children) - { - auto ident = std::dynamic_pointer_cast(arg); - if (ident) - { - indents.insert(ident->name()); - } - auto literal = std::dynamic_pointer_cast(arg); - if (literal) - { - ColumnType type = type_cast(literal->value.getType()); - - auto routine = [&](const auto & arr_values) - { - for (auto & val : arr_values) - { - type = type_cast(val.getType()); - if (type == Type::s || type == Type::d || type == Type::dt) - type = time_type(applyVisitor(DB::FieldVisitorToString(), val)); - type_value &= type; - values.insert(applyVisitor(DB::FieldVisitorToString(), val)); - } - }; - - if (type & Type::a) - { - auto arr_values = literal->value.get(); - routine(arr_values); - } - - if (type & Type::a) - { - auto arr_values = literal->value.get(); - routine(arr_values); - } - } - auto subfunc = std::dynamic_pointer_cast(arg); - if (subfunc) - { - FuncHandler f; - auto arg_func_name = std::dynamic_pointer_cast(arg)->name; - if (handlers.contains(arg_func_name)) - f = handlers[arg_func_name]; - else - f = handlers[""]; - FuncRet ret = f(arg, columns); - if (!ret.value.empty()) - { - values.insert(ret.value); - } - type_value &= ret.type; - } - } - for (const auto & indent : indents) - { - auto c = Column(indent); - c.type = type_value; - c.values.insert(values.begin(), values.end()); - c.generateValues(1); - if (columns.contains(indent)) - columns[indent].merge(c); - else - columns[indent] = c; - } - FuncRet r(Type::b | Type::i, ""); - return r; - } - return FuncRet(); -} - -FuncRet arrayFunc(DB::ASTPtr ch, std::map & columns) -{ - auto x = std::dynamic_pointer_cast(ch); - if (x) - { - std::set indents = {}; - std::string value = "["; - ColumnType type_value = Type::i | Type::f | Type::d | Type::dt | Type::s; - bool no_indent = true; - for (const auto & arg : x->arguments->children) - { - auto ident = std::dynamic_pointer_cast(arg); - if (ident) - { - no_indent = false; - indents.insert(ident->name()); - } - auto literal = std::dynamic_pointer_cast(arg); - if (literal) - { - ColumnType type = type_cast(literal->value.getType()); - if (type == Type::s || type == Type::d || type == Type::dt) - type = time_type(value); - type_value &= type; - - if (value != "[") - value += ", "; - value += applyVisitor(DB::FieldVisitorToString(), literal->value); - } - } - for (const auto & indent : indents) - { - auto c = Column(indent); - c.type = type_value; - if (columns.contains(indent)) - columns[indent].merge(c); - else - columns[indent] = c; - } - value += ']'; - FuncRet r(type_value, ""); - r.is_array = true; - if (no_indent) - r.value = value; - return r; - } - return FuncRet(); -} -FuncRet arithmeticFunc(DB::ASTPtr ch, std::map & columns) -{ - auto x = std::dynamic_pointer_cast(ch); - if (x) - { - std::set indents = {}; - std::set values = {}; - ColumnType type_value = Type::i | Type::f | Type::d | Type::dt; - ColumnType args_types = 0; - bool no_indent = true; - for (auto & arg : x->arguments->children) - { - ColumnType type = 0; - auto ident = std::dynamic_pointer_cast(arg); - if (ident) - { - no_indent = false; - indents.insert(ident->name()); - } - auto literal = std::dynamic_pointer_cast(arg); - if (literal) - type = type_cast(literal->value.getType()); - auto subfunc = std::dynamic_pointer_cast(arg); - if (subfunc) - { - FuncHandler f; - auto arg_func_name = std::dynamic_pointer_cast(arg)->name; - if (handlers.contains(arg_func_name)) - f = handlers[arg_func_name]; - else - f = handlers[""]; - FuncRet ret = f(arg, columns); - type = ret.type; - } - args_types |= type; - } - if (args_types & (Type::d | Type::dt)) - type_value -= Type::f; - if (args_types & Type::f) - type_value -= Type::d | Type::dt; - for (const auto & indent : indents) - { - auto c = Column(indent); - c.type = type_value; - if (columns.contains(indent)) - columns[indent].merge(c); - else - columns[indent] = c; - } - ColumnType ret_type = 0; - if (args_types & Type::dt) - ret_type = Type::dt; - else if (args_types & Type::d) - ret_type = Type::d | Type::dt; - else if (args_types & Type::f) - ret_type = Type::f; - else - ret_type = Type::d | Type::f | Type::dt | Type::i; - FuncRet r(ret_type, ""); - if (no_indent) - { - DB::WriteBufferFromOwnString buf; - formatAST(*ch, buf); - r.value = buf.str(); - } - return r; - } - return FuncRet(); -} -FuncRet likeFunc(DB::ASTPtr ch, std::map & columns) -{ - auto x = std::dynamic_pointer_cast(ch); - if (x) - { - std::set indents = {}; - std::set values = {}; - ColumnType type_value = Type::s; - for (auto & arg : x->arguments->children) - { - auto ident = std::dynamic_pointer_cast(arg); - if (ident) - indents.insert(ident->name()); - auto literal = std::dynamic_pointer_cast(arg); - if (literal) - { - std::string value = applyVisitor(DB::FieldVisitorToString(), literal->value); - std::string example{}; - for (size_t i = 0; i != value.size(); ++i) /// NOLINT - { - if (value[i] == '%') - example += randomString(rng() % 10); - else if (value[i] == '_') - example += randomString(1); - else - example += value[i]; - } - values.insert(example); - } - } - for (const auto & indent : indents) - { - auto c = Column(indent); - c.type = type_value; - c.values.insert(values.begin(), values.end()); - if (columns.contains(indent)) - columns[indent].merge(c); - else - columns[indent] = c; - } - FuncRet r(Type::b, ""); - return r; - } - return FuncRet(); -} - -FuncRet simpleFunc(DB::ASTPtr ch, std::map & columns) -{ - auto x = std::dynamic_pointer_cast(ch); - if (x) - { - std::set indents = {}; - std::set values = {}; - ColumnType type_value = Type::all; - bool is_array = false; - bool no_indent = true; - if (func_to_param_type.contains(boost::algorithm::to_lower_copy(x->name))) - { - type_value &= func_to_param_type[boost::algorithm::to_lower_copy(x->name)]; - is_array = func_to_param_type[boost::algorithm::to_lower_copy(x->name)] & Type::a; - } - for (const auto & arg : x->arguments->children) - { - ColumnType type = Type::all; - std::string value; - auto ident = std::dynamic_pointer_cast(arg); - if (ident) - { - no_indent = false; - indents.insert(ident->name()); - } - auto literal = std::dynamic_pointer_cast(arg); - if (literal) - { - value = applyVisitor(DB::FieldVisitorToString(), literal->value); - type = type_cast(literal->value.getType()); - is_array |= type & Type::a; - } - auto subfunc = std::dynamic_pointer_cast(arg); - if (subfunc) - { - FuncHandler f; - auto arg_func_name = std::dynamic_pointer_cast(arg)->name; - if (handlers.contains(arg_func_name)) - f = handlers[arg_func_name]; - else - f = handlers[""]; - FuncRet ret = f(arg, columns); - is_array |= ret.is_array; - type = ret.type; - value = ret.value; - if (value.empty()) - no_indent = false; - } - if (!value.empty()) - { - if (type == Type::i) - { - values.insert(value); - values.insert(value + " + " + randomInteger(1, 10)); - values.insert(value + " - " + randomInteger(1, 10)); - } - if (type == Type::f) - { - values.insert(value); - values.insert(value + " + " + randomFloat(1, 10)); - values.insert(value + " - " + randomFloat(1, 10)); - } - if (type & Type::s || type & Type::d || type & Type::dt) - { - if (type == Type::s) - type = time_type(value); - if (type == Type::s) - values.insert(value); - if (type & Type::d) - { - values.insert(value); - values.insert("toDate(" + value + ") + " + randomInteger(1, 10)); - values.insert("toDate(" + value + ") - " + randomInteger(1, 10)); - } - else if (type & Type::dt) - { - values.insert(value); - values.insert( - "toDateTime(" + value + ") + " + randomInteger(1, 10000)); - values.insert( - "toDateTime(" + value + ") - " + randomInteger(1, 10000)); - } - } - } - if (func_args_same_types.contains(boost::algorithm::to_lower_copy(x->name))) - type_value &= type; - } - for (const auto & indent : indents) - { - auto c = Column(indent); - c.type = type_value; - c.is_array = is_array; - if (func_args_same_types.contains( - boost::algorithm::to_lower_copy(x->name))) - c.values = values; - for (const auto & ind : indents) - if (ind != indent) - c.equals.insert(std::make_pair("", ind)); - - if (columns.contains(indent)) - columns[indent].merge(c); - else - columns[indent] = c; - } - if (func_to_return_type.contains(boost::algorithm::to_lower_copy(x->name))) - { - if (no_indent) - { - DB::WriteBufferFromOwnString buf; - formatAST(*ch, buf); - auto r = func_to_return_type[boost::algorithm::to_lower_copy(x->name)]; - r.value = buf.str(); - return r; - } - return func_to_return_type[boost::algorithm::to_lower_copy(x->name)]; - } - else if (func_to_param_type.contains( - boost::algorithm::to_lower_copy(x->name))) - { - if (no_indent) - { - DB::WriteBufferFromOwnString buf; - formatAST(*ch, buf); - return FuncRet( - func_to_param_type[boost::algorithm::to_lower_copy(x->name)], - buf.str()); - } - return FuncRet( - func_to_param_type[boost::algorithm::to_lower_copy(x->name)], - ""); - } - } - return FuncRet(); -} - -void processFunc(DB::ASTPtr ch, std::map & columns) -{ - auto x = std::dynamic_pointer_cast(ch); - if (x) - { - FuncHandler f; - auto arg_func_name = x->name; - if (handlers.contains(arg_func_name)) - f = handlers[arg_func_name]; - else - f = handlers[""]; - f(ch, columns); - } - else - { - for (const auto & child : (*ch).children) - processFunc(child, columns); - } -} - - -std::set getIndent(DB::ASTPtr ch) -{ - if (!ch) - return {}; - - std::set ret = {}; - auto x = std::dynamic_pointer_cast(ch); - if (x) - ret.insert(x->name()); - for (const auto & child : (*ch).children) - { - auto child_ind = getIndent(child); - ret.insert(child_ind.begin(), child_ind.end()); - } - return ret; -} - - -std::set getSelectIndent( - DB::ASTPtr asp, - std::set & column_alias) -{ - std::set ret = {}; - for (auto & ch : asp->children) - { - auto alias = getAlias(ch); - auto columns = getIndent(ch); - if (alias.empty()) - column_alias.insert(alias); - ret.insert(columns.begin(), columns.end()); - } - return ret; -} - - -std::set -connectedEqualityFind( - const Column & now, - std::map & columns_descriptions, - std::set & visited) -{ - std::set result; - for (const auto & column : now.equals) - if (!visited.contains(column)) - { - visited.insert(column); - auto sub_r = connectedEqualityFind( - columns_descriptions[column.first + "." + column.second], - columns_descriptions, - visited); - result.insert(sub_r.begin(), sub_r.end()); - } - result.insert(now.name); - return result; -} - - -std::map -unificateColumns( - std::map columns_descriptions, - const TableList & all_tables) -{ - for (auto & column : columns_descriptions) - { - std::set changed_equals; - for (const auto & eq : column.second.equals) - { - std::string t, c; - std::tie(t, c) = all_tables.getTable(eq.second); - changed_equals.insert(std::make_pair(t, c)); - } - column.second.equals = changed_equals; - } - std::map result; - for (auto & column : columns_descriptions) - { - std::string t, c; - std::tie(t, c) = all_tables.getTable(column.first); - column.second.name = std::make_pair(t, c); - result[t + "." + c].merge(column.second); - } - std::set visited; - for (auto & column : result) - if (!visited.contains(column.second.name)) - { - auto equal = connectedEqualityFind( - result[column.second.name.first + "." + column.second.name.second], - result, - visited); - for (const auto & c : equal) - result[c.first + "." + c.second].equals = equal; - } - for (auto & column : result) - for (const auto & e : column.second.equals) - column.second.merge(result[e.first + "." + e.second]); - - for (auto & column : result) - { - column.second.unifyType(); - if (column.second.generateValues()) - for (const auto & e : column.second.equals) - result[e.first + "." + e.second].merge(column.second); - - } - return result; -} - -std::vector getSelect(DB::ASTPtr vertex) -{ - auto z = std::dynamic_pointer_cast(vertex); - std::vector result; - if (z) - { - result.push_back(vertex); - return result; - } - - for (const auto & child : (*vertex).children) - { - auto v = getSelect(child); - result.insert(result.end(), v.begin(), v.end()); - } - return result; -} - - -void parseSelectQuery(DB::ASTPtr ast, TableList & all_tables) -{ - if (!ast) - throw std::runtime_error("Bad ASTPtr in parseSelectQuery" + StackTrace().toString()); - - auto select_ast = std::dynamic_pointer_cast(ast); - if (!select_ast) - { - std::cerr << "not select query"; - return; - } - std::set columns = {}; - - auto x = select_ast->tables(); - if (!x) - throw std::runtime_error("There is no tables in query. Nothing to generate."); - - for (auto & child : x->children) - { - auto ch = std::dynamic_pointer_cast(child); - auto table_expression_ast = std::dynamic_pointer_cast(ch->table_expression); - if (table_expression_ast && table_expression_ast->database_and_table_name) - { - auto table_name = *(getIndent(table_expression_ast->database_and_table_name).begin()); - all_tables.addTable(table_name); - auto alias = getAlias(ch); - if (!alias.empty()) - all_tables.aliases[alias] = table_name; - } - if (table_expression_ast && table_expression_ast->subquery) - { - for (const auto & select : getSelect(table_expression_ast->subquery)) - { - TableList local; - parseSelectQuery(select, local); - all_tables.merge(local); - } - } - - if (ch->table_join) - { - auto jch = std::dynamic_pointer_cast(ch->table_join); - if (jch->using_expression_list) - { - auto join_columns = getIndent(jch->using_expression_list); - columns.insert(join_columns.begin(), join_columns.end()); - } - else if (jch->on_expression) - { - auto join_columns = getIndent(jch->on_expression); - columns.insert(join_columns.begin(), join_columns.end()); - } - } - } - - std::set column_aliases; - auto select_columns = getSelectIndent(select_ast->select(), column_aliases); - columns.insert(select_columns.begin(), select_columns.end()); - - auto where_columns = getIndent(select_ast->where()); - columns.insert(where_columns.begin(), where_columns.end()); - - auto groupby_columns = getIndent(select_ast->groupBy()); - columns.insert(groupby_columns.begin(), groupby_columns.end()); - - auto orderby_columns = getIndent(select_ast->orderBy()); - columns.insert(orderby_columns.begin(), orderby_columns.end()); - - auto having_columns = getIndent(select_ast->having()); - columns.insert(having_columns.begin(), having_columns.end()); - - std::map columns_descriptions; - processFunc(ast, columns_descriptions); - - for (const auto & column : columns) - if (!column_aliases.contains(column)) - { - if (!columns_descriptions.contains(column)) - columns_descriptions[column] = Column(column); - all_tables.addColumn(column); - } - - columns_descriptions = unificateColumns(columns_descriptions, all_tables); - for (auto & column : columns_descriptions) - all_tables.addDescription(column.second); -} - - -TableList getTablesFromSelect(std::vector queries) -{ - TableList result; - for (std::string & query : queries) - { - DB::ParserQueryWithOutput parser(query.data() + query.size()); - DB::ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, 0); - for (auto & select : getSelect(ast)) - { - TableList local; - parseSelectQuery(select, local); - result.merge(local); - } - } - return result; -} - -int main(int argc, const char *argv[]) -{ - try - { - po::options_description desc("Allowed options"); - desc.add_options() - ("help,h", "Display greeting and allowed options.") - ("input,i", po::value(), "Input filename.") - ("output,o", po::value(), "Output filename."); - - po::variables_map vm; - po::store(po::parse_command_line(argc, argv, desc), vm); - po::notify(vm); - - if (vm.count("help") || vm.count("h")) - { - std::cout << "Hello! It is datasets generator for ClickHouse's queries." << std::endl; - std::cout << "Put some query as an input and it will produce queries for table creating and filling." << std::endl; - std::cout << "After that your query could be executed on this tables." << std::endl; - std::cout << desc << std::endl; - return 1; - } - if (vm.count("input")) - if (!freopen(vm["input"].as().c_str(), "r", stdin)) - std::cout << "Error while input." << std::endl; - if (vm.count("output")) - if (!freopen(vm["output"].as().c_str(), "w", stdout)) - std::cout << "Error while output." << std::endl; - if (vm.empty()) - std::cout << "Copy your queries (with semicolons) here, press Enter and Ctrl+D." << std::endl; - } - catch (...) - { - std::cerr << "Got error while parse command line arguments: " << DB::getCurrentExceptionMessage(true) << std::endl; - throw; - } - - handlers["plus"] = arithmeticFunc; - handlers["minus"] = arithmeticFunc; - handlers["like"] = likeFunc; - handlers["array"] = arrayFunc; - handlers["in"] = inFunc; - handlers[""] = simpleFunc; - - std::vector queries; - std::string in; - std::string query{}; - while (getline(std::cin, in)) - { - /// Skip comments - if (in.find("--") != std::string::npos) - continue; - - query += in + " "; - - if (in.find(';') != std::string::npos) - { - queries.push_back(query); - query = ""; - } - } - - try - { - auto result = getTablesFromSelect(queries); - - for (auto & table : result.tables) - { - std::cout << table.second.createQuery(); - std::cout << table.second.insertQuery(); - } - - for (auto & q: queries) - std::cout << q << std::endl; - } - catch (std::string & e) - { - std::cerr << "Exception: " << e << std::endl; - } -} diff --git a/utils/iotest/CMakeLists.txt b/utils/iotest/CMakeLists.txt deleted file mode 100644 index 356986eb493..00000000000 --- a/utils/iotest/CMakeLists.txt +++ /dev/null @@ -1,9 +0,0 @@ - -clickhouse_add_executable (iotest iotest.cpp ${SRCS}) -target_link_libraries (iotest PRIVATE clickhouse_common_io) - -clickhouse_add_executable (iotest_nonblock iotest_nonblock.cpp ${SRCS}) -target_link_libraries (iotest_nonblock PRIVATE clickhouse_common_io) - -clickhouse_add_executable (iotest_aio iotest_aio.cpp ${SRCS}) -target_link_libraries (iotest_aio PRIVATE clickhouse_common_io) diff --git a/utils/iotest/iotest.cpp b/utils/iotest/iotest.cpp deleted file mode 100644 index 7a1f35ddd52..00000000000 --- a/utils/iotest/iotest.cpp +++ /dev/null @@ -1,197 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -#include -#include -#include - - -namespace DB -{ - namespace ErrorCodes - { - extern const int CANNOT_OPEN_FILE; - extern const int CANNOT_CLOSE_FILE; - extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR; - extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR; - } -} - - -enum Mode -{ - MODE_NONE = 0, - MODE_READ = 1, - MODE_WRITE = 2, - MODE_ALIGNED = 4, - MODE_DIRECT = 8, - MODE_SYNC = 16, -}; - - -void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block_size, size_t count) -{ - using namespace DB; - - Memory<> direct_buf(block_size, ::getPageSize()); - std::vector simple_buf(block_size); - - char * buf; - if ((mode & MODE_DIRECT)) - buf = direct_buf.data(); - else - buf = simple_buf.data(); - - pcg64 rng(randomSeed()); - - for (size_t i = 0; i < count; ++i) - { - uint64_t rand_result1 = rng(); - uint64_t rand_result2 = rng(); - uint64_t rand_result3 = rng(); - - size_t rand_result = rand_result1 ^ (rand_result2 << 22) ^ (rand_result3 << 43); - size_t offset; - if ((mode & MODE_DIRECT) || (mode & MODE_ALIGNED)) - offset = min_offset + rand_result % ((max_offset - min_offset) / block_size) * block_size; - else - offset = min_offset + rand_result % (max_offset - min_offset - block_size + 1); - - if (mode & MODE_READ) - { - if (static_cast(block_size) != pread(fd, buf, block_size, offset)) - throwFromErrno("Cannot read", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR); - } - else - { - if (static_cast(block_size) != pwrite(fd, buf, block_size, offset)) - throwFromErrno("Cannot write", ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR); - } - } -} - - -int mainImpl(int argc, char ** argv) -{ - using namespace DB; - - const char * file_name = nullptr; - int mode = MODE_NONE; - UInt64 min_offset = 0; - UInt64 max_offset = 0; - UInt64 block_size = 0; - UInt64 threads = 0; - UInt64 count = 0; - - if (argc != 8) - { - std::cerr << "Usage: " << argv[0] << " file_name (r|w)[a][d][s] min_offset max_offset block_size threads count" << std::endl << - "a - aligned, d - direct, s - sync" << std::endl; - return 1; - } - - file_name = argv[1]; - min_offset = parse(argv[3]); - max_offset = parse(argv[4]); - block_size = parse(argv[5]); - threads = parse(argv[6]); - count = parse(argv[7]); - - for (int i = 0; argv[2][i]; ++i) - { - char c = argv[2][i]; - switch (c) - { - case 'r': - mode |= MODE_READ; - break; - case 'w': - mode |= MODE_WRITE; - break; - case 'a': - mode |= MODE_ALIGNED; - break; - case 'd': - mode |= MODE_DIRECT; - break; - case 's': - mode |= MODE_SYNC; - break; - default: - throw Poco::Exception("Invalid mode"); - } - } - - ThreadPool pool(threads); - - #ifndef OS_DARWIN - int fd = open(file_name, ((mode & MODE_READ) ? O_RDONLY : O_WRONLY) | ((mode & MODE_DIRECT) ? O_DIRECT : 0) | ((mode & MODE_SYNC) ? O_SYNC : 0)); - #else - int fd = open(file_name, ((mode & MODE_READ) ? O_RDONLY : O_WRONLY) | ((mode & MODE_SYNC) ? O_SYNC : 0)); - #endif - if (-1 == fd) - throwFromErrno("Cannot open file", ErrorCodes::CANNOT_OPEN_FILE); - #ifdef OS_DARWIN - if (mode & MODE_DIRECT) - if (fcntl(fd, F_NOCACHE, 1) == -1) - throwFromErrno("Cannot open file", ErrorCodes::CANNOT_CLOSE_FILE); - #endif - Stopwatch watch; - - for (size_t i = 0; i < threads; ++i) - pool.scheduleOrThrowOnError([=]{ thread(fd, mode, min_offset, max_offset, block_size, count); }); - pool.wait(); - - #if defined(OS_DARWIN) - fsync(fd); - #else - fdatasync(fd); - #endif - - watch.stop(); - - if (0 != close(fd)) - throwFromErrno("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE); - - std::cout << std::fixed << std::setprecision(2) - << "Done " << count << " * " << threads << " ops"; - if (mode & MODE_ALIGNED) - std::cout << " (aligned)"; - if (mode & MODE_DIRECT) - std::cout << " (direct)"; - if (mode & MODE_SYNC) - std::cout << " (sync)"; - std::cout << " in " << watch.elapsedSeconds() << " sec." - << ", " << count * threads / watch.elapsedSeconds() << " ops/sec." - << ", " << count * threads * block_size / watch.elapsedSeconds() / 1000000 << " MB/sec." - << std::endl; - - return 0; -} - - -int main(int argc, char ** argv) -{ - try - { - return mainImpl(argc, argv); - } - catch (const Poco::Exception & e) - { - std::cerr << e.what() << ", " << e.message() << std::endl; - return 1; - } -} diff --git a/utils/iotest/iotest_aio.cpp b/utils/iotest/iotest_aio.cpp deleted file mode 100644 index c0cf002ce58..00000000000 --- a/utils/iotest/iotest_aio.cpp +++ /dev/null @@ -1,203 +0,0 @@ -#if !defined(OS_LINUX) -int main(int, char **) { return 0; } -#else - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - namespace ErrorCodes - { - extern const int CANNOT_OPEN_FILE; - extern const int CANNOT_CLOSE_FILE; - extern const int CANNOT_IO_SUBMIT; - extern const int CANNOT_IO_GETEVENTS; - } -} - - -enum Mode -{ - MODE_READ = 1, - MODE_WRITE = 2, -}; - - -void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block_size, size_t buffers_count, size_t count) -{ - using namespace DB; - - AIOContext ctx; - - std::vector> buffers(buffers_count); - for (size_t i = 0; i < buffers_count; ++i) - buffers[i] = Memory<>(block_size, ::getPageSize()); - - pcg64_fast rng(randomSeed()); - - size_t in_progress = 0; - size_t blocks_sent = 0; - std::vector buffer_used(buffers_count, false); - std::vector iocbs(buffers_count); - std::vector query_cbs; - std::vector events(buffers_count); - - while (blocks_sent < count || in_progress > 0) - { - /// Prepare queries. - query_cbs.clear(); - for (size_t i = 0; i < buffers_count; ++i) - { - if (blocks_sent >= count || in_progress >= buffers_count) - break; - - if (buffer_used[i]) - continue; - - buffer_used[i] = true; - ++blocks_sent; - ++in_progress; - - char * buf = buffers[i].data(); - - uint64_t rand_result1 = rng(); - uint64_t rand_result2 = rng(); - uint64_t rand_result3 = rng(); - - size_t rand_result = rand_result1 ^ (rand_result2 << 22) ^ (rand_result3 << 43); - size_t offset = min_offset + rand_result % ((max_offset - min_offset) / block_size) * block_size; - - iocb & cb = iocbs[i]; - memset(&cb, 0, sizeof(cb)); - cb.aio_buf = reinterpret_cast(buf); - cb.aio_fildes = fd; - cb.aio_nbytes = block_size; - cb.aio_offset = offset; - cb.aio_data = static_cast(i); - - if (mode == MODE_READ) - { - cb.aio_lio_opcode = IOCB_CMD_PREAD; - } - else - { - cb.aio_lio_opcode = IOCB_CMD_PWRITE; - } - - query_cbs.push_back(&cb); - } - - /// Send queries. - if (io_submit(ctx.ctx, query_cbs.size(), query_cbs.data()) < 0) - throwFromErrno("io_submit failed", ErrorCodes::CANNOT_IO_SUBMIT); - - /// Receive answers. If we have something else to send, then receive at least one answer (after that send them), otherwise wait all answers. - memset(events.data(), 0, buffers_count * sizeof(events[0])); - int evs = io_getevents(ctx.ctx, (blocks_sent < count ? 1 : in_progress), buffers_count, events.data(), nullptr); - if (evs < 0) - throwFromErrno("io_getevents failed", ErrorCodes::CANNOT_IO_GETEVENTS); - - for (int i = 0; i < evs; ++i) - { - int b = static_cast(events[i].data); - if (events[i].res != static_cast(block_size)) - throw Poco::Exception("read/write error"); - --in_progress; - buffer_used[b] = false; - } - } -} - - -int mainImpl(int argc, char ** argv) -{ - using namespace DB; - - const char * file_name = nullptr; - int mode = MODE_READ; - UInt64 min_offset = 0; - UInt64 max_offset = 0; - UInt64 block_size = 0; - UInt64 buffers_count = 0; - UInt64 threads_count = 0; - UInt64 count = 0; - - if (argc != 9) - { - std::cerr << "Usage: " << argv[0] << " file_name r|w min_offset max_offset block_size threads buffers count" << std::endl; - return 1; - } - - file_name = argv[1]; - if (argv[2][0] == 'w') - mode = MODE_WRITE; - min_offset = parse(argv[3]); - max_offset = parse(argv[4]); - block_size = parse(argv[5]); - threads_count = parse(argv[6]); - buffers_count = parse(argv[7]); - count = parse(argv[8]); - - int fd = open(file_name, ((mode == MODE_READ) ? O_RDONLY : O_WRONLY) | O_DIRECT); - if (-1 == fd) - throwFromErrno("Cannot open file", ErrorCodes::CANNOT_OPEN_FILE); - - ThreadPool pool(threads_count); - - Stopwatch watch; - - for (size_t i = 0; i < threads_count; ++i) - pool.scheduleOrThrowOnError([=]{ thread(fd, mode, min_offset, max_offset, block_size, buffers_count, count); }); - pool.wait(); - - watch.stop(); - - if (0 != close(fd)) - throwFromErrno("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE); - - std::cout << std::fixed << std::setprecision(2) - << "Done " << count << " * " << threads_count << " ops"; - std::cout << " in " << watch.elapsedSeconds() << " sec." - << ", " << count * threads_count / watch.elapsedSeconds() << " ops/sec." - << ", " << count * threads_count * block_size / watch.elapsedSeconds() / 1000000 << " MB/sec." - << std::endl; - - return 0; -} - - -int main(int argc, char ** argv) -{ - try - { - return mainImpl(argc, argv); - } - catch (const Poco::Exception & e) - { - std::cerr << e.what() << ", " << e.message() << std::endl; - return 1; - } -} -#endif diff --git a/utils/iotest/iotest_nonblock.cpp b/utils/iotest/iotest_nonblock.cpp deleted file mode 100644 index 32c86282743..00000000000 --- a/utils/iotest/iotest_nonblock.cpp +++ /dev/null @@ -1,177 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -#if defined (OS_LINUX) -# include -#endif - - -namespace DB -{ - namespace ErrorCodes - { - extern const int CANNOT_OPEN_FILE; - extern const int CANNOT_CLOSE_FILE; - extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR; - extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR; - extern const int CANNOT_FSYNC; - extern const int SYSTEM_ERROR; - } -} - - -enum Mode -{ - MODE_READ, - MODE_WRITE, -}; - - -int mainImpl(int argc, char ** argv) -{ - using namespace DB; - - const char * file_name = nullptr; - Mode mode = MODE_READ; - UInt64 min_offset = 0; - UInt64 max_offset = 0; - UInt64 block_size = 0; - UInt64 descriptors = 0; - UInt64 count = 0; - - if (argc != 8) - { - std::cerr << "Usage: " << argv[0] << " file_name r|w min_offset max_offset block_size descriptors count" << std::endl; - return 1; - } - - file_name = argv[1]; - min_offset = parse(argv[3]); - max_offset = parse(argv[4]); - block_size = parse(argv[5]); - descriptors = parse(argv[6]); - count = parse(argv[7]); - - if (!strcmp(argv[2], "r")) - mode = MODE_READ; - else if (!strcmp(argv[2], "w")) - mode = MODE_WRITE; - else - throw Poco::Exception("Invalid mode"); - - std::vector fds(descriptors); - for (size_t i = 0; i < descriptors; ++i) - { - fds[i] = open(file_name, O_SYNC | ((mode == MODE_READ) ? O_RDONLY : O_WRONLY)); - if (-1 == fds[i]) - throwFromErrno("Cannot open file", ErrorCodes::CANNOT_OPEN_FILE); - } - - std::vector buf(block_size); - - pcg64 rng(randomSeed()); - - Stopwatch watch; - - std::vector polls(descriptors); - - for (size_t i = 0; i < descriptors; ++i) - { - polls[i].fd = fds[i]; - polls[i].events = (mode == MODE_READ) ? POLLIN : POLLOUT; - polls[i].revents = 0; - } - - size_t ops = 0; - while (ops < count) - { - if (poll(polls.data(), static_cast(descriptors), -1) <= 0) - throwFromErrno("poll failed", ErrorCodes::SYSTEM_ERROR); - for (size_t i = 0; i < descriptors; ++i) - { - if (!polls[i].revents) - continue; - - if (polls[i].revents != polls[i].events) - throw Poco::Exception("revents indicates error"); - polls[i].revents = 0; - ++ops; - - uint64_t rand_result1 = rng(); - uint64_t rand_result2 = rng(); - uint64_t rand_result3 = rng(); - - size_t rand_result = rand_result1 ^ (rand_result2 << 22) ^ (rand_result3 << 43); - size_t offset; - offset = min_offset + rand_result % ((max_offset - min_offset) / block_size) * block_size; - - if (mode == MODE_READ) - { - if (static_cast(block_size) != pread(fds[i], buf.data(), block_size, offset)) - throwFromErrno("Cannot read", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR); - } - else - { - if (static_cast(block_size) != pwrite(fds[i], buf.data(), block_size, offset)) - throwFromErrno("Cannot write", ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR); - } - } - } - - for (size_t i = 0; i < descriptors; ++i) - { -#if defined(OS_DARWIN) - if (fsync(fds[i])) - throwFromErrno("Cannot fsync", ErrorCodes::CANNOT_FSYNC); -#else - if (fdatasync(fds[i])) - throwFromErrno("Cannot fdatasync", ErrorCodes::CANNOT_FSYNC); -#endif - } - - watch.stop(); - - for (size_t i = 0; i < descriptors; ++i) - { - if (0 != close(fds[i])) - throwFromErrno("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE); - } - - std::cout << std::fixed << std::setprecision(2) - << "Done " << count << " ops" << " in " << watch.elapsedSeconds() << " sec." - << ", " << count / watch.elapsedSeconds() << " ops/sec." - << ", " << count * block_size / watch.elapsedSeconds() / 1000000 << " MB/sec." - << std::endl; - - return 0; -} - - -int main(int argc, char ** argv) -{ - try - { - return mainImpl(argc, argv); - } - catch (const Poco::Exception & e) - { - std::cerr << e.what() << ", " << e.message() << std::endl; - return 1; - } -} diff --git a/utils/zookeeper-adjust-block-numbers-to-parts/CMakeLists.txt b/utils/zookeeper-adjust-block-numbers-to-parts/CMakeLists.txt deleted file mode 100644 index b63373bacf7..00000000000 --- a/utils/zookeeper-adjust-block-numbers-to-parts/CMakeLists.txt +++ /dev/null @@ -1,3 +0,0 @@ -clickhouse_add_executable (zookeeper-adjust-block-numbers-to-parts main.cpp ${SRCS}) -target_compile_options(zookeeper-adjust-block-numbers-to-parts PRIVATE -Wno-format) -target_link_libraries (zookeeper-adjust-block-numbers-to-parts PRIVATE clickhouse_aggregate_functions dbms clickhouse_common_zookeeper boost::program_options) diff --git a/utils/zookeeper-adjust-block-numbers-to-parts/main.cpp b/utils/zookeeper-adjust-block-numbers-to-parts/main.cpp deleted file mode 100644 index 7736921a9c6..00000000000 --- a/utils/zookeeper-adjust-block-numbers-to-parts/main.cpp +++ /dev/null @@ -1,286 +0,0 @@ -#include -#include -#include -#include -#include -#include - -#include -#include - - -std::vector getAllShards(zkutil::ZooKeeper & zk, const std::string & root) -{ - return zk.getChildren(root); -} - - -std::vector removeNotExistingShards(zkutil::ZooKeeper & zk, const std::string & root, const std::vector & shards) -{ - auto existing_shards = getAllShards(zk, root); - std::vector filtered_shards; - filtered_shards.reserve(shards.size()); - for (const auto & shard : shards) - if (std::find(existing_shards.begin(), existing_shards.end(), shard) == existing_shards.end()) - std::cerr << "Shard " << shard << " not found." << std::endl; - else - filtered_shards.emplace_back(shard); - return filtered_shards; -} - - -std::vector getAllTables(zkutil::ZooKeeper & zk, const std::string & root, const std::string & shard) -{ - return zk.getChildren(root + "/" + shard); -} - - -std::vector removeNotExistingTables(zkutil::ZooKeeper & zk, const std::string & root, const std::string & shard, const std::vector & tables) -{ - auto existing_tables = getAllTables(zk, root, shard); - std::vector filtered_tables; - filtered_tables.reserve(tables.size()); - for (const auto & table : tables) - if (std::find(existing_tables.begin(), existing_tables.end(), table) == existing_tables.end()) - std::cerr << "\tTable " << table << " not found on shard " << shard << "." << std::endl; - else - filtered_tables.emplace_back(table); - return filtered_tables; -} - - -Int64 getMaxBlockNumberForPartition(zkutil::ZooKeeper & zk, - const std::string & replica_path, - const std::string & partition_name, - const DB::MergeTreeDataFormatVersion & format_version) -{ - auto replicas_path = replica_path + "/replicas"; - auto replica_hosts = zk.getChildren(replicas_path); - Int64 max_block_num = 0; - for (const auto & replica_host : replica_hosts) - { - auto parts = zk.getChildren(replicas_path + "/" + replica_host + "/parts"); - for (const auto & part : parts) - { - try - { - auto info = DB::MergeTreePartInfo::fromPartName(part, format_version); - if (info.partition_id == partition_name) - max_block_num = std::max(info.max_block, max_block_num); - } - catch (const DB::Exception & ex) - { - std::cerr << ex.displayText() << ", Part " << part << "skipped." << std::endl; - } - } - } - return max_block_num; -} - - -Int64 getCurrentBlockNumberForPartition(zkutil::ZooKeeper & zk, const std::string & part_path) -{ - Coordination::Stat stat; - zk.get(part_path, &stat); - - /// References: - /// https://stackoverflow.com/a/10347910 - /// https://bowenli86.github.io/2016/07/07/distributed%20system/zookeeper/How-does-ZooKeeper-s-persistent-sequential-id-work/ - return (stat.cversion + stat.numChildren) / 2; -} - - -std::unordered_map getPartitionsNeedAdjustingBlockNumbers( - zkutil::ZooKeeper & zk, const std::string & root, const std::vector & shards, const std::vector & tables) -{ - std::unordered_map result; - - std::vector use_shards = shards.empty() ? getAllShards(zk, root) : removeNotExistingShards(zk, root, shards); - - for (const auto & shard : use_shards) - { - std::cout << "Shard: " << shard << std::endl; - std::vector use_tables = tables.empty() ? getAllTables(zk, root, shard) : removeNotExistingTables(zk, root, shard, tables); - - for (const auto & table : use_tables) - { - std::cout << "\tTable: " << table << std::endl; - std::string table_path = root + "/" + shard + "/" + table; - std::string blocks_path = table_path + "/block_numbers"; - - std::vector partitions; - DB::MergeTreeDataFormatVersion format_version; - try - { - format_version = DB::ReplicatedMergeTreeTableMetadata::parse(zk.get(table_path + "/metadata")).data_format_version; - partitions = zk.getChildren(blocks_path); - } - catch (const DB::Exception & ex) - { - std::cerr << ex.displayText() << ", table " << table << " skipped." << std::endl; - continue; - } - - for (const auto & partition : partitions) - { - try - { - std::string part_path = blocks_path + "/" + partition; - Int64 partition_max_block = getMaxBlockNumberForPartition(zk, table_path, partition, format_version); - Int64 current_block_number = getCurrentBlockNumberForPartition(zk, part_path); - if (current_block_number < partition_max_block + 1) - { - std::cout << "\t\tPartition: " << partition << ": current block_number: " << current_block_number - << ", max block number: " << partition_max_block << ". Adjusting is required." << std::endl; - result.emplace(part_path, partition_max_block); - } - } - catch (const DB::Exception & ex) - { - std::cerr << ex.displayText() << ", partition " << partition << " skipped." << std::endl; - } - } - } - } - return result; -} - - -void setCurrentBlockNumber(zkutil::ZooKeeper & zk, const std::string & path, Int64 new_current_block_number) -{ - Int64 current_block_number = getCurrentBlockNumberForPartition(zk, path); - - auto create_ephemeral_nodes = [&](size_t count) - { - std::string block_prefix = path + "/block-"; - Coordination::Requests requests; - requests.reserve(count); - for (size_t i = 0; i != count; ++i) - requests.emplace_back(zkutil::makeCreateRequest(block_prefix, "", zkutil::CreateMode::EphemeralSequential)); - auto responses = zk.multi(requests); - - std::vector paths_created; - paths_created.reserve(responses.size()); - for (const auto & response : responses) - { - const auto * create_response = dynamic_cast(response.get()); - if (!create_response) - { - std::cerr << "\tCould not create ephemeral node " << block_prefix << std::endl; - return false; - } - paths_created.emplace_back(create_response->path_created); - } - - std::sort(paths_created.begin(), paths_created.end()); - for (const auto & path_created : paths_created) - { - Int64 number = DB::parse(path_created.c_str() + block_prefix.size(), path_created.size() - block_prefix.size()); - if (number != current_block_number) - { - char suffix[11] = ""; - size_t size = sprintf(suffix, "%010lld", current_block_number); - std::string expected_path = block_prefix + std::string(suffix, size); - std::cerr << "\t" << path_created << ": Ephemeral node has been created with an unexpected path (expected something like " - << expected_path << ")." << std::endl; - return false; - } - std::cout << "\t" << path_created << std::endl; - ++current_block_number; - } - - return true; - }; - - if (current_block_number >= new_current_block_number) - return; - - std::cout << "Creating ephemeral sequential nodes:" << std::endl; - create_ephemeral_nodes(1); /// Firstly try to create just a single node. - - /// Create other nodes in batches of 50 nodes. - while (current_block_number + 50 <= new_current_block_number) // NOLINT: clang-tidy thinks that the loop is infinite - create_ephemeral_nodes(50); - - create_ephemeral_nodes(new_current_block_number - current_block_number); -} - - -int main(int argc, char ** argv) -try -{ - /// Parse the command line. - namespace po = boost::program_options; - po::options_description desc("Allowed options"); - desc.add_options() - ("help,h", "show help") - ("zookeeper,z", po::value(), "Addresses of ZooKeeper instances, comma-separated. Example: example01e.clickhouse.com:2181") - ("path,p", po::value(), "[optional] Path of replica queue to insert node (without trailing slash). By default it's /clickhouse/tables") - ("shard,s", po::value(), "[optional] Shards to process, comma-separated. If not specified then the utility will process all the shards.") - ("table,t", po::value(), "[optional] Tables to process, comma-separated. If not specified then the utility will process all the tables.") - ("dry-run", "[optional] Specify if you want this utility just to analyze block numbers without any changes."); - - po::variables_map options; - po::store(po::parse_command_line(argc, argv, desc), options); - - auto show_usage = [&] - { - std::cout << "Usage: " << std::endl; - std::cout << " " << argv[0] << " [options]" << std::endl; - std::cout << desc << std::endl; - }; - - if (options.count("help") || (argc == 1)) - { - std::cout << "This utility adjusts the /block_numbers zookeeper nodes to the correct block number in partition." << std::endl; - std::cout << "It might be useful when incorrect block numbers stored in zookeeper don't allow you to insert data into a table or drop/detach a partition." << std::endl; - show_usage(); - return 0; - } - - if (!options.count("zookeeper")) - { - std::cerr << "Option --zookeeper should be set." << std::endl; - show_usage(); - return 1; - } - - std::string root = options.count("path") ? options.at("path").as() : "/clickhouse/tables"; - - std::vector shards, tables; - if (options.count("shard")) - boost::split(shards, options.at("shard").as(), boost::algorithm::is_any_of(",")); - if (options.count("table")) - boost::split(tables, options.at("table").as(), boost::algorithm::is_any_of(",")); - - /// Check if the adjusting of the block numbers is required. - std::cout << "Checking if adjusting of the block numbers is required:" << std::endl; - zkutil::ZooKeeper zookeeper(options.at("zookeeper").as()); - auto part_paths_with_max_block_numbers = getPartitionsNeedAdjustingBlockNumbers(zookeeper, root, shards, tables); - - if (part_paths_with_max_block_numbers.empty()) - { - std::cout << "No adjusting required." << std::endl; - return 0; - } - - std::cout << "Required adjusting of " << part_paths_with_max_block_numbers.size() << " block numbers." << std::endl; - - /// Adjust the block numbers. - if (options.count("dry-run")) - { - std::cout << "This is a dry-run, exiting." << std::endl; - return 0; - } - - std::cout << std::endl << "Adjusting the block numbers:" << std::endl; - for (const auto & [part_path, max_block_number] : part_paths_with_max_block_numbers) - setCurrentBlockNumber(zookeeper, part_path, max_block_number + 1); - - return 0; -} -catch (...) -{ - std::cerr << DB::getCurrentExceptionMessage(true) << '\n'; - throw; -} diff --git a/utils/zookeeper-create-entry-to-download-part/CMakeLists.txt b/utils/zookeeper-create-entry-to-download-part/CMakeLists.txt deleted file mode 100644 index 4c7a9ba9560..00000000000 --- a/utils/zookeeper-create-entry-to-download-part/CMakeLists.txt +++ /dev/null @@ -1,2 +0,0 @@ -clickhouse_add_executable (zookeeper-create-entry-to-download-part main.cpp ${SRCS}) -target_link_libraries (zookeeper-create-entry-to-download-part PRIVATE dbms clickhouse_common_zookeeper boost::program_options) diff --git a/utils/zookeeper-create-entry-to-download-part/main.cpp b/utils/zookeeper-create-entry-to-download-part/main.cpp deleted file mode 100644 index b92857929b7..00000000000 --- a/utils/zookeeper-create-entry-to-download-part/main.cpp +++ /dev/null @@ -1,47 +0,0 @@ -#include -#include -#include -#include - - -int main(int argc, char ** argv) -try -{ - boost::program_options::options_description desc("Allowed options"); - desc.add_options() - ("help,h", "produce help message") - ("address,a", boost::program_options::value()->required(), - "addresses of ZooKeeper instances, comma separated. Example: example01e.clickhouse.com:2181") - ("path,p", boost::program_options::value()->required(), "path of replica queue to insert node (without trailing slash)") - ("name,n", boost::program_options::value()->required(), "name of part to download") - ; - - boost::program_options::variables_map options; - boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); - - if (options.count("help")) - { - std::cout << "Insert log entry to replication queue to download part from any replica." << std::endl; - std::cout << "Usage: " << argv[0] << " [options]" << std::endl; - std::cout << desc << std::endl; - return 1; - } - - std::string path = options.at("path").as(); - std::string name = options.at("name").as(); - - zkutil::ZooKeeper zookeeper(options.at("address").as()); - - DB::ReplicatedMergeTreeLogEntry entry; - entry.type = DB::ReplicatedMergeTreeLogEntry::MERGE_PARTS; - entry.source_parts = {name}; - entry.new_part_name = name; - - zookeeper.create(path + "/queue-", entry.toString(), zkutil::CreateMode::PersistentSequential); - return 0; -} -catch (...) -{ - std::cerr << DB::getCurrentExceptionMessage(true) << '\n'; - throw; -} From 37048e986a303b8563cc9882d29384e3e9ec7123 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Fri, 4 Nov 2022 20:49:24 +0000 Subject: [PATCH 108/151] Add comments, fix review issues --- src/Storages/StorageDelta.cpp | 27 ++++++++++++++++++++------- src/Storages/StorageDelta.h | 15 ++++++++++++--- src/Storages/StorageHudi.cpp | 12 ++++++------ src/Storages/StorageHudi.h | 11 +++++++++++ 4 files changed, 49 insertions(+), 16 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 7ab522860e8..59e7c3d3187 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -60,20 +60,20 @@ std::vector DeltaLakeMetadata::ListCurrentFiles() && return keys; } -JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_) +JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, ContextPtr context) : base_configuration(configuration_), table_path(table_path_) { - Init(); + Init(context); } -void JsonMetadataGetter::Init() +void JsonMetadataGetter::Init(ContextPtr context) { auto keys = getJsonLogFiles(); // read data from every json log file for (const String & key : keys) { - auto buf = createS3ReadBuffer(key); + auto buf = createS3ReadBuffer(key, context); while (!buf->eof()) { @@ -110,6 +110,8 @@ std::vector JsonMetadataGetter::getJsonLogFiles() const auto bucket{base_configuration.uri.bucket}; request.SetBucket(bucket); + + // DeltaLake format stores all metadata json files in _delta_log directory request.SetPrefix(std::filesystem::path(table_path) / "_delta_log"); while (!is_finished) @@ -129,18 +131,25 @@ std::vector JsonMetadataGetter::getJsonLogFiles() { const auto & filename = obj.GetKey(); + // DeltaLake metadata files have json extension if (std::filesystem::path(filename).extension() == ".json") keys.push_back(filename); } + // Needed in case any more results are available + // if so, we will continue reading, and not read keys that were already read request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + + /// Set to false if all of the results were returned. Set to true if more keys + /// are available to return. If the number of results exceeds that specified by + /// MaxKeys, all of the results might not be returned is_finished = !outcome.GetResult().GetIsTruncated(); } return keys; } -std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key) +std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key, ContextPtr context) { // TBD: add parallel downloads return std::make_shared( @@ -149,7 +158,7 @@ std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String key, base_configuration.uri.version_id, /* max single read retries */ 10, - ReadSettings{}); + context->getReadSettings()); } void JsonMetadataGetter::handleJSON(const JSON & json) @@ -186,7 +195,7 @@ StorageDelta::StorageDelta( StorageInMemoryMetadata storage_metadata; StorageS3::updateS3Configuration(context_, base_configuration); - JsonMetadataGetter getter{base_configuration, table_path}; + JsonMetadataGetter getter{base_configuration, table_path, context_}; auto keys = getter.getFiles(); @@ -245,6 +254,9 @@ Pipe StorageDelta::read( String StorageDelta::generateQueryFromKeys(std::vector && keys) { + // DeltaLake store data parts in different files + // keys are filenames of parts + // for StorageS3 to read all parts we need format {key1,key2,key3,...keyn} std::string new_query = fmt::format("{{{}}}", fmt::join(keys, ",")); return new_query; } @@ -270,6 +282,7 @@ void registerStorageDelta(StorageFactory & factory) if (engine_args.size() == 4) configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); + // DeltaLake uses Parquet by default if (configuration.format == "auto") configuration.format = "Parquet"; diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index d0992dd4cb7..f759d0086a2 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -42,16 +42,16 @@ private: class JsonMetadataGetter { public: - JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_); + JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, ContextPtr context); std::vector getFiles() { return std::move(metadata).ListCurrentFiles(); } private: - void Init(); + void Init(ContextPtr context); std::vector getJsonLogFiles(); - std::shared_ptr createS3ReadBuffer(const String & key); + std::shared_ptr createS3ReadBuffer(const String & key, ContextPtr context); void handleJSON(const JSON & json); @@ -63,6 +63,9 @@ private: class StorageDelta : public IStorage { public: + // 1. Parses internal file structure of table + // 2. Finds out parts with latest version + // 3. Creates url for underlying StorageS3 enigne to handle reads StorageDelta( const StorageS3Configuration & configuration_, const StorageID & table_id_, @@ -74,6 +77,7 @@ public: String getName() const override { return "DeltaLake"; } + // Reads latest version of DeltaLake table Pipe read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, @@ -85,6 +89,11 @@ public: private: void Init(); + + // DeltaLake stores data in parts in different files + // keys is vector of parts with latest version + // generateQueryFromKeys constructs query from parts filenames for + // underlying StorageS3 engine static String generateQueryFromKeys(std::vector && keys); StorageS3::S3Configuration base_configuration; diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index e2a46fad2d0..46016771f49 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -142,8 +142,7 @@ std::string StorageHudi::generateQueryFromKeys(std::vector && keys, std::erase_if(keys, [&format](const std::string & s) { return std::filesystem::path(s).extension() != "." + format; }); // for each partition path take only latest file - - std::unordered_map> latest_parquets; + std::unordered_map> latest_parts; for (const auto & key : keys) { @@ -161,9 +160,9 @@ std::string StorageHudi::generateQueryFromKeys(std::vector && keys, // every filename contains metadata split by "_", timestamp is after last "_" uint64_t timestamp = std::stoul(key.substr(key.find_last_of("_") + 1)); - auto it = latest_parquets.find(path); + auto it = latest_parts.find(path); - if (it != latest_parquets.end()) + if (it != latest_parts.end()) { if (it->second.second < timestamp) { @@ -172,13 +171,13 @@ std::string StorageHudi::generateQueryFromKeys(std::vector && keys, } else { - latest_parquets[path] = {key, timestamp}; + latest_parts[path] = {key, timestamp}; } } std::vector filtered_keys; std::transform( - latest_parquets.begin(), latest_parquets.end(), std::back_inserter(filtered_keys), [](const auto & kv) { return kv.second.first; }); + latest_parts.begin(), latest_parts.end(), std::back_inserter(filtered_keys), [](const auto & kv) { return kv.second.first; }); std::string new_query; @@ -218,6 +217,7 @@ void registerStorageHudi(StorageFactory & factory) if (engine_args.size() == 4) configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); + // Apache Hudi uses Parquet by default if (configuration.format == "auto") configuration.format = "Parquet"; diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index 91a77ec83ff..c25c347f4c2 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -23,6 +23,9 @@ namespace DB class StorageHudi : public IStorage { public: + // 1. Parses internal file structure of table + // 2. Finds out parts with latest version + // 3. Creates url for underlying StorageS3 enigne to handle reads StorageHudi( const StorageS3Configuration & configuration_, const StorageID & table_id_, @@ -34,6 +37,8 @@ public: String getName() const override { return "Hudi"; } + + // Reads latest version of Apache Hudi table Pipe read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, @@ -45,6 +50,12 @@ public: private: std::vector getKeysFromS3(); + + // Apache Hudi store parts of data in different files + // Every part file has timestamp in it + // Every partition(directory) in Apache Hudi has different versions of part + // To find needed parts we need to find out latest part file for every partition + // Part format is usually parquet, but can differ static std::string generateQueryFromKeys(std::vector && keys, String format); StorageS3::S3Configuration base_configuration; From 7500cc38c0e47de381a323ab25c90b10f013f468 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Fri, 4 Nov 2022 20:51:44 +0000 Subject: [PATCH 109/151] Apply clang-format --- src/Storages/StorageDelta.cpp | 4 ++-- src/Storages/StorageDelta.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 59e7c3d3187..e14fa2401a4 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -111,7 +111,7 @@ std::vector JsonMetadataGetter::getJsonLogFiles() request.SetBucket(bucket); - // DeltaLake format stores all metadata json files in _delta_log directory + // DeltaLake format stores all metadata json files in _delta_log directory request.SetPrefix(std::filesystem::path(table_path) / "_delta_log"); while (!is_finished) @@ -139,7 +139,7 @@ std::vector JsonMetadataGetter::getJsonLogFiles() // Needed in case any more results are available // if so, we will continue reading, and not read keys that were already read request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); - + /// Set to false if all of the results were returned. Set to true if more keys /// are available to return. If the number of results exceeds that specified by /// MaxKeys, all of the results might not be returned diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index f759d0086a2..e3bb4c0b416 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -89,7 +89,7 @@ public: private: void Init(); - + // DeltaLake stores data in parts in different files // keys is vector of parts with latest version // generateQueryFromKeys constructs query from parts filenames for From 73dfe4acd7d527be1ed5698b2bf2db83bc81bc0a Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Sat, 5 Nov 2022 17:56:55 +0800 Subject: [PATCH 110/151] Add to default white list --- docs/en/operations/clickhouse-keeper.md | 4 ++-- src/Coordination/CoordinationSettings.cpp | 2 +- src/Coordination/FourLetterCommand.cpp | 2 +- src/Coordination/KeeperServer.cpp | 7 ++++--- 4 files changed, 8 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 17cf3ade6ab..cf2964b1a7c 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -126,7 +126,7 @@ clickhouse keeper --config /etc/your_path_to_config/config.xml ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively. -The 4lw commands has a white list configuration `four_letter_word_white_list` which has default value `conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro`. +The 4lw commands has a white list configuration `four_letter_word_white_list` which has default value `conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif`. You can issue the commands to ClickHouse Keeper via telnet or nc, at the client port. @@ -309,7 +309,7 @@ Sessions with Ephemerals (1): /clickhouse/task_queue/ddl ``` -- `csnp`: Schedule a snapshot creation task. Return the last committed log index of the scheduled snapshot if success or `Fail to scheduled snapshot creation task.` if failed. +- `csnp`: Schedule a snapshot creation task. Return the last committed log index of the scheduled snapshot if success or `Failed to schedule snapshot creation task.` if failed. ``` 100 diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 3e03ee0d6f4..08b8668a3ab 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -36,7 +36,7 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco } -const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv"; +const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif"; KeeperConfigurationAndSettings::KeeperConfigurationAndSettings() : server_id(NOT_EXIST) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 402270640d2..82123dc8218 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -481,7 +481,7 @@ String ApiVersionCommand::run() String CreateSnapshotCommand::run() { auto log_index = keeper_dispatcher.createSnapshot(); - return log_index > 0 ? std::to_string(log_index) : "Fail to scheduled snapshot creation task."; + return log_index > 0 ? std::to_string(log_index) : "Failed to schedule snapshot creation task."; } String LogInfoCommand::run() diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 5b2659e9a1b..487d0dc4cc3 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -913,15 +913,16 @@ uint64_t KeeperServer::createSnapshot() if (log_idx != 0) LOG_INFO(log, "Snapshot creation scheduled with last committed log index {}.", log_idx); else - LOG_WARNING(log, "Fail to scheduled snapshot creation task."); + LOG_WARNING(log, "Failed to schedule snapshot creation task."); return log_idx; } KeeperLogInfo KeeperServer::getKeeperLogInfo() { KeeperLogInfo log_info; - log_info.first_log_idx = state_manager->load_log_store()->start_index(); - log_info.first_log_term = state_manager->load_log_store()->term_at(log_info.first_log_idx); + auto log_store = state_manager->load_log_store(); + log_info.first_log_idx = log_store->start_index(); + log_info.first_log_term = log_store->term_at(log_info.first_log_idx); log_info.last_log_idx = raft_instance->get_last_log_idx(); log_info.last_log_term = raft_instance->get_last_log_term(); log_info.last_committed_log_idx = raft_instance->get_committed_log_idx(); From 00e93482c85df1ecaa1bbb66bb6ee4e2e6ea966e Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Sat, 5 Nov 2022 18:00:26 +0800 Subject: [PATCH 111/151] fix docs --- docs/en/operations/clickhouse-keeper.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index cf2964b1a7c..0324f742988 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -309,7 +309,7 @@ Sessions with Ephemerals (1): /clickhouse/task_queue/ddl ``` -- `csnp`: Schedule a snapshot creation task. Return the last committed log index of the scheduled snapshot if success or `Failed to schedule snapshot creation task.` if failed. +- `csnp`: Schedule a snapshot creation task. Return the last committed log index of the scheduled snapshot if success or `Failed to schedule snapshot creation task.` if failed. Note that `lgif` command can help you determine whether the snapshot is done. ``` 100 From b0b3942aae0452774b229ecfda1dc357ef85ddaf Mon Sep 17 00:00:00 2001 From: Rami Dridi Date: Sat, 5 Nov 2022 21:33:47 +0100 Subject: [PATCH 112/151] docs : updating comparing mode command and results --- .../utilities/clickhouse-benchmark.md | 58 +++++++------------ 1 file changed, 20 insertions(+), 38 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-benchmark.md b/docs/en/operations/utilities/clickhouse-benchmark.md index 1a250ea5481..faa7ac75c74 100644 --- a/docs/en/operations/utilities/clickhouse-benchmark.md +++ b/docs/en/operations/utilities/clickhouse-benchmark.md @@ -109,56 +109,38 @@ In the report you can find: `clickhouse-benchmark` can compare performances for two running ClickHouse servers. -To use the comparison mode, specify endpoints of both servers by two pairs of `--host`, `--port` keys. Keys matched together by position in arguments list, the first `--host` is matched with the first `--port` and so on. `clickhouse-benchmark` establishes connections to both servers, then sends queries. Each query addressed to a randomly selected server. The results are shown for each server separately. +To use the comparison mode, specify endpoints of both servers by two pairs of `--host`, `--port` keys. Keys matched together by position in arguments list, the first `--host` is matched with the first `--port` and so on. `clickhouse-benchmark` establishes connections to both servers, then sends queries. Each query addressed to a randomly selected server. The results are shown in a table. ## Example {#clickhouse-benchmark-example} ``` bash -$ echo "SELECT * FROM system.numbers LIMIT 10000000 OFFSET 10000000" | clickhouse-benchmark -i 10 +$ echo "SELECT * FROM system.numbers LIMIT 10000000 OFFSET 10000000" | clickhouse-benchmark --host=localhost --port=9001 --host=localhost --port=9000 -i 10 ``` ``` text Loaded 1 queries. -Queries executed: 6. +Queries executed: 5. -localhost:9000, queries 6, QPS: 6.153, RPS: 123398340.957, MiB/s: 941.455, result RPS: 61532982.200, result MiB/s: 469.459. +localhost:9001, queries 2, QPS: 3.764, RPS: 75446929.370, MiB/s: 575.614, result RPS: 37639659.982, result MiB/s: 287.168. +localhost:9000, queries 3, QPS: 3.815, RPS: 76466659.385, MiB/s: 583.394, result RPS: 38148392.297, result MiB/s: 291.049. -0.000% 0.159 sec. -10.000% 0.159 sec. -20.000% 0.159 sec. -30.000% 0.160 sec. -40.000% 0.160 sec. -50.000% 0.162 sec. -60.000% 0.164 sec. -70.000% 0.165 sec. -80.000% 0.166 sec. -90.000% 0.166 sec. -95.000% 0.167 sec. -99.000% 0.167 sec. -99.900% 0.167 sec. -99.990% 0.167 sec. +0.000% 0.258 sec. 0.250 sec. +10.000% 0.258 sec. 0.250 sec. +20.000% 0.258 sec. 0.250 sec. +30.000% 0.258 sec. 0.267 sec. +40.000% 0.258 sec. 0.267 sec. +50.000% 0.273 sec. 0.267 sec. +60.000% 0.273 sec. 0.267 sec. +70.000% 0.273 sec. 0.267 sec. +80.000% 0.273 sec. 0.269 sec. +90.000% 0.273 sec. 0.269 sec. +95.000% 0.273 sec. 0.269 sec. +99.000% 0.273 sec. 0.269 sec. +99.900% 0.273 sec. 0.269 sec. +99.990% 0.273 sec. 0.269 sec. - - -Queries executed: 10. - -localhost:9000, queries 10, QPS: 6.082, RPS: 121959604.568, MiB/s: 930.478, result RPS: 60815551.642, result MiB/s: 463.986. - -0.000% 0.159 sec. -10.000% 0.159 sec. -20.000% 0.160 sec. -30.000% 0.163 sec. -40.000% 0.164 sec. -50.000% 0.165 sec. -60.000% 0.166 sec. -70.000% 0.166 sec. -80.000% 0.167 sec. -90.000% 0.167 sec. -95.000% 0.170 sec. -99.000% 0.172 sec. -99.900% 0.172 sec. -99.990% 0.172 sec. +No difference proven at 99.5% confidence ``` [Original article](https://clickhouse.com/docs/en/operations/utilities/clickhouse-benchmark.md) From 69d6d4244334b454b02271ec6b3d7bdb356a023f Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Sun, 6 Nov 2022 15:26:19 +0800 Subject: [PATCH 113/151] NOT LIKE only work for 'prefix%' --- src/Storages/MergeTree/KeyCondition.cpp | 29 ++++++++++++------- src/Storages/MergeTree/KeyCondition.h | 2 +- .../StorageSystemMergeTreeMetadataCache.cpp | 2 +- ...58_key_condition_not_like_prefix.reference | 1 + .../02458_key_condition_not_like_prefix.sql | 9 +++++- 5 files changed, 29 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 3bea261fd52..a4ac9bf8c25 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -29,6 +29,7 @@ #include #include +#include #include #include #include @@ -55,11 +56,15 @@ String Range::toString() const } -/// Example: for `Hello\_World% ...` string it returns `Hello_World`, and for `%test%` returns an empty string. -/// If perfect_prefix_match == true, only consider pattern in the format `prefix%_` -String extractFixedPrefixFromLikePattern(const String & like_pattern, bool perfect_prefix_match) +/// Returns the prefix of like_pattern before the first wildcard, e.g. 'Hello\_World% ...' --> 'Hello\_World' +/// We call a prefix "perfect" if: +/// - (1) the prefix must have willcard +/// - (2) the first wildcard is '%' and is only followed by nothing or other '%' +/// e.g. 'test%' or 'test%% has perfect prefix 'test', 'test%x', 'test%_' or 'test_' has no perfect prefix. +String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool require_perfect_prefix) { String fixed_prefix; + fixed_prefix.reserve(like_pattern.size()); const char * pos = like_pattern.data(); const char * end = pos + like_pattern.size(); @@ -68,12 +73,13 @@ String extractFixedPrefixFromLikePattern(const String & like_pattern, bool perfe switch (*pos) { case '%': - [[fallthrough]]; case '_': - if (perfect_prefix_match && std::find_if(pos+1, end, [](const char c) { return c != '%' && c != '_'; }) != end) - return ""; + if (require_perfect_prefix) + { + bool is_prefect_prefix = std::all_of(pos, end, [](auto c) { return c == '%'; }); + return is_prefect_prefix ? fixed_prefix : ""; + } return fixed_prefix; - case '\\': ++pos; if (pos == end) @@ -81,12 +87,13 @@ String extractFixedPrefixFromLikePattern(const String & like_pattern, bool perfe [[fallthrough]]; default: fixed_prefix += *pos; - break; } ++pos; } - + /// If we can reach this code, it means there was no wildcard found in the pattern, so it is not a perfect prefix + if (require_perfect_prefix) + return ""; return fixed_prefix; } @@ -349,7 +356,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = extractFixedPrefixFromLikePattern(value.get()); + String prefix = extractFixedPrefixFromLikePattern(value.get(), /*required_perfect_prefix*/ false); if (prefix.empty()) return false; @@ -370,7 +377,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = extractFixedPrefixFromLikePattern(value.get(), true); + String prefix = extractFixedPrefixFromLikePattern(value.get(), /*required_perfect_prefix*/ true); if (prefix.empty()) return false; diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index e7eb855b1cc..81cfea3d359 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -485,6 +485,6 @@ private: bool strict; }; -String extractFixedPrefixFromLikePattern(const String & like_pattern, bool perfect_prefix_match = false); +String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool require_perfect_prefix); } diff --git a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp index 275365648f3..776b85cbffd 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp @@ -116,7 +116,7 @@ void StorageSystemMergeTreeMetadataCache::fillData(MutableColumns & res_columns, } else { - String target = extractFixedPrefixFromLikePattern(key); + String target = extractFixedPrefixFromLikePattern(key, /*required_perfect_prefix*/ false); if (target.empty()) throw Exception( "SELECT from system.merge_tree_metadata_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); diff --git a/tests/queries/0_stateless/02458_key_condition_not_like_prefix.reference b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.reference index 87766d889a3..02357561e02 100644 --- a/tests/queries/0_stateless/02458_key_condition_not_like_prefix.reference +++ b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.reference @@ -1 +1,2 @@ 200000 +200000 diff --git a/tests/queries/0_stateless/02458_key_condition_not_like_prefix.sql b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.sql index 2c1402df27e..e821b16ed5c 100644 --- a/tests/queries/0_stateless/02458_key_condition_not_like_prefix.sql +++ b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.sql @@ -2,4 +2,11 @@ CREATE TABLE data (str String) ENGINE=MergeTree ORDER BY str; INSERT INTO data (str) SELECT 'aa' FROM numbers(100000); INSERT INTO data (str) SELECT 'ba' FROM numbers(100000); INSERT INTO data (str) SELECT 'ca' FROM numbers(100000); -SELECT count()FROM data WHERE str NOT LIKE 'a%' SETTINGS force_primary_key=1; +SELECT count() FROM data WHERE str NOT LIKE 'a%' SETTINGS force_primary_key=1; +SELECT count() FROM data WHERE str NOT LIKE 'a%%' SETTINGS force_primary_key=1; +SELECT count() FROM data WHERE str NOT LIKE 'a' SETTINGS force_primary_key=1; -- { serverError 277 } +SELECT count() FROM data WHERE str NOT LIKE '%a' SETTINGS force_primary_key=1; -- { serverError 277 } +SELECT count() FROM data WHERE str NOT LIKE 'a_' SETTINGS force_primary_key=1; -- { serverError 277 } +SELECT count() FROM data WHERE str NOT LIKE 'a%_' SETTINGS force_primary_key=1; -- { serverError 277 } +SELECT count() FROM data WHERE str NOT LIKE '_a' SETTINGS force_primary_key=1; -- { serverError 277 } +SELECT count() FROM data WHERE str NOT LIKE 'a%\_' SETTINGS force_primary_key=1; -- { serverError 277 } From f7e65ebd72a749c79cdaaa03635811ab5539baa2 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 7 Nov 2022 16:25:36 +0800 Subject: [PATCH 114/151] make comments more clear --- src/Storages/MergeTree/KeyCondition.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index a4ac9bf8c25..a74a3e58dff 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -57,8 +57,8 @@ String Range::toString() const /// Returns the prefix of like_pattern before the first wildcard, e.g. 'Hello\_World% ...' --> 'Hello\_World' -/// We call a prefix "perfect" if: -/// - (1) the prefix must have willcard +/// We call a pattern "perfect prefix" if: +/// - (1) the pattern has a wildcard /// - (2) the first wildcard is '%' and is only followed by nothing or other '%' /// e.g. 'test%' or 'test%% has perfect prefix 'test', 'test%x', 'test%_' or 'test_' has no perfect prefix. String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool require_perfect_prefix) From e2c23344d14e2a90e1dda9899ee3a41368f50089 Mon Sep 17 00:00:00 2001 From: Camilo Sierra Date: Mon, 7 Nov 2022 09:54:45 +0100 Subject: [PATCH 115/151] full example using AggregatingMergeTree Discussing with a dev the `AggregatingMergeTree` usage was not clear, they do not understand how use the `AggregateFunction` or how we could get the final aggregated value. I hope this full example could better show the capabilities of this feature and help the new users to better understand it. --- .../mergetree-family/aggregatingmergetree.md | 41 ++++++++++++++----- 1 file changed, 31 insertions(+), 10 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md index ba518f51657..267e5c81dda 100644 --- a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md @@ -68,36 +68,57 @@ In the results of `SELECT` query, the values of `AggregateFunction` type have im ## Example of an Aggregated Materialized View {#example-of-an-aggregated-materialized-view} -`AggregatingMergeTree` materialized view that watches the `test.visits` table: +We will create the table `test.visits` that contain the raw data: ``` sql -CREATE MATERIALIZED VIEW test.basic -ENGINE = AggregatingMergeTree() PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate) +CREATE TABLE test.visits + ( + StartDate DateTime64 NOT NULL, + CounterID UInt64, + Sign Nullable(Int32), + UserID Nullable(Int32) +) ENGINE = MergeTree ORDER BY (StartDate, CounterID); +``` + +`AggregatingMergeTree` materialized view that watches the `test.visits` table, and use the `AggregateFunction` type: + +``` sql +CREATE MATERIALIZED VIEW test.mv_visits +( + StartDate DateTime64 NOT NULL, + CounterID UInt64, + Visits AggregateFunction(sum, Nullable(Int32)), + Users AggregateFunction(uniq, Nullable(Int32)) +) +ENGINE = AggregatingMergeTree() ORDER BY (StartDate, CounterID) AS SELECT - CounterID, StartDate, - sumState(Sign) AS Visits, + CounterID, + sumState(Sign) AS Visits, uniqState(UserID) AS Users FROM test.visits -GROUP BY CounterID, StartDate; +GROUP BY StartDate, CounterID; ``` Inserting data into the `test.visits` table. ``` sql -INSERT INTO test.visits ... +INSERT INTO test.visits (StartDate, CounterID, Sign, UserID) + VALUES (1667446031, 1, 3, 4) +INSERT INTO test.visits (StartDate, CounterID, Sign, UserID) + VALUES (1667446031, 1, 6, 3) ``` -The data are inserted in both the table and view `test.basic` that will perform the aggregation. +The data are inserted in both the table and the materialized view `test.mv_visits`. -To get the aggregated data, we need to execute a query such as `SELECT ... GROUP BY ...` from the view `test.basic`: +To get the aggregated data, we need to execute a query such as `SELECT ... GROUP BY ...` from the materialized view `test.mv_visits`: ``` sql SELECT StartDate, sumMerge(Visits) AS Visits, uniqMerge(Users) AS Users -FROM test.basic +FROM test.mv_visits GROUP BY StartDate ORDER BY StartDate; ``` From bd2bd7149db533dc19b9a81877ea4d357f3be62b Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 7 Nov 2022 18:22:50 +0800 Subject: [PATCH 116/151] fix typo --- src/Storages/MergeTree/KeyCondition.cpp | 10 +++++----- src/Storages/MergeTree/KeyCondition.h | 2 +- .../System/StorageSystemMergeTreeMetadataCache.cpp | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index a74a3e58dff..8ffd1059a58 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -61,7 +61,7 @@ String Range::toString() const /// - (1) the pattern has a wildcard /// - (2) the first wildcard is '%' and is only followed by nothing or other '%' /// e.g. 'test%' or 'test%% has perfect prefix 'test', 'test%x', 'test%_' or 'test_' has no perfect prefix. -String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool require_perfect_prefix) +String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool requires_perfect_prefix) { String fixed_prefix; fixed_prefix.reserve(like_pattern.size()); @@ -74,7 +74,7 @@ String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool req { case '%': case '_': - if (require_perfect_prefix) + if (requires_perfect_prefix) { bool is_prefect_prefix = std::all_of(pos, end, [](auto c) { return c == '%'; }); return is_prefect_prefix ? fixed_prefix : ""; @@ -92,7 +92,7 @@ String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool req ++pos; } /// If we can reach this code, it means there was no wildcard found in the pattern, so it is not a perfect prefix - if (require_perfect_prefix) + if (requires_perfect_prefix) return ""; return fixed_prefix; } @@ -356,7 +356,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = extractFixedPrefixFromLikePattern(value.get(), /*required_perfect_prefix*/ false); + String prefix = extractFixedPrefixFromLikePattern(value.get(), /*requires_perfect_prefix*/ false); if (prefix.empty()) return false; @@ -377,7 +377,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = extractFixedPrefixFromLikePattern(value.get(), /*required_perfect_prefix*/ true); + String prefix = extractFixedPrefixFromLikePattern(value.get(), /*requires_perfect_prefix*/ true); if (prefix.empty()) return false; diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 81cfea3d359..138dce83db9 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -485,6 +485,6 @@ private: bool strict; }; -String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool require_perfect_prefix); +String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool requires_perfect_prefix); } diff --git a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp index 776b85cbffd..731cdf94157 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp @@ -116,7 +116,7 @@ void StorageSystemMergeTreeMetadataCache::fillData(MutableColumns & res_columns, } else { - String target = extractFixedPrefixFromLikePattern(key, /*required_perfect_prefix*/ false); + String target = extractFixedPrefixFromLikePattern(key, /*requires_perfect_prefix*/ false); if (target.empty()) throw Exception( "SELECT from system.merge_tree_metadata_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); From 940e859eadf0d686952aa27472b7433fbd9c31aa Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 7 Nov 2022 11:40:18 +0100 Subject: [PATCH 117/151] Add debug information to nightly builds --- .github/workflows/nightly.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 612bb1f8f9b..7dff1e205a1 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -10,6 +10,9 @@ env: workflow_dispatch: jobs: + Debug: + # The task for having a preserved ENV and event.json for later investigation + uses: ./.github/workflows/debug.yml DockerHubPushAarch64: runs-on: [self-hosted, style-checker-aarch64] steps: From 897811371dc47314e5224a96cd40ecd485a21770 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 7 Nov 2022 12:45:26 +0100 Subject: [PATCH 118/151] Minor changes --- src/Storages/StorageDelta.cpp | 65 +++++++++---------- src/Storages/StorageHudi.cpp | 114 ++++++++++++++++------------------ src/Storages/StorageHudi.h | 20 +++--- 3 files changed, 94 insertions(+), 105 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index e14fa2401a4..e8287a2fd61 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -1,29 +1,29 @@ #include "config.h" - #if USE_AWS_S3 -# include -# include +#include +#include -# include -# include -# include -# include +#include +#include +#include +#include -# include -# include -# include +#include +#include +#include -# include +#include -# include -# include -# include +#include +#include +#include -# include +#include -# include -# include +#include +#include +#include namespace DB { @@ -53,9 +53,7 @@ std::vector DeltaLakeMetadata::ListCurrentFiles() && keys.reserve(file_update_time.size()); for (auto && [k, _] : file_update_time) - { keys.push_back(k); - } return keys; } @@ -75,12 +73,13 @@ void JsonMetadataGetter::Init(ContextPtr context) { auto buf = createS3ReadBuffer(key, context); + char c; while (!buf->eof()) { - // may be some invalid characters before json - char c; + /// May be some invalid characters before json. while (buf->peek(c) && c != '{') buf->ignore(); + if (buf->eof()) break; @@ -91,7 +90,6 @@ void JsonMetadataGetter::Init(ContextPtr context) continue; const JSON json(json_str); - handleJSON(json); } } @@ -111,8 +109,9 @@ std::vector JsonMetadataGetter::getJsonLogFiles() request.SetBucket(bucket); - // DeltaLake format stores all metadata json files in _delta_log directory - request.SetPrefix(std::filesystem::path(table_path) / "_delta_log"); + /// DeltaLake format stores all metadata json files in _delta_log directory + static constexpr auto deltalake_metadata_directory = "_delta_log"; + request.SetPrefix(std::filesystem::path(table_path) / deltalake_metadata_directory); while (!is_finished) { @@ -136,8 +135,8 @@ std::vector JsonMetadataGetter::getJsonLogFiles() keys.push_back(filename); } - // Needed in case any more results are available - // if so, we will continue reading, and not read keys that were already read + /// Needed in case any more results are available + /// if so, we will continue reading, and not read keys that were already read request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); /// Set to false if all of the results were returned. Set to true if more keys @@ -151,13 +150,13 @@ std::vector JsonMetadataGetter::getJsonLogFiles() std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key, ContextPtr context) { - // TBD: add parallel downloads + /// TODO: add parallel downloads return std::make_shared( base_configuration.client, base_configuration.uri.bucket, key, base_configuration.uri.version_id, - /* max single read retries */ 10, + /* max single read retries */10, context->getReadSettings()); } @@ -198,7 +197,6 @@ StorageDelta::StorageDelta( JsonMetadataGetter getter{base_configuration, table_path, context_}; auto keys = getter.getFiles(); - auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); LOG_DEBUG(log, "New uri: {}", new_uri); @@ -281,12 +279,11 @@ void registerStorageDelta(StorageFactory & factory) if (engine_args.size() == 4) configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); - - // DeltaLake uses Parquet by default - if (configuration.format == "auto") + else + { + /// DeltaLake uses Parquet by default. configuration.format = "Parquet"; - - //auto format_settings = getFormatSettings(args.getContext()); + } return std::make_shared( configuration, args.table_id, args.columns, args.constraints, args.comment, args.getContext(), std::nullopt); diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 46016771f49..d31d04beef5 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -2,18 +2,21 @@ #if USE_AWS_S3 -# include -# include +#include +#include -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include +#include +#include -# include +#include + +#include namespace DB { @@ -41,7 +44,6 @@ StorageHudi::StorageHudi( StorageS3::updateS3Configuration(context_, base_configuration); auto keys = getKeysFromS3(); - auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys), configuration_.format); LOG_DEBUG(log, "New uri: {}", new_uri); @@ -121,7 +123,7 @@ std::vector StorageHudi::getKeysFromS3() const auto & result_batch = outcome.GetResult().GetContents(); for (const auto & obj : result_batch) { - const auto & filename = obj.GetKey().substr(table_path.size()); // object name without tablepath prefix + const auto & filename = obj.GetKey().substr(table_path.size()); /// Object name without tablepath prefix. keys.push_back(filename); LOG_DEBUG(log, "Found file: {}", filename); } @@ -133,65 +135,55 @@ std::vector StorageHudi::getKeysFromS3() return keys; } -std::string StorageHudi::generateQueryFromKeys(std::vector && keys, String format) +String StorageHudi::generateQueryFromKeys(const std::vector & keys, const String & format) { - // make format lowercase - std::transform(format.begin(), format.end(), format.begin(), [](unsigned char c) { return std::tolower(c); }); - - // filter only files with specific format - std::erase_if(keys, [&format](const std::string & s) { return std::filesystem::path(s).extension() != "." + format; }); - - // for each partition path take only latest file - std::unordered_map> latest_parts; - - for (const auto & key : keys) + /// For each partition path take only latest file. + struct FileInfo { - auto slash = key.find_last_of("/"); - std::string path; - if (slash == std::string::npos) - { - path = ""; - } - else - { - path = key.substr(0, slash); - } + String filename; + UInt64 timestamp; + }; + std::unordered_map latest_parts; /// Partition path (directory) -> latest part file info. - // every filename contains metadata split by "_", timestamp is after last "_" - uint64_t timestamp = std::stoul(key.substr(key.find_last_of("_") + 1)); + /// Make format lowercase. + const auto expected_extension= "." + Poco::toLower(format); + /// Filter only files with specific format. + auto keys_filter = [&](const String & key) { return std::filesystem::path(key).extension() == expected_extension; }; - auto it = latest_parts.find(path); + for (const auto & key : keys | std::views::filter(keys_filter)) + { + const auto key_path = fs::path(key); + const String filename = key_path.filename(); + const String partition_path = key_path.parent_path(); - if (it != latest_parts.end()) + /// Every filename contains metadata split by "_", timestamp is after last "_". + const auto delim = key.find_last_of("_") + 1; + if (delim == std::string::npos) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected format of metadata files"); + const auto timestamp = parse(key.substr(delim + 1)); + + auto it = latest_parts.find(partition_path); + if (it == latest_parts.end()) { - if (it->second.second < timestamp) - { - it->second = {key, timestamp}; - } + latest_parts.emplace(partition_path, FileInfo{filename, timestamp}); } - else + else if (it->second.timestamp < timestamp) { - latest_parts[path] = {key, timestamp}; + it->second = {filename, timestamp}; } } - std::vector filtered_keys; - std::transform( - latest_parts.begin(), latest_parts.end(), std::back_inserter(filtered_keys), [](const auto & kv) { return kv.second.first; }); + std::string list_of_keys; - std::string new_query; - - for (auto && key : filtered_keys) + for (const auto & [directory, file_info] : latest_parts) { - if (!new_query.empty()) - { - new_query += ","; - } - new_query += key; - } - new_query = "{" + new_query + "}"; + if (!list_of_keys.empty()) + list_of_keys += ","; - return new_query; + list_of_keys += std::filesystem::path(directory) / file_info.filename; + } + + return "{" + list_of_keys + "}"; } @@ -207,7 +199,6 @@ void registerStorageHudi(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage Hudi requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); - StorageS3Configuration configuration; configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); @@ -216,10 +207,11 @@ void registerStorageHudi(StorageFactory & factory) if (engine_args.size() == 4) configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); - - // Apache Hudi uses Parquet by default - if (configuration.format == "auto") + else + { + // Apache Hudi uses Parquet by default configuration.format = "Parquet"; + } auto format_settings = getFormatSettings(args.getContext()); diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index c25c347f4c2..bebda4cd4f6 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -23,9 +23,9 @@ namespace DB class StorageHudi : public IStorage { public: - // 1. Parses internal file structure of table - // 2. Finds out parts with latest version - // 3. Creates url for underlying StorageS3 enigne to handle reads + /// 1. Parses internal file structure of table. + /// 2. Finds out parts with latest version. + /// 3. Creates url for underlying StorageS3 enigne to handle reads. StorageHudi( const StorageS3Configuration & configuration_, const StorageID & table_id_, @@ -38,7 +38,7 @@ public: String getName() const override { return "Hudi"; } - // Reads latest version of Apache Hudi table + /// Reads latest version of Apache Hudi table Pipe read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, @@ -51,12 +51,12 @@ public: private: std::vector getKeysFromS3(); - // Apache Hudi store parts of data in different files - // Every part file has timestamp in it - // Every partition(directory) in Apache Hudi has different versions of part - // To find needed parts we need to find out latest part file for every partition - // Part format is usually parquet, but can differ - static std::string generateQueryFromKeys(std::vector && keys, String format); + /// Apache Hudi store parts of data in different files. + /// Every part file has timestamp in it. + /// Every partition(directory) in Apache Hudi has different versions of part. + /// To find needed parts we need to find out latest part file for every partition. + /// Part format is usually parquet, but can differ. + static String generateQueryFromKeys(const std::vector & keys, const String & format); StorageS3::S3Configuration base_configuration; std::shared_ptr s3engine; From 65c86e9ebd8ea03bedeec0422006fb730d7ce5a1 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 7 Nov 2022 13:10:18 +0100 Subject: [PATCH 119/151] Add `on: workflow_call` to debug CI --- .github/workflows/debug.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/debug.yml b/.github/workflows/debug.yml index fa980a95a39..993fa8c0d07 100644 --- a/.github/workflows/debug.yml +++ b/.github/workflows/debug.yml @@ -2,7 +2,7 @@ name: Debug 'on': - [push, pull_request, release, workflow_dispatch] + [push, pull_request, release, workflow_dispatch, workflow_call] jobs: DebugInfo: From 105c6a70f00c25d6e04d5ad2ee179a90f3f370a3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 7 Nov 2022 13:19:40 +0100 Subject: [PATCH 120/151] Update GH actions checkers --- docker/test/style/Dockerfile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index 683124feaa0..cb8c914e53d 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -1,7 +1,7 @@ # docker build -t clickhouse/style-test . FROM ubuntu:20.04 -ARG ACT_VERSION=0.2.25 -ARG ACTIONLINT_VERSION=1.6.8 +ARG ACT_VERSION=0.2.33 +ARG ACTIONLINT_VERSION=1.6.22 # ARG for quick switch to a given ubuntu mirror ARG apt_archive="http://archive.ubuntu.com" From e7fbe40b1c112adb985c39eda064134801d2a384 Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Mon, 7 Nov 2022 14:30:52 +0100 Subject: [PATCH 121/151] Simple fixes for restart replica description --- docs/en/sql-reference/statements/system.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index e9ff4d45c79..2257bbe64f3 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -281,8 +281,8 @@ After running this statement the `[db.]replicated_merge_tree_family_table_name` ### RESTART REPLICA -Provides possibility to reinitialize Zookeeper sessions state for `ReplicatedMergeTree` table, will compare current state with Zookeeper as source of true and add tasks to Zookeeper queue if needed. -Initialization replication queue based on ZooKeeper date happens in the same way as `ATTACH TABLE` statement. For a short time the table will be unavailable for any operations. +Provides possibility to reinitialize Zookeeper session's state for `ReplicatedMergeTree` table, will compare current state with Zookeeper as source of truth and add tasks to Zookeeper queue if needed. +Initialization replication queue based on ZooKeeper date happens in the same way as for `ATTACH TABLE` statement. For a short time, the table will be unavailable for any operations. ``` sql SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name From 500609ccd31b5f199a7841d95b24ffd519e3fc66 Mon Sep 17 00:00:00 2001 From: xlwh <1079039435@qq.com> Date: Tue, 31 May 2022 17:07:27 +0800 Subject: [PATCH 122/151] Add miscellaneous function displayName() --- src/Functions/serverConstants.cpp | 15 +++++++++++++++ .../0_stateless/02313_displayname_test.reference | 2 ++ .../0_stateless/02313_displayname_test.sql | 2 ++ 3 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02313_displayname_test.reference create mode 100644 tests/queries/0_stateless/02313_displayname_test.sql diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index ccefd74b85b..3a213adb02d 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -1,4 +1,6 @@ #include +#include +#include #include #include #include @@ -115,6 +117,13 @@ namespace static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } }; + class FunctionDisplayName : public FunctionConstantBase + { + public: + static constexpr auto name = "displayName"; + static FunctionPtr create(ContextPtr context) {return std::make_shared(context); } + explicit FunctionDisplayName(ContextPtr context) : FunctionConstantBase(context->getConfigRef().getString("display_name", getFQDNOrHostName()), context->isDistributed()) {} + }; } #if defined(__ELF__) && !defined(OS_FREEBSD) @@ -173,4 +182,10 @@ REGISTER_FUNCTION(GetOSKernelVersion) } +REGISTER_FUNCTION(DisplayName) +{ + factory.registerFunction(); +} + + } diff --git a/tests/queries/0_stateless/02313_displayname_test.reference b/tests/queries/0_stateless/02313_displayname_test.reference new file mode 100644 index 00000000000..20030bef031 --- /dev/null +++ b/tests/queries/0_stateless/02313_displayname_test.reference @@ -0,0 +1,2 @@ +localhost +localhost \ No newline at end of file diff --git a/tests/queries/0_stateless/02313_displayname_test.sql b/tests/queries/0_stateless/02313_displayname_test.sql new file mode 100644 index 00000000000..c0c91287e3d --- /dev/null +++ b/tests/queries/0_stateless/02313_displayname_test.sql @@ -0,0 +1,2 @@ +select displayname(); +select displayName(); \ No newline at end of file From b3435a7b9ca0cdfbcb3774f050988b7270145597 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Tue, 31 May 2022 15:41:47 +0200 Subject: [PATCH 123/151] =?UTF-8?q?Add=20newlines=20to=2002313=5Fdisplayna?= =?UTF-8?q?me=5Ftes=D0=B5?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- tests/queries/0_stateless/02313_displayname_test.reference | 2 +- tests/queries/0_stateless/02313_displayname_test.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02313_displayname_test.reference b/tests/queries/0_stateless/02313_displayname_test.reference index 20030bef031..c3fdafec5eb 100644 --- a/tests/queries/0_stateless/02313_displayname_test.reference +++ b/tests/queries/0_stateless/02313_displayname_test.reference @@ -1,2 +1,2 @@ localhost -localhost \ No newline at end of file +localhost diff --git a/tests/queries/0_stateless/02313_displayname_test.sql b/tests/queries/0_stateless/02313_displayname_test.sql index c0c91287e3d..f6db937f1b8 100644 --- a/tests/queries/0_stateless/02313_displayname_test.sql +++ b/tests/queries/0_stateless/02313_displayname_test.sql @@ -1,2 +1,2 @@ select displayname(); -select displayName(); \ No newline at end of file +select displayName(); From 136f589343b4ce985bf33b662f64f4a80fb67672 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Wed, 24 Aug 2022 18:38:15 +0200 Subject: [PATCH 124/151] Set display_name in functional tests --- tests/config/config.d/display_name.xml | 3 +++ tests/queries/0_stateless/02313_displayname_test.reference | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) create mode 100644 tests/config/config.d/display_name.xml diff --git a/tests/config/config.d/display_name.xml b/tests/config/config.d/display_name.xml new file mode 100644 index 00000000000..ba4d49c4838 --- /dev/null +++ b/tests/config/config.d/display_name.xml @@ -0,0 +1,3 @@ + + functional-tests + \ No newline at end of file diff --git a/tests/queries/0_stateless/02313_displayname_test.reference b/tests/queries/0_stateless/02313_displayname_test.reference index c3fdafec5eb..0c0e11ecf8e 100644 --- a/tests/queries/0_stateless/02313_displayname_test.reference +++ b/tests/queries/0_stateless/02313_displayname_test.reference @@ -1,2 +1,2 @@ -localhost -localhost +functional-tests +functional-tests From 0481999e3558a61565f88f7aba01eb64eb3f070c Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Mon, 19 Sep 2022 10:21:35 +0200 Subject: [PATCH 125/151] Fix displayName test, add doc --- src/Functions/serverConstants.cpp | 14 ++++++++++++-- tests/config/config.d/display_name.xml | 2 +- ..._test.reference => 02313_displayname.reference} | 1 - ..._displayname_test.sql => 02313_displayname.sql} | 1 - 4 files changed, 13 insertions(+), 5 deletions(-) rename tests/queries/0_stateless/{02313_displayname_test.reference => 02313_displayname.reference} (50%) rename tests/queries/0_stateless/{02313_displayname_test.sql => 02313_displayname.sql} (50%) diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 3a213adb02d..96615d0a4c9 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -121,8 +121,8 @@ namespace { public: static constexpr auto name = "displayName"; - static FunctionPtr create(ContextPtr context) {return std::make_shared(context); } explicit FunctionDisplayName(ContextPtr context) : FunctionConstantBase(context->getConfigRef().getString("display_name", getFQDNOrHostName()), context->isDistributed()) {} + static FunctionPtr create(ContextPtr context) {return std::make_shared(context); } }; } @@ -184,7 +184,17 @@ REGISTER_FUNCTION(GetOSKernelVersion) REGISTER_FUNCTION(DisplayName) { - factory.registerFunction(); + factory.registerFunction( + { + R"( +Returns the value of `display_name` from config or server FQDN if not set. + +[example:displayName] +)", + Documentation::Examples{{"displayName", "SELECT displayName();"}}, + Documentation::Categories{"Constant", "Miscellaneous"} + }, + FunctionFactory::CaseSensitive); } diff --git a/tests/config/config.d/display_name.xml b/tests/config/config.d/display_name.xml index ba4d49c4838..d386a8efaf2 100644 --- a/tests/config/config.d/display_name.xml +++ b/tests/config/config.d/display_name.xml @@ -1,3 +1,3 @@ functional-tests - \ No newline at end of file + diff --git a/tests/queries/0_stateless/02313_displayname_test.reference b/tests/queries/0_stateless/02313_displayname.reference similarity index 50% rename from tests/queries/0_stateless/02313_displayname_test.reference rename to tests/queries/0_stateless/02313_displayname.reference index 0c0e11ecf8e..604a4944b33 100644 --- a/tests/queries/0_stateless/02313_displayname_test.reference +++ b/tests/queries/0_stateless/02313_displayname.reference @@ -1,2 +1 @@ functional-tests -functional-tests diff --git a/tests/queries/0_stateless/02313_displayname_test.sql b/tests/queries/0_stateless/02313_displayname.sql similarity index 50% rename from tests/queries/0_stateless/02313_displayname_test.sql rename to tests/queries/0_stateless/02313_displayname.sql index f6db937f1b8..6b6411b798b 100644 --- a/tests/queries/0_stateless/02313_displayname_test.sql +++ b/tests/queries/0_stateless/02313_displayname.sql @@ -1,2 +1 @@ -select displayname(); select displayName(); From ebb51a2023c5873ef9e3192cf0cd7a6997082229 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 7 Nov 2022 10:53:36 +0000 Subject: [PATCH 126/151] Use config.d/display_name.xml in tests --- tests/config/install.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/install.sh b/tests/config/install.sh index 9a172f0be15..876d05507fe 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -51,6 +51,7 @@ ln -sf $SRC_PATH/config.d/system_unfreeze.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_zero_copy_replication.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/nlp.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_keeper_map.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/display_name.xml $DEST_SERVER_PATH/config.d/ # Not supported with fasttest. if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ] From a8098db25de62eb1d5ffae6a1c4f3ad988f357f3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 4 Nov 2022 09:09:48 +0000 Subject: [PATCH 127/151] Minor cleanups Semantics are unchanged. Some special case handling was changed to early-out, because of that the indentation changed but the logic is the same as before. --- src/Functions/FunctionsStringSearch.h | 27 +- src/Functions/MatchImpl.h | 353 ++++++++++++-------------- src/Functions/notLike.cpp | 3 +- 3 files changed, 184 insertions(+), 199 deletions(-) diff --git a/src/Functions/FunctionsStringSearch.h b/src/Functions/FunctionsStringSearch.h index 68425ee496e..d189a0bacf4 100644 --- a/src/Functions/FunctionsStringSearch.h +++ b/src/Functions/FunctionsStringSearch.h @@ -91,23 +91,30 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.size() < 2 || 3 < arguments.size()) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be 2 or 3.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3.", + getName(), arguments.size()); if (!isStringOrFixedString(arguments[0])) throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[0]->getName(), getName()); if (!isString(arguments[1])) throw Exception( - "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[1]->getName(), getName()); if (arguments.size() >= 3) { if (!isUnsignedInteger(arguments[2])) throw Exception( - "Illegal type " + arguments[2]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[2]->getName(), getName()); } return std::make_shared>(); @@ -196,9 +203,11 @@ public: vec_res); else throw Exception( - "Illegal columns " + arguments[0].column->getName() + " and " - + arguments[1].column->getName() + " of arguments of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); + ErrorCodes::ILLEGAL_COLUMN, + "Illegal columns {} and {} of arguments of function {}", + arguments[0].column->getName(), + arguments[1].column->getName(), + getName()); return col_res; } diff --git a/src/Functions/MatchImpl.h b/src/Functions/MatchImpl.h index 6862a097d0e..edb0df3ae34 100644 --- a/src/Functions/MatchImpl.h +++ b/src/Functions/MatchImpl.h @@ -25,7 +25,7 @@ namespace impl /// Is the [I]LIKE expression reduced to finding a substring in a string? inline bool likePatternIsSubstring(std::string_view pattern, String & res) { - if (pattern.size() < 2 || pattern.front() != '%' || pattern.back() != '%') + if (pattern.size() < 2 || !pattern.starts_with('%') || !pattern.ends_with('%')) return false; res.clear(); @@ -101,9 +101,7 @@ struct MatchImpl static constexpr bool case_insensitive = (case_ == MatchTraits::Case::Insensitive); static constexpr bool negate = (result_ == MatchTraits::Result::Negate); - using Searcher = std::conditional_t; + using Searcher = std::conditional_t; static void vectorConstant( const ColumnString::Chars & haystack_data, @@ -115,13 +113,12 @@ struct MatchImpl const size_t haystack_size = haystack_offsets.size(); assert(haystack_size == res.size()); - assert(start_pos_ == nullptr); if (haystack_offsets.empty()) return; - /// A simple case where the [I]LIKE expression reduces to finding a substring in a string + /// Special case that the [I]LIKE expression reduces to finding a substring in a string String strstr_pattern; if (is_like && impl::likePatternIsSubstring(needle, strstr_pattern)) { @@ -158,105 +155,101 @@ struct MatchImpl /// Tail, in which there can be no substring. if (i < res.size()) memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); + + return; + } + + const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); + + String required_substring; + bool is_trivial; + bool required_substring_is_prefix; /// for `anchored` execution of the regexp. + + regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + + if (required_substring.empty()) + { + if (!regexp.getRE2()) /// An empty regexp. Always matches. + memset(res.data(), !negate, haystack_size * sizeof(res[0])); + else + { + size_t prev_offset = 0; + for (size_t i = 0; i < haystack_size; ++i) + { + const bool match = regexp.getRE2()->Match( + {reinterpret_cast(&haystack_data[prev_offset]), haystack_offsets[i] - prev_offset - 1}, + 0, + haystack_offsets[i] - prev_offset - 1, + re2_st::RE2::UNANCHORED, + nullptr, + 0); + res[i] = negate ^ match; + + prev_offset = haystack_offsets[i]; + } + } } else { - const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); + /// NOTE This almost matches with the case of impl::likePatternIsSubstring. - String required_substring; - bool is_trivial; - bool required_substring_is_prefix; /// for `anchored` execution of the regexp. + const UInt8 * const begin = haystack_data.data(); + const UInt8 * const end = haystack_data.begin() + haystack_data.size(); + const UInt8 * pos = begin; - regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + /// The current index in the array of strings. + size_t i = 0; - if (required_substring.empty()) + Searcher searcher(required_substring.data(), required_substring.size(), end - pos); + + /// We will search for the next occurrence in all rows at once. + while (pos < end && end != (pos = searcher.search(pos, end - pos))) { - if (!regexp.getRE2()) /// An empty regexp. Always matches. + /// Determine which index it refers to. + while (begin + haystack_offsets[i] <= pos) { - if (haystack_size) - memset(res.data(), !negate, haystack_size * sizeof(res[0])); + res[i] = negate; + ++i; } - else + + /// We check that the entry does not pass through the boundaries of strings. + if (pos + required_substring.size() < begin + haystack_offsets[i]) { - size_t prev_offset = 0; - for (size_t i = 0; i < haystack_size; ++i) + /// And if it does not, if necessary, we check the regexp. + if (is_trivial) + res[i] = !negate; + else { + const char * str_data = reinterpret_cast(&haystack_data[haystack_offsets[i - 1]]); + size_t str_size = haystack_offsets[i] - haystack_offsets[i - 1] - 1; + + /** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp, + * so that it can match when `required_substring` occurs into the string several times, + * and at the first occurrence, the regexp is not a match. + */ + const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast(pos) - str_data) : 0; + const size_t end_pos = str_size; + const bool match = regexp.getRE2()->Match( - {reinterpret_cast(&haystack_data[prev_offset]), haystack_offsets[i] - prev_offset - 1}, - 0, - haystack_offsets[i] - prev_offset - 1, + {str_data, str_size}, + start_pos, + end_pos, re2_st::RE2::UNANCHORED, nullptr, 0); res[i] = negate ^ match; - - prev_offset = haystack_offsets[i]; } } + else + res[i] = negate; + + pos = begin + haystack_offsets[i]; + ++i; } - else - { - /// NOTE This almost matches with the case of impl::likePatternIsSubstring. - const UInt8 * const begin = haystack_data.data(); - const UInt8 * const end = haystack_data.begin() + haystack_data.size(); - const UInt8 * pos = begin; - - /// The current index in the array of strings. - size_t i = 0; - - Searcher searcher(required_substring.data(), required_substring.size(), end - pos); - - /// We will search for the next occurrence in all rows at once. - while (pos < end && end != (pos = searcher.search(pos, end - pos))) - { - /// Determine which index it refers to. - while (begin + haystack_offsets[i] <= pos) - { - res[i] = negate; - ++i; - } - - /// We check that the entry does not pass through the boundaries of strings. - if (pos + required_substring.size() < begin + haystack_offsets[i]) - { - /// And if it does not, if necessary, we check the regexp. - - if (is_trivial) - res[i] = !negate; - else - { - const char * str_data = reinterpret_cast(&haystack_data[haystack_offsets[i - 1]]); - size_t str_size = haystack_offsets[i] - haystack_offsets[i - 1] - 1; - - /** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp, - * so that it can match when `required_substring` occurs into the string several times, - * and at the first occurrence, the regexp is not a match. - */ - const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast(pos) - str_data) : 0; - const size_t end_pos = str_size; - - const bool match = regexp.getRE2()->Match( - {str_data, str_size}, - start_pos, - end_pos, - re2_st::RE2::UNANCHORED, - nullptr, - 0); - res[i] = negate ^ match; - } - } - else - res[i] = negate; - - pos = begin + haystack_offsets[i]; - ++i; - } - - /// Tail, in which there can be no substring. - if (i < res.size()) - memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); - } + /// Tail, in which there can be no substring. + if (i < res.size()) + memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); } } @@ -274,7 +267,7 @@ struct MatchImpl if (haystack.empty()) return; - /// A simple case where the LIKE expression reduces to finding a substring in a string + /// Special case that the [I]LIKE expression reduces to finding a substring in a string String strstr_pattern; if (is_like && impl::likePatternIsSubstring(needle, strstr_pattern)) { @@ -316,109 +309,105 @@ struct MatchImpl /// Tail, in which there can be no substring. if (i < res.size()) memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); + + return; + } + + const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); + + String required_substring; + bool is_trivial; + bool required_substring_is_prefix; /// for `anchored` execution of the regexp. + + regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + + if (required_substring.empty()) + { + if (!regexp.getRE2()) /// An empty regexp. Always matches. + memset(res.data(), !negate, haystack_size * sizeof(res[0])); + else + { + size_t offset = 0; + for (size_t i = 0; i < haystack_size; ++i) + { + const bool match = regexp.getRE2()->Match( + {reinterpret_cast(&haystack[offset]), N}, + 0, + N, + re2_st::RE2::UNANCHORED, + nullptr, + 0); + res[i] = negate ^ match; + + offset += N; + } + } } else { - const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); + /// NOTE This almost matches with the case of likePatternIsSubstring. - String required_substring; - bool is_trivial; - bool required_substring_is_prefix; /// for `anchored` execution of the regexp. + const UInt8 * const begin = haystack.data(); + const UInt8 * const end = haystack.data() + haystack.size(); + const UInt8 * pos = begin; - regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + size_t i = 0; + const UInt8 * next_pos = begin; - if (required_substring.empty()) + /// If required substring is larger than string size - it cannot be found. + if (required_substring.size() <= N) { - if (!regexp.getRE2()) /// An empty regexp. Always matches. + Searcher searcher(required_substring.data(), required_substring.size(), end - pos); + + /// We will search for the next occurrence in all rows at once. + while (pos < end && end != (pos = searcher.search(pos, end - pos))) { - if (haystack_size) - memset(res.data(), !negate, haystack_size * sizeof(res[0])); - } - else - { - size_t offset = 0; - for (size_t i = 0; i < haystack_size; ++i) + /// Let's determine which index it refers to. + while (next_pos + N <= pos) { - const bool match = regexp.getRE2()->Match( - {reinterpret_cast(&haystack[offset]), N}, - 0, - N, - re2_st::RE2::UNANCHORED, - nullptr, - 0); - res[i] = negate ^ match; - - offset += N; - } - } - } - else - { - /// NOTE This almost matches with the case of likePatternIsSubstring. - - const UInt8 * const begin = haystack.data(); - const UInt8 * const end = haystack.data() + haystack.size(); - const UInt8 * pos = begin; - - size_t i = 0; - const UInt8 * next_pos = begin; - - /// If required substring is larger than string size - it cannot be found. - if (required_substring.size() <= N) - { - Searcher searcher(required_substring.data(), required_substring.size(), end - pos); - - /// We will search for the next occurrence in all rows at once. - while (pos < end && end != (pos = searcher.search(pos, end - pos))) - { - /// Let's determine which index it refers to. - while (next_pos + N <= pos) - { - res[i] = negate; - next_pos += N; - ++i; - } + res[i] = negate; next_pos += N; - - if (pos + required_substring.size() <= next_pos) - { - /// And if it does not, if necessary, we check the regexp. - - if (is_trivial) - res[i] = !negate; - else - { - const char * str_data = reinterpret_cast(next_pos - N); - - /** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp, - * so that it can match when `required_substring` occurs into the string several times, - * and at the first occurrence, the regexp is not a match. - */ - const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast(pos) - str_data) : 0; - const size_t end_pos = N; - - const bool match = regexp.getRE2()->Match( - {str_data, N}, - start_pos, - end_pos, - re2_st::RE2::UNANCHORED, - nullptr, - 0); - res[i] = negate ^ match; - } - } - else - res[i] = negate; - - pos = next_pos; ++i; } - } + next_pos += N; - /// Tail, in which there can be no substring. - if (i < res.size()) - memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); + if (pos + required_substring.size() <= next_pos) + { + /// And if it does not, if necessary, we check the regexp. + if (is_trivial) + res[i] = !negate; + else + { + const char * str_data = reinterpret_cast(next_pos - N); + + /** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp, + * so that it can match when `required_substring` occurs into the string several times, + * and at the first occurrence, the regexp is not a match. + */ + const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast(pos) - str_data) : 0; + const size_t end_pos = N; + + const bool match = regexp.getRE2()->Match( + {str_data, N}, + start_pos, + end_pos, + re2_st::RE2::UNANCHORED, + nullptr, + 0); + res[i] = negate ^ match; + } + } + else + res[i] = negate; + + pos = next_pos; + ++i; + } } + + /// Tail, in which there can be no substring. + if (i < res.size()) + memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); } } @@ -434,7 +423,6 @@ struct MatchImpl assert(haystack_size == needle_offset.size()); assert(haystack_size == res.size()); - assert(start_pos_ == nullptr); if (haystack_offsets.empty()) @@ -481,9 +469,7 @@ struct MatchImpl if (required_substr.empty()) { if (!regexp->getRE2()) /// An empty regexp. Always matches. - { res[i] = !negate; - } else { const bool match = regexp->getRE2()->Match( @@ -502,15 +488,11 @@ struct MatchImpl const auto * match = searcher.search(cur_haystack_data, cur_haystack_length); if (match == cur_haystack_data + cur_haystack_length) - { res[i] = negate; // no match - } else { if (is_trivial) - { res[i] = !negate; // no wildcards in pattern - } else { const size_t start_pos = (required_substring_is_prefix) ? (match - cur_haystack_data) : 0; @@ -546,7 +528,6 @@ struct MatchImpl assert(haystack_size == needle_offset.size()); assert(haystack_size == res.size()); - assert(start_pos_ == nullptr); if (haystack.empty()) @@ -593,9 +574,7 @@ struct MatchImpl if (required_substr.empty()) { if (!regexp->getRE2()) /// An empty regexp. Always matches. - { res[i] = !negate; - } else { const bool match = regexp->getRE2()->Match( @@ -614,15 +593,11 @@ struct MatchImpl const auto * match = searcher.search(cur_haystack_data, cur_haystack_length); if (match == cur_haystack_data + cur_haystack_length) - { res[i] = negate; // no match - } else { if (is_trivial) - { res[i] = !negate; // no wildcards in pattern - } else { const size_t start_pos = (required_substring_is_prefix) ? (match - cur_haystack_data) : 0; diff --git a/src/Functions/notLike.cpp b/src/Functions/notLike.cpp index a546b511a0b..200890d77d8 100644 --- a/src/Functions/notLike.cpp +++ b/src/Functions/notLike.cpp @@ -12,7 +12,8 @@ struct NameNotLike static constexpr auto name = "notLike"; }; -using FunctionNotLike = FunctionsStringSearch>; +using NotLikeImpl = MatchImpl; +using FunctionNotLike = FunctionsStringSearch; } From 8fa5cae01a37c625a6675cdd9b6ef34f200d65c8 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 7 Nov 2022 14:11:02 +0000 Subject: [PATCH 128/151] Fix style check --- src/Analyzer/QueryTreePassManager.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index d59e7457ed2..9dc9cda44a3 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -28,6 +28,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } namespace From 891e1675093aea0eefd62dc8f5bb62709b566aa3 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 7 Nov 2022 15:17:16 +0100 Subject: [PATCH 129/151] Update StorageHudi.cpp --- src/Storages/StorageHudi.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index d31d04beef5..986bb6b955c 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int S3_ERROR; + extern const int LOGICAL_ERROR; } StorageHudi::StorageHudi( From d4cdc38b71dc6e443045070bbec7166beb9cc369 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Mon, 7 Nov 2022 22:49:58 +0800 Subject: [PATCH 130/151] better --- .../System/StorageSystemDetachedParts.cpp | 30 +++++++------------ 1 file changed, 10 insertions(+), 20 deletions(-) diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 11c4f7843ac..75bf3fdf72a 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -36,28 +36,11 @@ static void calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & fr /// Files or directories of detached part may not exist. Only count the size of existing files. if (disk->isFile(from)) { - try - { - total_size += disk->getFileSize(from); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + total_size += disk->getFileSize(from); } else { - DirectoryIteratorPtr it; - try - { - it = disk->iterateDirectory(from); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - - for (; it->isValid(); it->next()) + for (auto it = disk->iterateDirectory(from); it->isValid(); it->next()) calculateTotalSizeOnDiskImpl(disk, fs::path(from) / it->name(), total_size); } } @@ -65,7 +48,14 @@ static void calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & fr static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from) { UInt64 total_size = 0; - calculateTotalSizeOnDiskImpl(disk, from, total_size); + try + { + calculateTotalSizeOnDiskImpl(disk, from, total_size); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } return total_size; } From d6bfbeb95f1ec6ede8e701aa6bfb3220c2e6b6f9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 7 Nov 2022 15:53:43 +0100 Subject: [PATCH 131/151] Fix TSan errors (correctly ignore _exit interception) Because safeExit() does not includes header with defines, it does not know about THREAD_SANITIZER. And it also fixes Azure blob storage, actually everything is fine with the sdk itself, the problem is only in TSan that intercepts _exit() and report leak, even thoug that tread will be joined later. Refs: #23056 (#23616) Fixes: #38474 Closes: #42640 Fixes: #42638 Fixes: #34988 Cc: @alexey-milovidov, @tavplubix Signed-off-by: Azat Khuzhin --- base/base/safeExit.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/base/base/safeExit.cpp b/base/base/safeExit.cpp index ddb93dac65b..2d4e5cf43b4 100644 --- a/base/base/safeExit.cpp +++ b/base/base/safeExit.cpp @@ -3,6 +3,7 @@ #endif #include #include +#include /// for THREAD_SANITIZER [[noreturn]] void safeExit(int code) { From 58a12300ef46c7d2dd4712d332a701f2b7b03603 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 7 Nov 2022 15:33:53 +0100 Subject: [PATCH 132/151] Update src/Functions/FunctionsStringSearch.h Co-authored-by: Vladimir C --- src/Functions/FunctionsStringSearch.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsStringSearch.h b/src/Functions/FunctionsStringSearch.h index d189a0bacf4..d8da525e63a 100644 --- a/src/Functions/FunctionsStringSearch.h +++ b/src/Functions/FunctionsStringSearch.h @@ -93,7 +93,7 @@ public: if (arguments.size() < 2 || 3 < arguments.size()) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3.", + "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", getName(), arguments.size()); if (!isStringOrFixedString(arguments[0])) From d446eca8826e1f45c9380d753fc8dba1677e314e Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 7 Nov 2022 11:56:09 -0500 Subject: [PATCH 133/151] move troubleshooting to FAQ --- .../{troubleshooting.md => _troubleshooting.md} | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) rename docs/en/operations/{troubleshooting.md => _troubleshooting.md} (98%) diff --git a/docs/en/operations/troubleshooting.md b/docs/en/operations/_troubleshooting.md similarity index 98% rename from docs/en/operations/troubleshooting.md rename to docs/en/operations/_troubleshooting.md index ad92e773ea3..aed63ec4d0f 100644 --- a/docs/en/operations/troubleshooting.md +++ b/docs/en/operations/_troubleshooting.md @@ -1,9 +1,5 @@ ---- -slug: /en/operations/troubleshooting -sidebar_position: 46 -sidebar_label: Troubleshooting -title: Troubleshooting ---- + +[//]: # (This file is included in FAQ > Troubleshooting) - [Installation](#troubleshooting-installation-errors) - [Connecting to the server](#troubleshooting-accepts-no-connections) From b4ecbbf22469cd4a827aec1cd604b3ea900a26e1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 7 Nov 2022 21:07:09 +0300 Subject: [PATCH 134/151] Update safeExit.cpp --- base/base/safeExit.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/base/base/safeExit.cpp b/base/base/safeExit.cpp index 2d4e5cf43b4..12ad9dc12ee 100644 --- a/base/base/safeExit.cpp +++ b/base/base/safeExit.cpp @@ -1,6 +1,7 @@ #if defined(OS_LINUX) # include #endif +#include #include #include #include /// for THREAD_SANITIZER From da114e7ab7ebff76449ce0f3274d4c805a1d6a56 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Nov 2022 18:29:00 +0000 Subject: [PATCH 135/151] fix bandwidth throttlers initialization order --- src/Interpreters/ProcessList.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 551d20f835a..84f5570349b 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -263,6 +263,11 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as user_process_list.user_memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator_for_user); user_process_list.user_memory_tracker.setDescription("(for user)"); + if (!total_network_throttler && settings.max_network_bandwidth_for_all_users) + { + total_network_throttler = std::make_shared(settings.max_network_bandwidth_for_all_users); + } + if (!user_process_list.user_throttler) { if (settings.max_network_bandwidth_for_user) @@ -270,11 +275,6 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as else if (settings.max_network_bandwidth_for_all_users) user_process_list.user_throttler = total_network_throttler; } - - if (!total_network_throttler && settings.max_network_bandwidth_for_all_users) - { - total_network_throttler = std::make_shared(settings.max_network_bandwidth_for_all_users); - } } return res; From 434c0f24f9a342470d702db60a740570597c10f5 Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Mon, 7 Nov 2022 19:40:59 +0100 Subject: [PATCH 136/151] Update docs/en/sql-reference/statements/system.md Co-authored-by: Alexander Tokmakov --- docs/en/sql-reference/statements/system.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 2257bbe64f3..c8b104ea91f 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -282,7 +282,7 @@ After running this statement the `[db.]replicated_merge_tree_family_table_name` ### RESTART REPLICA Provides possibility to reinitialize Zookeeper session's state for `ReplicatedMergeTree` table, will compare current state with Zookeeper as source of truth and add tasks to Zookeeper queue if needed. -Initialization replication queue based on ZooKeeper date happens in the same way as for `ATTACH TABLE` statement. For a short time, the table will be unavailable for any operations. +Initialization of replication queue based on ZooKeeper data happens in the same way as for `ATTACH TABLE` statement. For a short time, the table will be unavailable for any operations. ``` sql SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name From d695240d5351d75b6b424fb3223385713b6b68f1 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 7 Nov 2022 23:07:00 +0100 Subject: [PATCH 137/151] Update StorageHudi.cpp --- src/Storages/StorageHudi.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 986bb6b955c..121856c4a57 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -45,7 +45,7 @@ StorageHudi::StorageHudi( StorageS3::updateS3Configuration(context_, base_configuration); auto keys = getKeysFromS3(); - auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys), configuration_.format); + auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(keys, configuration_.format); LOG_DEBUG(log, "New uri: {}", new_uri); LOG_DEBUG(log, "Table path: {}", table_path); @@ -158,7 +158,7 @@ String StorageHudi::generateQueryFromKeys(const std::vector & keys, const String partition_path = key_path.parent_path(); /// Every filename contains metadata split by "_", timestamp is after last "_". - const auto delim = key.find_last_of("_") + 1; + const auto delim = key.find_last_of('_') + 1; if (delim == std::string::npos) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected format of metadata files"); const auto timestamp = parse(key.substr(delim + 1)); From 48cc1d8492c5838958ec14a7da6a2490ca696986 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Tue, 8 Nov 2022 00:15:19 +0100 Subject: [PATCH 138/151] Update DateLUTImpl.h --- src/Common/DateLUTImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index 3afbb6735dc..2f8aa487621 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -1331,7 +1331,7 @@ public: } template - inline auto addQuarters(DateOrTime d, Int64 delta) const + inline auto NO_SANITIZE_UNDEFINED addQuarters(DateOrTime d, Int64 delta) const { return addMonths(d, delta * 3); } From ab0174f3e864e069639c1df975466cf2ce0f6fca Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 8 Nov 2022 08:51:01 +0000 Subject: [PATCH 139/151] Add test for issue #42520 --- .../02477_number_to_date.reference | 25 +++++++++++++++++++ .../0_stateless/02477_number_to_date.sql | 17 +++++++++++++ 2 files changed, 42 insertions(+) create mode 100644 tests/queries/0_stateless/02477_number_to_date.reference create mode 100644 tests/queries/0_stateless/02477_number_to_date.sql diff --git a/tests/queries/0_stateless/02477_number_to_date.reference b/tests/queries/0_stateless/02477_number_to_date.reference new file mode 100644 index 00000000000..9161909966c --- /dev/null +++ b/tests/queries/0_stateless/02477_number_to_date.reference @@ -0,0 +1,25 @@ +-- { echoOn } +select toDate(1666249120::Float); +2022-10-20 +select toDate(1666249120::Double); +2022-10-20 +select toDate(1666249120::UInt32); +2022-10-20 +select toDate32(1666249120::Float); +2022-10-20 +select toDate32(1666249120::Double); +2022-10-20 +select toDate32(1666249120::UInt32); +2022-10-20 +select toDateTime(1666249120::Float); +2022-10-20 06:58:08 +select toDateTime(1666249120::Double); +2022-10-20 06:58:40 +select toDateTime(1666249120::UInt32); +2022-10-20 06:58:40 +select toDateTime64(1666249120::Float, 3); +2022-10-20 06:58:49.984 +select toDateTime64(1666249120::Double, 3); +2022-10-20 06:58:40.000 +select toDateTime64(1666249120::UInt32, 3); +2022-10-20 06:58:40.000 diff --git a/tests/queries/0_stateless/02477_number_to_date.sql b/tests/queries/0_stateless/02477_number_to_date.sql new file mode 100644 index 00000000000..b365bf5a014 --- /dev/null +++ b/tests/queries/0_stateless/02477_number_to_date.sql @@ -0,0 +1,17 @@ +-- { echoOn } +select toDate(1666249120::Float); +select toDate(1666249120::Double); +select toDate(1666249120::UInt32); + +select toDate32(1666249120::Float); +select toDate32(1666249120::Double); +select toDate32(1666249120::UInt32); + +select toDateTime(1666249120::Float); +select toDateTime(1666249120::Double); +select toDateTime(1666249120::UInt32); + +select toDateTime64(1666249120::Float, 3); +select toDateTime64(1666249120::Double, 3); +select toDateTime64(1666249120::UInt32, 3); +-- { echoOff } From 9ec313b05c2dbf8d699491013c8454b4470668cc Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Tue, 8 Nov 2022 18:32:53 +0800 Subject: [PATCH 140/151] remove generic driver lld Signed-off-by: Lloyd-Pottiger --- cmake/tools.cmake | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/cmake/tools.cmake b/cmake/tools.cmake index 23f34bb24cd..e8fecd9f30b 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -57,13 +57,14 @@ if (NOT LINKER_NAME) if (COMPILER_GCC) find_program (LLD_PATH NAMES "ld.lld") find_program (GOLD_PATH NAMES "ld.gold") - # llvm lld is a generic driver. - # Invoke ld.lld (Unix), ld64.lld (macOS), lld-link (Windows), wasm-ld (WebAssembly) instead - elseif (COMPILER_CLANG AND OS_LINUX) - find_program (LLD_PATH NAMES "ld.lld-${COMPILER_VERSION_MAJOR}" "lld-${COMPILER_VERSION_MAJOR}" "ld.lld" "lld") - find_program (GOLD_PATH NAMES "ld.gold" "gold") - elseif (COMPILER_CLANG AND OS_DARWIN) - find_program (LLD_PATH NAMES "ld64.lld-${COMPILER_VERSION_MAJOR}" "lld-${COMPILER_VERSION_MAJOR}" "ld64.lld" "lld") + elseif (COMPILER_CLANG) + # llvm lld is a generic driver. + # Invoke ld.lld (Unix), ld64.lld (macOS), lld-link (Windows), wasm-ld (WebAssembly) instead + if (OS_LINUX) + find_program (LLD_PATH NAMES "ld.lld-${COMPILER_VERSION_MAJOR}" "ld.lld") + elseif (OS_DARWIN) + find_program (LLD_PATH NAMES "ld64.lld-${COMPILER_VERSION_MAJOR}" "ld64.lld") + endif () find_program (GOLD_PATH NAMES "ld.gold" "gold") endif () endif() From ff083e1b55190c1db38d8f39c040242dd4e3d4fb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 7 Nov 2022 18:17:54 +0100 Subject: [PATCH 141/151] Forbit launching release.py from branches but `master` --- tests/ci/release.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/ci/release.py b/tests/ci/release.py index c0c248aa513..fd4bda3eae4 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -118,6 +118,8 @@ class Release: except subprocess.CalledProcessError: logging.fatal("Repo contains uncommitted changes") raise + if self._git.branch != "master": + raise Exception("the script must be launched only from master") self.set_release_branch() From 7549b223173d387aa65dc41c307aaf4515394577 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 7 Nov 2022 18:20:39 +0100 Subject: [PATCH 142/151] Add a script to mark commit as ready for release --- tests/ci/mark_release_ready.py | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 tests/ci/mark_release_ready.py diff --git a/tests/ci/mark_release_ready.py b/tests/ci/mark_release_ready.py new file mode 100644 index 00000000000..be1771e62bd --- /dev/null +++ b/tests/ci/mark_release_ready.py @@ -0,0 +1,25 @@ +#!/usr/bin/env python3 + +from commit_status_helper import get_commit +from env_helper import GITHUB_JOB_URL +from get_robot_token import get_best_robot_token +from github_helper import GitHub +from pr_info import PRInfo + +RELEASE_READY_STATUS = "Ready for release" + + +def main(): + pr_info = PRInfo() + gh = GitHub(get_best_robot_token(), per_page=100) + commit = get_commit(gh, pr_info.sha) + commit.create_status( + context=RELEASE_READY_STATUS, + description="the release can be created from the commit", + state="success", + target_url=GITHUB_JOB_URL(), + ) + + +if __name__ == "__main__": + main() From aebb30c7d74a01e8aae746c932b76a67adf5a4d2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 7 Nov 2022 21:28:05 +0100 Subject: [PATCH 143/151] Add CI jobs to mark commit as release ready --- .github/workflows/master.yml | 18 ++++++++++++++++++ .github/workflows/release_branches.yml | 18 ++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 3d43a960534..08177eace76 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -1056,6 +1056,23 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" + MarkReleaseReady: + needs: + - BuilderBinDarwin + - BuilderBinDarwinAarch64 + - BuilderDebRelease + - BuilderDebAarch64 + 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: Mark Commit Release Ready + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 mark_release_ready.py ############################################################################################## ########################### FUNCTIONAl STATELESS TESTS ####################################### ############################################################################################## @@ -3069,6 +3086,7 @@ jobs: needs: - DockerHubPush - BuilderReport + - MarkReleaseReady - FunctionalStatelessTestDebug0 - FunctionalStatelessTestDebug1 - FunctionalStatelessTestDebug2 diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index abe85d3e72d..8148905cec7 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -615,6 +615,23 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" + MarkReleaseReady: + needs: + - BuilderBinDarwin + - BuilderBinDarwinAarch64 + - BuilderDebRelease + - BuilderDebAarch64 + 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: Mark Commit Release Ready + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 mark_release_ready.py ############################################################################################## ########################### FUNCTIONAl STATELESS TESTS ####################################### ############################################################################################## @@ -1888,6 +1905,7 @@ jobs: - DockerServerImages - BuilderReport - BuilderSpecialReport + - MarkReleaseReady - FunctionalStatelessTestDebug0 - FunctionalStatelessTestDebug1 - FunctionalStatelessTestDebug2 From 7069919706749db9662eeb8b24cf898b883e12eb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 8 Nov 2022 11:16:01 +0100 Subject: [PATCH 144/151] Add BuilderSpecialReport to dependencies of FinishCheck --- .github/workflows/master.yml | 1 + .github/workflows/pull_request.yml | 1 + 2 files changed, 2 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 08177eace76..da84500559a 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -3086,6 +3086,7 @@ jobs: needs: - DockerHubPush - BuilderReport + - BuilderSpecialReport - MarkReleaseReady - FunctionalStatelessTestDebug0 - FunctionalStatelessTestDebug1 diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 09ca64977f0..d3580f4c4d7 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -3579,6 +3579,7 @@ jobs: - DockerServerImages - CheckLabels - BuilderReport + - BuilderSpecialReport - FastTest - FunctionalStatelessTestDebug0 - FunctionalStatelessTestDebug1 From f8bb9ce787f56d6a97e070fac7b090f26537fc00 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 8 Nov 2022 10:49:45 +0000 Subject: [PATCH 145/151] Try to fix time zone difference --- .../02477_number_to_date.reference | 25 ------------------- .../0_stateless/02477_number_to_date.sql | 17 ------------- 2 files changed, 42 deletions(-) delete mode 100644 tests/queries/0_stateless/02477_number_to_date.reference delete mode 100644 tests/queries/0_stateless/02477_number_to_date.sql diff --git a/tests/queries/0_stateless/02477_number_to_date.reference b/tests/queries/0_stateless/02477_number_to_date.reference deleted file mode 100644 index 9161909966c..00000000000 --- a/tests/queries/0_stateless/02477_number_to_date.reference +++ /dev/null @@ -1,25 +0,0 @@ --- { echoOn } -select toDate(1666249120::Float); -2022-10-20 -select toDate(1666249120::Double); -2022-10-20 -select toDate(1666249120::UInt32); -2022-10-20 -select toDate32(1666249120::Float); -2022-10-20 -select toDate32(1666249120::Double); -2022-10-20 -select toDate32(1666249120::UInt32); -2022-10-20 -select toDateTime(1666249120::Float); -2022-10-20 06:58:08 -select toDateTime(1666249120::Double); -2022-10-20 06:58:40 -select toDateTime(1666249120::UInt32); -2022-10-20 06:58:40 -select toDateTime64(1666249120::Float, 3); -2022-10-20 06:58:49.984 -select toDateTime64(1666249120::Double, 3); -2022-10-20 06:58:40.000 -select toDateTime64(1666249120::UInt32, 3); -2022-10-20 06:58:40.000 diff --git a/tests/queries/0_stateless/02477_number_to_date.sql b/tests/queries/0_stateless/02477_number_to_date.sql deleted file mode 100644 index b365bf5a014..00000000000 --- a/tests/queries/0_stateless/02477_number_to_date.sql +++ /dev/null @@ -1,17 +0,0 @@ --- { echoOn } -select toDate(1666249120::Float); -select toDate(1666249120::Double); -select toDate(1666249120::UInt32); - -select toDate32(1666249120::Float); -select toDate32(1666249120::Double); -select toDate32(1666249120::UInt32); - -select toDateTime(1666249120::Float); -select toDateTime(1666249120::Double); -select toDateTime(1666249120::UInt32); - -select toDateTime64(1666249120::Float, 3); -select toDateTime64(1666249120::Double, 3); -select toDateTime64(1666249120::UInt32, 3); --- { echoOff } From d7cd038e759b48293482caa59c70c423c1bb0451 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 8 Nov 2022 11:59:12 +0000 Subject: [PATCH 146/151] Integrate tests into 02462_int_to_date + rename to 02462_num_to_datetype --- .../0_stateless/02462_int_to_date.reference | 4 -- .../queries/0_stateless/02462_int_to_date.sql | 4 -- .../02462_number_to_datetype.reference | 49 +++++++++++++++++++ .../0_stateless/02462_number_to_datetype.sql | 29 +++++++++++ 4 files changed, 78 insertions(+), 8 deletions(-) delete mode 100644 tests/queries/0_stateless/02462_int_to_date.reference delete mode 100644 tests/queries/0_stateless/02462_int_to_date.sql create mode 100644 tests/queries/0_stateless/02462_number_to_datetype.reference create mode 100644 tests/queries/0_stateless/02462_number_to_datetype.sql diff --git a/tests/queries/0_stateless/02462_int_to_date.reference b/tests/queries/0_stateless/02462_int_to_date.reference deleted file mode 100644 index f31441cf3b8..00000000000 --- a/tests/queries/0_stateless/02462_int_to_date.reference +++ /dev/null @@ -1,4 +0,0 @@ -20221011 2022-10-11 1665519765 -20221011 2022-10-11 1665519765 -20221011 2022-10-11 1665519765 Int32 -20221011 2022-10-11 1665519765 UInt32 diff --git a/tests/queries/0_stateless/02462_int_to_date.sql b/tests/queries/0_stateless/02462_int_to_date.sql deleted file mode 100644 index cd470ca12f6..00000000000 --- a/tests/queries/0_stateless/02462_int_to_date.sql +++ /dev/null @@ -1,4 +0,0 @@ -select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp; -select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp; -select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); -select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); diff --git a/tests/queries/0_stateless/02462_number_to_datetype.reference b/tests/queries/0_stateless/02462_number_to_datetype.reference new file mode 100644 index 00000000000..baa03dafdc9 --- /dev/null +++ b/tests/queries/0_stateless/02462_number_to_datetype.reference @@ -0,0 +1,49 @@ +-- { echoOn } +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Int64 +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 UInt64 +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Int32 +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 UInt32 +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519700 Float32 +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Float64 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Int64 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 UInt64 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Int32 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 UInt32 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519700 Float32 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Float64 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45 1665519765 Int64 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45 1665519765 UInt64 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45 1665519765 Int32 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45 1665519765 UInt32 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:24 1665519700 Float32 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45 1665519765 Float64 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45.000 1665519765 Int64 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45.000 1665519765 UInt64 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45.000 1665519765 Int32 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45.000 1665519765 UInt32 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:21:54.304 1665519700 Float32 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45.000 1665519765 Float64 diff --git a/tests/queries/0_stateless/02462_number_to_datetype.sql b/tests/queries/0_stateless/02462_number_to_datetype.sql new file mode 100644 index 00000000000..d877157314c --- /dev/null +++ b/tests/queries/0_stateless/02462_number_to_datetype.sql @@ -0,0 +1,29 @@ +-- { echoOn } +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); + +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); + +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); + +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +-- { echoOff } From c3e63f77844d6656ecc5c7389019f9da817642fe Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 8 Nov 2022 12:04:12 +0000 Subject: [PATCH 147/151] Add section headings --- .../queries/0_stateless/02462_number_to_datetype.reference | 5 +++++ tests/queries/0_stateless/02462_number_to_datetype.sql | 6 ++++++ 2 files changed, 11 insertions(+) diff --git a/tests/queries/0_stateless/02462_number_to_datetype.reference b/tests/queries/0_stateless/02462_number_to_datetype.reference index baa03dafdc9..22cee1308ff 100644 --- a/tests/queries/0_stateless/02462_number_to_datetype.reference +++ b/tests/queries/0_stateless/02462_number_to_datetype.reference @@ -1,4 +1,6 @@ -- { echoOn } + +-- toDate select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); 20221011 2022-10-11 1665519765 Int64 select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); @@ -11,6 +13,7 @@ select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTim 20221011 2022-10-11 1665519700 Float32 select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); 20221011 2022-10-11 1665519765 Float64 +-- toDate32 select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); 20221011 2022-10-11 1665519765 Int64 select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); @@ -23,6 +26,7 @@ select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recor 20221011 2022-10-11 1665519700 Float32 select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); 20221011 2022-10-11 1665519765 Float64 +-- toDateTime select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); 20221011 2022-10-11 22:22:45 1665519765 Int64 select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); @@ -35,6 +39,7 @@ select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(r 20221011 2022-10-11 22:22:24 1665519700 Float32 select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); 20221011 2022-10-11 22:22:45 1665519765 Float64 +-- toDateTime64 select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); 20221011 2022-10-11 22:22:45.000 1665519765 Int64 select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); diff --git a/tests/queries/0_stateless/02462_number_to_datetype.sql b/tests/queries/0_stateless/02462_number_to_datetype.sql index d877157314c..5d361f2b0ba 100644 --- a/tests/queries/0_stateless/02462_number_to_datetype.sql +++ b/tests/queries/0_stateless/02462_number_to_datetype.sql @@ -1,4 +1,6 @@ -- { echoOn } + +-- toDate select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); @@ -6,6 +8,7 @@ select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTim select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +-- toDate32 select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); @@ -13,6 +16,7 @@ select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recor select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +-- toDateTime select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); @@ -20,10 +24,12 @@ select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(r select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +-- toDateTime64 select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); + -- { echoOff } From 583c65d37741abb209e5cd6c4850a66994e8d555 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 1 Nov 2022 15:14:07 +0100 Subject: [PATCH 148/151] Analyzer subquery in JOIN TREE with aggregation --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 12 +++++++++--- .../02475_analyzer_join_tree_subquery.reference | 3 +++ .../02475_analyzer_join_tree_subquery.sql | 7 +++++++ 3 files changed, 19 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02475_analyzer_join_tree_subquery.reference create mode 100644 tests/queries/0_stateless/02475_analyzer_join_tree_subquery.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 9db2d66d99d..4f24051df79 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5527,9 +5527,15 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier * 3. Check that there are no columns that are not specified in GROUP BY keys. * 4. Validate GROUP BY modifiers. */ - assertNoAggregateFunctionNodes(query_node_typed.getJoinTree(), "in JOIN TREE"); - assertNoGroupingFunction(query_node_typed.getJoinTree(), "in JOIN TREE"); - assertNoWindowFunctionNodes(query_node_typed.getJoinTree(), "in JOIN TREE"); + auto join_tree_node_type = query_node_typed.getJoinTree()->getNodeType(); + bool join_tree_is_subquery = join_tree_node_type == QueryTreeNodeType::QUERY || join_tree_node_type == QueryTreeNodeType::UNION; + + if (!join_tree_is_subquery) + { + assertNoAggregateFunctionNodes(query_node_typed.getJoinTree(), "in JOIN TREE"); + assertNoGroupingFunction(query_node_typed.getJoinTree(), "in JOIN TREE"); + assertNoWindowFunctionNodes(query_node_typed.getJoinTree(), "in JOIN TREE"); + } if (query_node_typed.hasWhere()) { diff --git a/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.reference b/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.reference new file mode 100644 index 00000000000..623ba8fde99 --- /dev/null +++ b/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.reference @@ -0,0 +1,3 @@ +45 +-- +45 diff --git a/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.sql b/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.sql new file mode 100644 index 00000000000..eda90529166 --- /dev/null +++ b/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.sql @@ -0,0 +1,7 @@ +SET allow_experimental_analyzer = 1; + +WITH subquery AS (SELECT sum(number) FROM numbers(10)) SELECT * FROM subquery; + +SELECT '--'; + +WITH subquery AS (SELECT sum(number) FROM numbers(10)) SELECT (SELECT * FROM subquery); From 0aa4fd3c2db9edf7bc69ff8d8ed30e9720d23ad1 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 7 Nov 2022 12:53:57 +0100 Subject: [PATCH 149/151] Fixed tests --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 674e02b16ec..4091d37ec11 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -781,6 +781,11 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( ReadFromMergeTree::IndexStats & index_stats) { const Settings & settings = context->getSettingsRef(); + + /// TODO: Analyzer syntax analyzer result + if (settings.allow_experimental_analyzer) + return; + std::optional partition_pruner; std::optional minmax_idx_condition; DataTypes minmax_columns_types; From a4cd562914b3f9c7f16ac488b04dc933dca3515c Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 8 Nov 2022 10:12:29 -0500 Subject: [PATCH 150/151] fix next button --- docs/en/operations/system-tables/index.md | 4 ++-- docs/en/operations/utilities/index.md | 5 +++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/system-tables/index.md b/docs/en/operations/system-tables/index.md index e08a727a62a..5fc302cad34 100644 --- a/docs/en/operations/system-tables/index.md +++ b/docs/en/operations/system-tables/index.md @@ -1,7 +1,8 @@ --- slug: /en/operations/system-tables/ sidebar_position: 52 -sidebar_label: System Tables +sidebar_label: Overview +pagination_next: 'en/operations/system-tables/asynchronous_metric_log' --- # System Tables @@ -72,4 +73,3 @@ If procfs is supported and enabled on the system, ClickHouse server collects the - `OSReadBytes` - `OSWriteBytes` -[Original article](https://clickhouse.com/docs/en/operations/system-tables/) diff --git a/docs/en/operations/utilities/index.md b/docs/en/operations/utilities/index.md index df4af30768c..9de68923ea4 100644 --- a/docs/en/operations/utilities/index.md +++ b/docs/en/operations/utilities/index.md @@ -1,10 +1,11 @@ --- slug: /en/operations/utilities/ sidebar_position: 56 -sidebar_label: Utilities +sidebar_label: Overview +pagination_next: 'en/operations/utilities/clickhouse-copier' --- -# ClickHouse Utility +# ClickHouse Utilities - [clickhouse-local](../../operations/utilities/clickhouse-local.md) — Allows running SQL queries on data without starting the ClickHouse server, similar to how `awk` does this. - [clickhouse-copier](../../operations/utilities/clickhouse-copier.md) — Copies (and reshards) data from one cluster to another cluster. From 050f3ca7cf514e358e2d852a56f81ffcb4162908 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 8 Nov 2022 17:25:03 +0100 Subject: [PATCH 151/151] Fix msan warning --- base/glibc-compatibility/musl/getauxval.c | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index eba12604b4d..44a9f979f99 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -10,6 +10,12 @@ #include "syscall.h" +#if defined(__has_feature) +#if __has_feature(memory_sanitizer) +#include +#endif +#endif + #define ARRAY_SIZE(a) sizeof((a))/sizeof((a[0])) /// Suppress TSan since it is possible for this code to be called from multiple threads,