From 1d81bd539bc4e2041d6a494ab915ae9ca52389d8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 9 Aug 2023 10:14:10 +0000 Subject: [PATCH 001/644] Constexprify conditions --- src/Storages/Kafka/StorageKafka.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index c17defca673..723399bf206 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -939,14 +939,14 @@ void registerStorageKafka(StorageFactory & factory) /* move engine args to settings */ \ else \ { \ - if ((EVAL) == 1) \ + if constexpr ((EVAL) == 1) \ { \ engine_args[(ARG_NUM)-1] = \ evaluateConstantExpressionAsLiteral( \ engine_args[(ARG_NUM)-1], \ args.getLocalContext()); \ } \ - if ((EVAL) == 2) \ + if constexpr ((EVAL) == 2) \ { \ engine_args[(ARG_NUM)-1] = \ evaluateConstantExpressionOrIdentifierAsLiteral( \ From 72f21065bf9903d7f3e6178143dc093528ab0b86 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 10 Aug 2023 10:56:17 +0000 Subject: [PATCH 002/644] Add keeper path to Kafka table engine --- src/Core/Settings.h | 1 + src/Storages/Kafka/KafkaSettings.h | 1 + src/Storages/Kafka/StorageKafka.cpp | 9 +++++++++ 3 files changed, 11 insertions(+) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a6280a28436..bba0791e1fc 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -649,6 +649,7 @@ class IColumn; M(UInt64, max_size_to_preallocate_for_aggregation, 100'000'000, "For how many elements it is allowed to preallocate space in all hash tables in total before aggregation", 0) \ \ M(Bool, kafka_disable_num_consumers_limit, false, "Disable limit on kafka_num_consumers that depends on the number of available CPU cores", 0) \ + M(Bool, allow_experimental_kafka_store_offsets_in_keeper, false, "Allow experimental feature to store Kafka related offsets in Keeper", 0) \ M(Bool, enable_software_prefetch_in_aggregation, true, "Enable use of software prefetch in aggregation", 0) \ M(Bool, allow_aggregate_partitions_independently, false, "Enable independent aggregation of partitions on separate threads when partition key suits group by key. Beneficial when number of partitions close to number of cores and partitions have roughly the same size", 0) \ M(Bool, force_aggregate_partitions_independently, false, "Force the use of optimization when it is applicable, but heuristics decided not to use it", 0) \ diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 075e79c96f0..590ee0cbbc5 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -31,6 +31,7 @@ class ASTStorage; M(StreamingHandleErrorMode, kafka_handle_error_mode, StreamingHandleErrorMode::DEFAULT, "How to handle errors for Kafka engine. Possible values: default (throw an exception after rabbitmq_skip_broken_messages broken messages), stream (save broken messages and errors in virtual columns _raw_message, _error).", 0) \ M(Bool, kafka_commit_on_select, false, "Commit messages when select query is made", 0) \ M(UInt64, kafka_max_rows_per_message, 1, "The maximum number of rows produced in one kafka message for row-based formats.", 0) \ + M(String, keeper_path, "", "TODO(antaljanosbenjamin)", 0) \ #define OBSOLETE_KAFKA_SETTINGS(M, ALIAS) \ MAKE_OBSOLETE(M, Char, kafka_row_delimiter, '\0') \ diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 723399bf206..361b17db2bd 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -76,6 +76,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int QUERY_NOT_ALLOWED; + extern const int SUPPORT_IS_DISABLED; } struct StorageKafkaInterceptors @@ -992,6 +993,7 @@ void registerStorageKafka(StorageFactory & factory) CHECK_KAFKA_STORAGE_ARGUMENT(15, kafka_handle_error_mode, 0) CHECK_KAFKA_STORAGE_ARGUMENT(16, kafka_commit_on_select, 0) CHECK_KAFKA_STORAGE_ARGUMENT(17, kafka_max_rows_per_message, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(18, keeper_path, 0) } #undef CHECK_KAFKA_STORAGE_ARGUMENT @@ -1040,6 +1042,13 @@ void registerStorageKafka(StorageFactory & factory) "See https://clickhouse.com/docs/en/engines/table-engines/integrations/kafka/#configuration"); } + if (kafka_settings->keeper_path.changed && !args.getLocalContext()->getSettingsRef().allow_experimental_kafka_store_offsets_in_keeper){ + + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Storing the Kafka offsets in Keeper is experimental. " + "Set `allow_experimental_kafka_store_offsets_in_keeper` setting to enable it"); + } + return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); }; From ba55c09fbced1b1cf268e4ee55ad0d1d2603f493 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 15 Aug 2023 11:21:44 +0000 Subject: [PATCH 003/644] Add `KafkaStorage2` --- src/Storages/Kafka/KafkaConsumer2.cpp | 530 ++++++++++++++++ src/Storages/Kafka/KafkaConsumer2.h | 116 ++++ src/Storages/Kafka/KafkaSettings.h | 2 +- src/Storages/Kafka/KafkaSource2.cpp | 303 +++++++++ src/Storages/Kafka/KafkaSource2.h | 64 ++ src/Storages/Kafka/StorageKafka.cpp | 340 +--------- src/Storages/Kafka/StorageKafka.h | 6 +- src/Storages/Kafka/StorageKafka2.cpp | 722 ++++++++++++++++++++++ src/Storages/Kafka/StorageKafka2.h | 149 +++++ src/Storages/Kafka/StorageKafkaCommon.cpp | 386 ++++++++++++ src/Storages/Kafka/StorageKafkaCommon.h | 48 ++ 11 files changed, 2328 insertions(+), 338 deletions(-) create mode 100644 src/Storages/Kafka/KafkaConsumer2.cpp create mode 100644 src/Storages/Kafka/KafkaConsumer2.h create mode 100644 src/Storages/Kafka/KafkaSource2.cpp create mode 100644 src/Storages/Kafka/KafkaSource2.h create mode 100644 src/Storages/Kafka/StorageKafka2.cpp create mode 100644 src/Storages/Kafka/StorageKafka2.h create mode 100644 src/Storages/Kafka/StorageKafkaCommon.cpp create mode 100644 src/Storages/Kafka/StorageKafkaCommon.h diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp new file mode 100644 index 00000000000..ec32248af46 --- /dev/null +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -0,0 +1,530 @@ +// Needs to go first because its partial specialization of fmt::formatter +// should be defined before any instantiation +#include + +#include +#include + +#include + +#include +#include +#include + +#include +#include + +namespace CurrentMetrics +{ + extern const Metric KafkaAssignedPartitions; + extern const Metric KafkaConsumersWithAssignment; +} + +namespace ProfileEvents +{ + extern const Event KafkaRebalanceRevocations; + extern const Event KafkaRebalanceAssignments; + extern const Event KafkaRebalanceErrors; + extern const Event KafkaMessagesPolled; + extern const Event KafkaCommitFailures; + extern const Event KafkaCommits; + extern const Event KafkaConsumerErrors; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_COMMIT_OFFSET; +} + +using namespace std::chrono_literals; +const auto MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS = 15000; +const std::size_t POLL_TIMEOUT_WO_ASSIGNMENT_MS = 50; +const auto DRAIN_TIMEOUT_MS = 5000ms; + + +KafkaConsumer2::KafkaConsumer2( + ConsumerPtr consumer_, + Poco::Logger * log_, + size_t max_batch_size, + size_t poll_timeout_, + bool intermediate_commit_, + const std::atomic & stopped_, + const Names & _topics) + : consumer(consumer_) + , log(log_) + , batch_size(max_batch_size) + , poll_timeout(poll_timeout_) + , intermediate_commit(intermediate_commit_) + , stopped(stopped_) + , current(messages.begin()) + , topics(_topics) +{ + // called (synchronously, during poll) when we enter the consumer group + consumer->set_assignment_callback([this](const cppkafka::TopicPartitionList & topic_partitions) + { + CurrentMetrics::add(CurrentMetrics::KafkaAssignedPartitions, topic_partitions.size()); + ProfileEvents::increment(ProfileEvents::KafkaRebalanceAssignments); + + if (topic_partitions.empty()) + { + LOG_INFO(log, "Got empty assignment: Not enough partitions in the topic for all consumers?"); + } + else + { + LOG_TRACE(log, "Topics/partitions assigned: {}", topic_partitions); + CurrentMetrics::add(CurrentMetrics::KafkaConsumersWithAssignment, 1); + } + + assignment = topic_partitions; + }); + + // called (synchronously, during poll) when we leave the consumer group + consumer->set_revocation_callback([this](const cppkafka::TopicPartitionList & topic_partitions) + { + CurrentMetrics::sub(CurrentMetrics::KafkaAssignedPartitions, topic_partitions.size()); + ProfileEvents::increment(ProfileEvents::KafkaRebalanceRevocations); + + // Rebalance is happening now, and now we have a chance to finish the work + // with topics/partitions we were working with before rebalance + LOG_TRACE(log, "Rebalance initiated. Revoking partitions: {}", topic_partitions); + + if (!topic_partitions.empty()) + { + CurrentMetrics::sub(CurrentMetrics::KafkaConsumersWithAssignment, 1); + } + + // we can not flush data to target from that point (it is pulled, not pushed) + // so the best we can now it to + // 1) repeat last commit in sync mode (async could be still in queue, we need to be sure is is properly committed before rebalance) + // 2) stop / brake the current reading: + // * clean buffered non-commited messages + // * set flag / flush + + cleanUnprocessed(); + + stalled_status = REBALANCE_HAPPENED; + assignment.reset(); + waited_for_assignment = 0; + + // for now we use slower (but reliable) sync commit in main loop, so no need to repeat + // try + // { + // consumer->commit(); + // } + // catch (cppkafka::HandleException & e) + // { + // LOG_WARNING(log, "Commit error: {}", e.what()); + // } + }); + + consumer->set_rebalance_error_callback([this](cppkafka::Error err) + { + LOG_ERROR(log, "Rebalance error: {}", err); + ProfileEvents::increment(ProfileEvents::KafkaRebalanceErrors); + }); +} + +KafkaConsumer2::~KafkaConsumer2() +{ + try + { + if (!consumer->get_subscription().empty()) + { + try + { + consumer->unsubscribe(); + } + catch (const cppkafka::HandleException & e) + { + LOG_ERROR(log, "Error during unsubscribe: {}", e.what()); + } + drain(); + } + } + catch (const cppkafka::HandleException & e) + { + LOG_ERROR(log, "Error while destructing consumer: {}", e.what()); + } + +} + +// Needed to drain rest of the messages / queued callback calls from the consumer +// after unsubscribe, otherwise consumer will hang on destruction +// see https://github.com/edenhill/librdkafka/issues/2077 +// https://github.com/confluentinc/confluent-kafka-go/issues/189 etc. +void KafkaConsumer2::drain() +{ + auto start_time = std::chrono::steady_clock::now(); + cppkafka::Error last_error(RD_KAFKA_RESP_ERR_NO_ERROR); + + while (true) + { + auto msg = consumer->poll(100ms); + if (!msg) + break; + + auto error = msg.get_error(); + + if (error) + { + if (msg.is_eof() || error == last_error) + { + break; + } + else + { + LOG_ERROR(log, "Error during draining: {}", error); + } + } + + // i don't stop draining on first error, + // only if it repeats once again sequentially + last_error = error; + + auto ts = std::chrono::steady_clock::now(); + if (std::chrono::duration_cast(ts-start_time) > DRAIN_TIMEOUT_MS) + { + LOG_ERROR(log, "Timeout during draining."); + break; + } + } +} + + +void KafkaConsumer2::commit() +{ + auto print_offsets = [this] (const char * prefix, const cppkafka::TopicPartitionList & offsets) + { + for (const auto & topic_part : offsets) + { + auto print_special_offset = [&topic_part] + { + switch (topic_part.get_offset()) + { + case cppkafka::TopicPartition::OFFSET_BEGINNING: return "BEGINNING"; + case cppkafka::TopicPartition::OFFSET_END: return "END"; + case cppkafka::TopicPartition::OFFSET_STORED: return "STORED"; + case cppkafka::TopicPartition::OFFSET_INVALID: return "INVALID"; + default: return ""; + } + }; + + if (topic_part.get_offset() < 0) + { + LOG_TRACE(log, "{} {} (topic: {}, partition: {})", prefix, print_special_offset(), topic_part.get_topic(), topic_part.get_partition()); + } + else + { + LOG_TRACE(log, "{} {} (topic: {}, partition: {})", prefix, topic_part.get_offset(), topic_part.get_topic(), topic_part.get_partition()); + } + } + }; + + print_offsets("Polled offset", consumer->get_offsets_position(consumer->get_assignment())); + + if (hasMorePolledMessages()) + { + LOG_WARNING(log, "Logical error. Non all polled messages were processed."); + } + + if (offsets_stored > 0) + { + // if we will do async commit here (which is faster) + // we may need to repeat commit in sync mode in revocation callback, + // but it seems like existing API doesn't allow us to to that + // in a controlled manner (i.e. we don't know the offsets to commit then) + + size_t max_retries = 5; + bool committed = false; + + while (!committed && max_retries > 0) + { + try + { + // See https://github.com/edenhill/librdkafka/issues/1470 + // broker may reject commit if during offsets.commit.timeout.ms (5000 by default), + // there were not enough replicas available for the __consumer_offsets topic. + // also some other temporary issues like client-server connectivity problems are possible + consumer->commit(); + committed = true; + print_offsets("Committed offset", consumer->get_offsets_committed(consumer->get_assignment())); + } + catch (const cppkafka::HandleException & e) + { + // If there were actually no offsets to commit, return. Retrying won't solve + // anything here + if (e.get_error() == RD_KAFKA_RESP_ERR__NO_OFFSET) + committed = true; + else + LOG_ERROR(log, "Exception during commit attempt: {}", e.what()); + } + --max_retries; + } + + if (!committed) + { + // TODO: insert atomicity / transactions is needed here (possibility to rollback, on 2 phase commits) + ProfileEvents::increment(ProfileEvents::KafkaCommitFailures); + throw Exception(ErrorCodes::CANNOT_COMMIT_OFFSET, + "All commit attempts failed. Last block was already written to target table(s), " + "but was not committed to Kafka."); + } + else + { + ProfileEvents::increment(ProfileEvents::KafkaCommits); + } + + } + else + { + LOG_TRACE(log, "Nothing to commit."); + } + + offsets_stored = 0; +} + +void KafkaConsumer2::subscribe() +{ + LOG_TRACE(log, "Already subscribed to topics: [{}]", boost::algorithm::join(consumer->get_subscription(), ", ")); + + if (assignment.has_value()) + { + LOG_TRACE(log, "Already assigned to: {}", assignment.value()); + } + else + { + LOG_TRACE(log, "No assignment"); + } + + + size_t max_retries = 5; + + while (consumer->get_subscription().empty()) + { + --max_retries; + try + { + consumer->subscribe(topics); + // FIXME: if we failed to receive "subscribe" response while polling and destroy consumer now, then we may hang up. + // see https://github.com/edenhill/librdkafka/issues/2077 + } + catch (cppkafka::HandleException & e) + { + if (max_retries > 0 && e.get_error() == RD_KAFKA_RESP_ERR__TIMED_OUT) + continue; + throw; + } + } + + cleanUnprocessed(); + + // we can reset any flags (except of CONSUMER_STOPPED) before attempt of reading new block of data + if (stalled_status != CONSUMER_STOPPED) + stalled_status = NO_MESSAGES_RETURNED; +} + +void KafkaConsumer2::cleanUnprocessed() +{ + messages.clear(); + current = messages.begin(); + offsets_stored = 0; +} + +void KafkaConsumer2::unsubscribe() +{ + LOG_TRACE(log, "Re-joining claimed consumer after failure"); + cleanUnprocessed(); + + // it should not raise exception as used in destructor + try + { + // From docs: Any previous subscription will be unassigned and unsubscribed first. + consumer->subscribe(topics); + + // I wanted to avoid explicit unsubscribe as it requires draining the messages + // to close the consumer safely after unsubscribe + // see https://github.com/edenhill/librdkafka/issues/2077 + // https://github.com/confluentinc/confluent-kafka-go/issues/189 etc. + } + catch (const cppkafka::HandleException & e) + { + LOG_ERROR(log, "Exception from KafkaConsumer2::unsubscribe: {}", e.what()); + } + +} + + +void KafkaConsumer2::resetToLastCommitted(const char * msg) +{ + if (!assignment.has_value() || assignment->empty()) + { + LOG_TRACE(log, "Not assignned. Can't reset to last committed position."); + return; + } + auto committed_offset = consumer->get_offsets_committed(consumer->get_assignment()); + consumer->assign(committed_offset); + LOG_TRACE(log, "{} Returned to committed position: {}", msg, committed_offset); +} + +// it do the poll when needed +ReadBufferPtr KafkaConsumer2::consume() +{ + resetIfStopped(); + + if (polledDataUnusable()) + return nullptr; + + if (hasMorePolledMessages()) + return getNextMessage(); + + if (intermediate_commit) + commit(); + + while (true) + { + stalled_status = NO_MESSAGES_RETURNED; + + // we already wait enough for assignment in the past, + // let's make polls shorter and not block other consumer + // which can work successfully in parallel + // POLL_TIMEOUT_WO_ASSIGNMENT_MS (50ms) is 100% enough just to check if we got assignment + // (see https://github.com/ClickHouse/ClickHouse/issues/11218) + auto actual_poll_timeout_ms = (waited_for_assignment >= MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS) + ? std::min(POLL_TIMEOUT_WO_ASSIGNMENT_MS,poll_timeout) + : poll_timeout; + + /// Don't drop old messages immediately, since we may need them for virtual columns. + auto new_messages = consumer->poll_batch(batch_size, + std::chrono::milliseconds(actual_poll_timeout_ms)); + + resetIfStopped(); + if (stalled_status == CONSUMER_STOPPED) + { + return nullptr; + } + else if (stalled_status == REBALANCE_HAPPENED) + { + if (!new_messages.empty()) + { + // we have polled something just after rebalance. + // we will not use current batch, so we need to return to last committed position + // otherwise we will continue polling from that position + resetToLastCommitted("Rewind last poll after rebalance."); + } + return nullptr; + } + + if (new_messages.empty()) + { + // While we wait for an assignment after subscription, we'll poll zero messages anyway. + // If we're doing a manual select then it's better to get something after a wait, then immediate nothing. + if (!assignment.has_value()) + { + waited_for_assignment += poll_timeout; // slightly innaccurate, but rough calculation is ok. + if (waited_for_assignment < MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS) + { + continue; + } + else + { + LOG_WARNING(log, "Can't get assignment. Will keep trying."); + stalled_status = NO_ASSIGNMENT; + return nullptr; + } + } + else if (assignment->empty()) + { + LOG_TRACE(log, "Empty assignment."); + return nullptr; + } + else + { + LOG_TRACE(log, "Stalled"); + return nullptr; + } + } + else + { + messages = std::move(new_messages); + current = messages.begin(); + LOG_TRACE(log, "Polled batch of {} messages. Offsets position: {}", + messages.size(), consumer->get_offsets_position(consumer->get_assignment())); + break; + } + } + + filterMessageErrors(); + if (current == messages.end()) + { + LOG_ERROR(log, "Only errors left"); + stalled_status = ERRORS_RETURNED; + return nullptr; + } + + ProfileEvents::increment(ProfileEvents::KafkaMessagesPolled, messages.size()); + + stalled_status = NOT_STALLED; + return getNextMessage(); +} + +ReadBufferPtr KafkaConsumer2::getNextMessage() +{ + if (current == messages.end()) + return nullptr; + + const auto * data = current->get_payload().get_data(); + size_t size = current->get_payload().get_size(); + ++current; + + if (data) + return std::make_shared(data, size); + + return getNextMessage(); +} + +size_t KafkaConsumer2::filterMessageErrors() +{ + assert(current == messages.begin()); + + size_t skipped = std::erase_if(messages, [this](auto & message) + { + if (auto error = message.get_error()) + { + ProfileEvents::increment(ProfileEvents::KafkaConsumerErrors); + LOG_ERROR(log, "Consumer error: {}", error); + return true; + } + return false; + }); + + if (skipped) + LOG_ERROR(log, "There were {} messages with an error", skipped); + + return skipped; +} + +void KafkaConsumer2::resetIfStopped() +{ + // we can react on stop only during fetching data + // after block is formed (i.e. during copying data to MV / committing) we ignore stop attempts + if (stopped) + { + stalled_status = CONSUMER_STOPPED; + cleanUnprocessed(); + } +} + + +void KafkaConsumer2::storeLastReadMessageOffset() +{ + if (!isStalled()) + { + consumer->store_offset(*(current - 1)); + ++offsets_stored; + } +} + +} diff --git a/src/Storages/Kafka/KafkaConsumer2.h b/src/Storages/Kafka/KafkaConsumer2.h new file mode 100644 index 00000000000..6562a65a9b0 --- /dev/null +++ b/src/Storages/Kafka/KafkaConsumer2.h @@ -0,0 +1,116 @@ +#pragma once + +#include +#include +#include + +#include +#include + +namespace CurrentMetrics +{ + extern const Metric KafkaConsumers; +} + +namespace Poco +{ + class Logger; +} + +namespace DB +{ + +using ConsumerPtr = std::shared_ptr; + +class KafkaConsumer2 +{ +public: + KafkaConsumer2( + ConsumerPtr consumer_, + Poco::Logger * log_, + size_t max_batch_size, + size_t poll_timeout_, + bool intermediate_commit_, + const std::atomic & stopped_, + const Names & _topics + ); + + ~KafkaConsumer2(); + void commit(); // Commit all processed messages. + void subscribe(); // Subscribe internal consumer to topics. + void unsubscribe(); // Unsubscribe internal consumer in case of failure. + + auto pollTimeout() const { return poll_timeout; } + + inline bool hasMorePolledMessages() const + { + return (stalled_status == NOT_STALLED) && (current != messages.end()); + } + + inline bool polledDataUnusable() const + { + return (stalled_status != NOT_STALLED) && (stalled_status != NO_MESSAGES_RETURNED); + } + + inline bool isStalled() const { return stalled_status != NOT_STALLED; } + + void storeLastReadMessageOffset(); + void resetToLastCommitted(const char * msg); + + /// Polls batch of messages from Kafka and returns read buffer containing the next message or + /// nullptr when there are no messages to process. + ReadBufferPtr consume(); + + // Return values for the message that's being read. + String currentTopic() const { return current[-1].get_topic(); } + String currentKey() const { return current[-1].get_key(); } + auto currentOffset() const { return current[-1].get_offset(); } + auto currentPartition() const { return current[-1].get_partition(); } + auto currentTimestamp() const { return current[-1].get_timestamp(); } + const auto & currentHeaderList() const { return current[-1].get_header_list(); } + String currentPayload() const { return current[-1].get_payload(); } + +private: + using Messages = std::vector; + CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaConsumers}; + + enum StalledStatus + { + NOT_STALLED, + NO_MESSAGES_RETURNED, + REBALANCE_HAPPENED, + CONSUMER_STOPPED, + NO_ASSIGNMENT, + ERRORS_RETURNED + }; + + ConsumerPtr consumer; + Poco::Logger * log; + const size_t batch_size = 1; + const size_t poll_timeout = 0; + size_t offsets_stored = 0; + + StalledStatus stalled_status = NO_MESSAGES_RETURNED; + + bool intermediate_commit = true; + size_t waited_for_assignment = 0; + + const std::atomic & stopped; + + // order is important, need to be destructed before consumer + Messages messages; + Messages::const_iterator current; + + // order is important, need to be destructed before consumer + std::optional assignment; + const Names topics; + + void drain(); + void cleanUnprocessed(); + void resetIfStopped(); + /// Return number of messages with an error. + size_t filterMessageErrors(); + ReadBufferPtr getNextMessage(); +}; + +} diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 590ee0cbbc5..748090165d7 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -31,7 +31,7 @@ class ASTStorage; M(StreamingHandleErrorMode, kafka_handle_error_mode, StreamingHandleErrorMode::DEFAULT, "How to handle errors for Kafka engine. Possible values: default (throw an exception after rabbitmq_skip_broken_messages broken messages), stream (save broken messages and errors in virtual columns _raw_message, _error).", 0) \ M(Bool, kafka_commit_on_select, false, "Commit messages when select query is made", 0) \ M(UInt64, kafka_max_rows_per_message, 1, "The maximum number of rows produced in one kafka message for row-based formats.", 0) \ - M(String, keeper_path, "", "TODO(antaljanosbenjamin)", 0) \ + M(String, kafka_keeper_path, "", "TODO(antaljanosbenjamin)", 0) \ #define OBSOLETE_KAFKA_SETTINGS(M, ALIAS) \ MAKE_OBSOLETE(M, Char, kafka_row_delimiter, '\0') \ diff --git a/src/Storages/Kafka/KafkaSource2.cpp b/src/Storages/Kafka/KafkaSource2.cpp new file mode 100644 index 00000000000..3e14c57e8e2 --- /dev/null +++ b/src/Storages/Kafka/KafkaSource2.cpp @@ -0,0 +1,303 @@ +#include + +#include +#include +#include +#include +#include +#include + +#include + +namespace ProfileEvents +{ + extern const Event KafkaMessagesRead; + extern const Event KafkaMessagesFailed; + extern const Event KafkaRowsRead; + extern const Event KafkaRowsRejected; +} + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +// with default poll timeout (500ms) it will give about 5 sec delay for doing 10 retries +// when selecting from empty topic +const auto MAX_FAILED_POLL_ATTEMPTS = 10; + +KafkaSource2::KafkaSource2( + StorageKafka2 & storage_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + const Names & columns, + Poco::Logger * log_, + size_t max_block_size_, + bool commit_in_suffix_) + : ISource(storage_snapshot_->getSampleBlockForColumns(columns)) + , storage(storage_) + , storage_snapshot(storage_snapshot_) + , context(context_) + , column_names(columns) + , log(log_) + , max_block_size(max_block_size_) + , commit_in_suffix(commit_in_suffix_) + , non_virtual_header(storage_snapshot->metadata->getSampleBlockNonMaterialized()) + , virtual_header(storage_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames())) + , handle_error_mode(storage.getHandleKafkaErrorMode()) +{ +} + +KafkaSource2::~KafkaSource2() +{ + if (!consumer) + return; + + if (broken) + consumer->unsubscribe(); + + storage.pushConsumer(consumer); +} + +bool KafkaSource2::checkTimeLimit() const +{ + if (max_execution_time != 0) + { + auto elapsed_ns = total_stopwatch.elapsed(); + + if (elapsed_ns > static_cast(max_execution_time.totalMicroseconds()) * 1000) + return false; + } + + return true; +} + +Chunk KafkaSource2::generateImpl() +{ + if (!consumer) + { + auto timeout = std::chrono::milliseconds(context->getSettingsRef().kafka_max_wait_ms.totalMilliseconds()); + consumer = storage.popConsumer(timeout); + + if (!consumer) + return {}; + + consumer->subscribe(); + + broken = true; + } + + if (is_finished) + return {}; + + is_finished = true; + // now it's one-time usage InputStream + // one block of the needed size (or with desired flush timeout) is formed in one internal iteration + // otherwise external iteration will reuse that and logic will became even more fuzzy + MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); + + auto put_error_to_stream = handle_error_mode == HandleKafkaErrorMode::STREAM; + + EmptyReadBuffer empty_buf; + auto input_format = FormatFactory::instance().getInput( + storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size, std::nullopt, 1); + + std::optional exception_message; + size_t total_rows = 0; + size_t failed_poll_attempts = 0; + + auto on_error = [&](const MutableColumns & result_columns, Exception & e) + { + ProfileEvents::increment(ProfileEvents::KafkaMessagesFailed); + + if (put_error_to_stream) + { + exception_message = e.message(); + for (const auto & column : result_columns) + { + // read_kafka_message could already push some rows to result_columns + // before exception, we need to fix it. + auto cur_rows = column->size(); + if (cur_rows > total_rows) + column->popBack(cur_rows - total_rows); + + // all data columns will get default value in case of error + column->insertDefault(); + } + + return 1; + } + else + { + e.addMessage("while parsing Kafka message (topic: {}, partition: {}, offset: {})'", + consumer->currentTopic(), consumer->currentPartition(), consumer->currentOffset()); + throw std::move(e); + } + }; + + StreamingFormatExecutor executor(non_virtual_header, input_format, std::move(on_error)); + + while (true) + { + size_t new_rows = 0; + exception_message.reset(); + if (auto buf = consumer->consume()) + { + ProfileEvents::increment(ProfileEvents::KafkaMessagesRead); + new_rows = executor.execute(*buf); + } + + if (new_rows) + { + // In read_kafka_message(), KafkaConsumer::nextImpl() + // will be called, that may make something unusable, i.e. clean + // KafkaConsumer::messages, which is accessed from + // KafkaConsumer::currentTopic() (and other helpers). + if (consumer->isStalled()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Polled messages became unusable"); + + ProfileEvents::increment(ProfileEvents::KafkaRowsRead, new_rows); + + consumer->storeLastReadMessageOffset(); + + auto topic = consumer->currentTopic(); + auto key = consumer->currentKey(); + auto offset = consumer->currentOffset(); + auto partition = consumer->currentPartition(); + auto timestamp_raw = consumer->currentTimestamp(); + auto header_list = consumer->currentHeaderList(); + + Array headers_names; + Array headers_values; + + if (!header_list.empty()) + { + headers_names.reserve(header_list.size()); + headers_values.reserve(header_list.size()); + for (const auto & header : header_list) + { + headers_names.emplace_back(header.get_name()); + headers_values.emplace_back(static_cast(header.get_value())); + } + } + + for (size_t i = 0; i < new_rows; ++i) + { + virtual_columns[0]->insert(topic); + virtual_columns[1]->insert(key); + virtual_columns[2]->insert(offset); + virtual_columns[3]->insert(partition); + if (timestamp_raw) + { + auto ts = timestamp_raw->get_timestamp(); + virtual_columns[4]->insert(std::chrono::duration_cast(ts).count()); + virtual_columns[5]->insert(DecimalField(std::chrono::duration_cast(ts).count(),3)); + } + else + { + virtual_columns[4]->insertDefault(); + virtual_columns[5]->insertDefault(); + } + virtual_columns[6]->insert(headers_names); + virtual_columns[7]->insert(headers_values); + if (put_error_to_stream) + { + if (exception_message) + { + auto payload = consumer->currentPayload(); + virtual_columns[8]->insert(payload); + virtual_columns[9]->insert(*exception_message); + } + else + { + virtual_columns[8]->insertDefault(); + virtual_columns[9]->insertDefault(); + } + } + } + + total_rows = total_rows + new_rows; + } + else if (consumer->polledDataUnusable()) + { + break; + } + else if (consumer->isStalled()) + { + ++failed_poll_attempts; + } + else + { + // We came here in case of tombstone (or sometimes zero-length) messages, and it is not something abnormal + // TODO: it seems like in case of put_error_to_stream=true we may need to process those differently + // currently we just skip them with note in logs. + consumer->storeLastReadMessageOffset(); + LOG_DEBUG(log, "Parsing of message (topic: {}, partition: {}, offset: {}) return no rows.", consumer->currentTopic(), consumer->currentPartition(), consumer->currentOffset()); + } + + if (!consumer->hasMorePolledMessages() + && (total_rows >= max_block_size || !checkTimeLimit() || failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS)) + { + break; + } + } + + if (total_rows == 0) + { + return {}; + } + else if (consumer->polledDataUnusable()) + { + // the rows were counted already before by KafkaRowsRead, + // so let's count the rows we ignore separately + // (they will be retried after the rebalance) + ProfileEvents::increment(ProfileEvents::KafkaRowsRejected, total_rows); + return {}; + } + + /// MATERIALIZED columns can be added here, but I think + // they are not needed here: + // and it's misleading to use them here, + // as columns 'materialized' that way stays 'ephemeral' + // i.e. will not be stored anythere + // If needed any extra columns can be added using DEFAULT they can be added at MV level if needed. + + auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns()); + auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); + + for (const auto & column : virtual_block.getColumnsWithTypeAndName()) + result_block.insert(column); + + auto converting_dag = ActionsDAG::makeConvertingActions( + result_block.cloneEmpty().getColumnsWithTypeAndName(), + getPort().getHeader().getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto converting_actions = std::make_shared(std::move(converting_dag)); + converting_actions->execute(result_block); + + return Chunk(result_block.getColumns(), result_block.rows()); +} + +Chunk KafkaSource2::generate() +{ + auto chunk = generateImpl(); + if (!chunk && commit_in_suffix) + commit(); + + return chunk; +} + +void KafkaSource2::commit() +{ + if (!consumer) + return; + + consumer->commit(); + + broken = false; +} + +} diff --git a/src/Storages/Kafka/KafkaSource2.h b/src/Storages/Kafka/KafkaSource2.h new file mode 100644 index 00000000000..0a49001a686 --- /dev/null +++ b/src/Storages/Kafka/KafkaSource2.h @@ -0,0 +1,64 @@ +#pragma once + +#include + +#include +#include +#include + + +namespace Poco +{ + class Logger; +} +namespace DB +{ + +class KafkaSource2 : public ISource +{ +public: + KafkaSource2( + StorageKafka2 & storage_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + const Names & columns, + Poco::Logger * log_, + size_t max_block_size_, + bool commit_in_suffix = false); + ~KafkaSource2() override; + + String getName() const override { return storage.getName(); } + + Chunk generate() override; + + void commit(); + bool isStalled() const { return !consumer || consumer->isStalled(); } + + void setTimeLimit(Poco::Timespan max_execution_time_) { max_execution_time = max_execution_time_; } + +private: + StorageKafka2 & storage; + StorageSnapshotPtr storage_snapshot; + ContextPtr context; + Names column_names; + Poco::Logger * log; + UInt64 max_block_size; + + KafkaConsumer2Ptr consumer; + bool broken = true; + bool is_finished = false; + bool commit_in_suffix; + + const Block non_virtual_header; + const Block virtual_header; + const HandleKafkaErrorMode handle_error_mode; + + Poco::Timespan max_execution_time = 0; + Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; + + bool checkTimeLimit() const; + + Chunk generateImpl(); +}; + +} diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 361b17db2bd..0c9285e2ef6 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -52,7 +53,6 @@ namespace CurrentMetrics { - extern const Metric KafkaLibrdkafkaThreads; extern const Metric KafkaBackgroundReads; extern const Metric KafkaConsumersInUse; extern const Metric KafkaWrites; @@ -73,175 +73,14 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int QUERY_NOT_ALLOWED; - extern const int SUPPORT_IS_DISABLED; } -struct StorageKafkaInterceptors -{ - static rd_kafka_resp_err_t rdKafkaOnThreadStart(rd_kafka_t *, rd_kafka_thread_type_t thread_type, const char *, void * ctx) - { - StorageKafka * self = reinterpret_cast(ctx); - CurrentMetrics::add(CurrentMetrics::KafkaLibrdkafkaThreads, 1); - - const auto & storage_id = self->getStorageID(); - const auto & table = storage_id.getTableName(); - - switch (thread_type) - { - case RD_KAFKA_THREAD_MAIN: - setThreadName(("rdk:m/" + table.substr(0, 9)).c_str()); - break; - case RD_KAFKA_THREAD_BACKGROUND: - setThreadName(("rdk:bg/" + table.substr(0, 8)).c_str()); - break; - case RD_KAFKA_THREAD_BROKER: - setThreadName(("rdk:b/" + table.substr(0, 9)).c_str()); - break; - } - - /// Create ThreadStatus to track memory allocations from librdkafka threads. - // - /// And store them in a separate list (thread_statuses) to make sure that they will be destroyed, - /// regardless how librdkafka calls the hooks. - /// But this can trigger use-after-free if librdkafka will not destroy threads after rd_kafka_wait_destroyed() - auto thread_status = std::make_shared(); - std::lock_guard lock(self->thread_statuses_mutex); - self->thread_statuses.emplace_back(std::move(thread_status)); - - return RD_KAFKA_RESP_ERR_NO_ERROR; - } - static rd_kafka_resp_err_t rdKafkaOnThreadExit(rd_kafka_t *, rd_kafka_thread_type_t, const char *, void * ctx) - { - StorageKafka * self = reinterpret_cast(ctx); - CurrentMetrics::sub(CurrentMetrics::KafkaLibrdkafkaThreads, 1); - - std::lock_guard lock(self->thread_statuses_mutex); - const auto it = std::find_if(self->thread_statuses.begin(), self->thread_statuses.end(), [](const auto & thread_status_ptr) - { - return thread_status_ptr.get() == current_thread; - }); - if (it == self->thread_statuses.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread status for this librdkafka thread."); - - self->thread_statuses.erase(it); - - return RD_KAFKA_RESP_ERR_NO_ERROR; - } - - static rd_kafka_resp_err_t rdKafkaOnNew(rd_kafka_t * rk, const rd_kafka_conf_t *, void * ctx, char * /*errstr*/, size_t /*errstr_size*/) - { - StorageKafka * self = reinterpret_cast(ctx); - rd_kafka_resp_err_t status; - - status = rd_kafka_interceptor_add_on_thread_start(rk, "init-thread", rdKafkaOnThreadStart, ctx); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - { - LOG_ERROR(self->log, "Cannot set on thread start interceptor due to {} error", status); - return status; - } - - status = rd_kafka_interceptor_add_on_thread_exit(rk, "exit-thread", rdKafkaOnThreadExit, ctx); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - LOG_ERROR(self->log, "Cannot set on thread exit interceptor due to {} error", status); - - return status; - } - - static rd_kafka_resp_err_t rdKafkaOnConfDup(rd_kafka_conf_t * new_conf, const rd_kafka_conf_t * /*old_conf*/, size_t /*filter_cnt*/, const char ** /*filter*/, void * ctx) - { - StorageKafka * self = reinterpret_cast(ctx); - rd_kafka_resp_err_t status; - - // cppkafka copies configuration multiple times - status = rd_kafka_conf_interceptor_add_on_conf_dup(new_conf, "init", rdKafkaOnConfDup, ctx); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - { - LOG_ERROR(self->log, "Cannot set on conf dup interceptor due to {} error", status); - return status; - } - - status = rd_kafka_conf_interceptor_add_on_new(new_conf, "init", rdKafkaOnNew, ctx); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - LOG_ERROR(self->log, "Cannot set on conf new interceptor due to {} error", status); - - return status; - } -}; - namespace { const auto RESCHEDULE_MS = 500; const auto CLEANUP_TIMEOUT_MS = 3000; const auto MAX_THREAD_WORK_DURATION_MS = 60000; // once per minute leave do reschedule (we can't lock threads in pool forever) - - const String CONFIG_KAFKA_TAG = "kafka"; - const String CONFIG_KAFKA_TOPIC_TAG = "kafka_topic"; - const String CONFIG_NAME_TAG = "name"; - - /// Read server configuration into cppkafka configuration, used by global configuration and by legacy per-topic configuration - void loadFromConfig(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) - { - /// Read all tags one level below - Poco::Util::AbstractConfiguration::Keys tags; - config.keys(config_prefix, tags); - - for (const auto & tag : tags) - { - if (tag.starts_with(CONFIG_KAFKA_TOPIC_TAG)) /// multiple occurrences given as "kafka_topic", "kafka_topic[1]", etc. - continue; /// used by new per-topic configuration, ignore - - const String setting_path = config_prefix + "." + tag; - const String setting_value = config.getString(setting_path); - - /// "log_level" has valid underscore, the remaining librdkafka setting use dot.separated.format which isn't acceptable for XML. - /// See https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md - const String setting_name_in_kafka_config = (tag == "log_level") ? tag : boost::replace_all_copy(tag, "_", "."); - kafka_config.set(setting_name_in_kafka_config, setting_value); - } - } - - /// Read server configuration into cppkafa configuration, used by new per-topic configuration - void loadTopicConfig(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const String & topic) - { - /// Read all tags one level below - Poco::Util::AbstractConfiguration::Keys tags; - config.keys(config_prefix, tags); - - for (const auto & tag : tags) - { - /// Only consider tag . Multiple occurrences given as "kafka_topic", "kafka_topic[1]", etc. - if (!tag.starts_with(CONFIG_KAFKA_TOPIC_TAG)) - continue; - - /// Read topic name between ... - const String kafka_topic_path = config_prefix + "." + tag; - const String kafpa_topic_name_path = kafka_topic_path + "." + CONFIG_NAME_TAG; - - const String topic_name = config.getString(kafpa_topic_name_path); - if (topic_name == topic) - { - /// Found it! Now read the per-topic configuration into cppkafka. - Poco::Util::AbstractConfiguration::Keys inner_tags; - config.keys(kafka_topic_path, inner_tags); - for (const auto & inner_tag : inner_tags) - { - if (inner_tag == CONFIG_NAME_TAG) - continue; // ignore - - /// "log_level" has valid underscore, the remaining librdkafka setting use dot.separated.format which isn't acceptable for XML. - /// See https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md - const String setting_path = kafka_topic_path + "." + inner_tag; - const String setting_value = config.getString(setting_path); - - const String setting_name_in_kafka_config = (inner_tag == "log_level") ? inner_tag : boost::replace_all_copy(inner_tag, "_", "."); - kafka_config.set(setting_name_in_kafka_config, setting_value); - } - } - } - } } StorageKafka::StorageKafka( @@ -575,8 +414,8 @@ size_t StorageKafka::getPollTimeoutMillisecond() const String StorageKafka::getConfigPrefix() const { if (!collection_name.empty()) - return "named_collections." + collection_name + "." + CONFIG_KAFKA_TAG; /// Add one more level to separate librdkafka configuration. - return CONFIG_KAFKA_TAG; + return "named_collections." + collection_name + "." + String{KafkaConfigLoader::CONFIG_KAFKA_TAG}; /// Add one more level to separate librdkafka configuration. + return String{KafkaConfigLoader::CONFIG_KAFKA_TAG}; } void StorageKafka::updateConfiguration(cppkafka::Configuration & kafka_config, @@ -590,7 +429,7 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & kafka_config, const auto & config = getContext()->getConfigRef(); auto config_prefix = getConfigPrefix(); if (config.has(config_prefix)) - loadFromConfig(kafka_config, config, config_prefix); + KafkaConfigLoader::loadConfig(kafka_config, config, config_prefix); #if USE_KRB5 if (kafka_config.has_property("sasl.kerberos.kinit.cmd")) @@ -631,7 +470,7 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & kafka_config, { const auto topic_config_key = config_prefix + "_" + topic; if (config.has(topic_config_key)) - loadFromConfig(kafka_config, config, topic_config_key); + KafkaConfigLoader::loadConfig(kafka_config, config, topic_config_key); } // Update consumer topic-specific configuration (new syntax). Example with topics "football" and "baseball": @@ -651,7 +490,7 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & kafka_config, // Kafka-related is below . for (const auto & topic : topics) if (config.has(config_prefix)) - loadTopicConfig(kafka_config, config, config_prefix, topic); + KafkaConfigLoader::loadTopicConfig(kafka_config, config, config_prefix, topic); // No need to add any prefix, messages can be distinguished kafka_config.set_log_callback([this](cppkafka::KafkaHandleBase &, int level, const std::string & facility, const std::string & message) @@ -888,173 +727,6 @@ bool StorageKafka::streamToViews() return some_stream_is_stalled; } -void registerStorageKafka(StorageFactory & factory) -{ - auto creator_fn = [](const StorageFactory::Arguments & args) - { - ASTs & engine_args = args.engine_args; - size_t args_count = engine_args.size(); - const bool has_settings = args.storage_def->settings; - - auto kafka_settings = std::make_unique(); - String collection_name; - if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args, args.getLocalContext())) - { - for (const auto & setting : kafka_settings->all()) - { - const auto & setting_name = setting.getName(); - if (named_collection->has(setting_name)) - kafka_settings->set(setting_name, named_collection->get(setting_name)); - } - collection_name = assert_cast(args.engine_args[0].get())->name(); - } - - if (has_settings) - { - kafka_settings->loadFromQuery(*args.storage_def); - } - - // Check arguments and settings - #define CHECK_KAFKA_STORAGE_ARGUMENT(ARG_NUM, PAR_NAME, EVAL) \ - /* One of the four required arguments is not specified */ \ - if (args_count < (ARG_NUM) && (ARG_NUM) <= 4 && \ - !kafka_settings->PAR_NAME.changed) \ - { \ - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,\ - "Required parameter '{}' " \ - "for storage Kafka not specified", \ - #PAR_NAME); \ - } \ - if (args_count >= (ARG_NUM)) \ - { \ - /* The same argument is given in two places */ \ - if (has_settings && \ - kafka_settings->PAR_NAME.changed) \ - { \ - throw Exception(ErrorCodes::BAD_ARGUMENTS, \ - "The argument №{} of storage Kafka " \ - "and the parameter '{}' " \ - "in SETTINGS cannot be specified at the same time", \ - #ARG_NUM, #PAR_NAME); \ - } \ - /* move engine args to settings */ \ - else \ - { \ - if constexpr ((EVAL) == 1) \ - { \ - engine_args[(ARG_NUM)-1] = \ - evaluateConstantExpressionAsLiteral( \ - engine_args[(ARG_NUM)-1], \ - args.getLocalContext()); \ - } \ - if constexpr ((EVAL) == 2) \ - { \ - engine_args[(ARG_NUM)-1] = \ - evaluateConstantExpressionOrIdentifierAsLiteral( \ - engine_args[(ARG_NUM)-1], \ - args.getLocalContext()); \ - } \ - kafka_settings->PAR_NAME = \ - engine_args[(ARG_NUM)-1]->as().value; \ - } \ - } - - /** Arguments of engine is following: - * - Kafka broker list - * - List of topics - * - Group ID (may be a constant expression with a string result) - * - Message format (string) - * - Row delimiter - * - Schema (optional, if the format supports it) - * - Number of consumers - * - Max block size for background consumption - * - Skip (at least) unreadable messages number - * - Do intermediate commits when the batch consumed and handled - */ - - /* 0 = raw, 1 = evaluateConstantExpressionAsLiteral, 2=evaluateConstantExpressionOrIdentifierAsLiteral */ - /// In case of named collection we already validated the arguments. - if (collection_name.empty()) - { - CHECK_KAFKA_STORAGE_ARGUMENT(1, kafka_broker_list, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(2, kafka_topic_list, 1) - CHECK_KAFKA_STORAGE_ARGUMENT(3, kafka_group_name, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(4, kafka_format, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(5, kafka_row_delimiter, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(6, kafka_schema, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(7, kafka_num_consumers, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(10, kafka_commit_every_batch, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(11, kafka_client_id, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(12, kafka_poll_timeout_ms, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(13, kafka_flush_interval_ms, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(14, kafka_thread_per_consumer, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(15, kafka_handle_error_mode, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(16, kafka_commit_on_select, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(17, kafka_max_rows_per_message, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(18, keeper_path, 0) - } - - #undef CHECK_KAFKA_STORAGE_ARGUMENT - - auto num_consumers = kafka_settings->kafka_num_consumers.value; - auto max_consumers = std::max(getNumberOfPhysicalCPUCores(), 16); - - if (!args.getLocalContext()->getSettingsRef().kafka_disable_num_consumers_limit && num_consumers > max_consumers) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The number of consumers can not be bigger than {}. " - "A single consumer can read any number of partitions. " - "Extra consumers are relatively expensive, " - "and using a lot of them can lead to high memory and CPU usage. " - "To achieve better performance " - "of getting data from Kafka, consider using a setting kafka_thread_per_consumer=1, " - "and ensure you have enough threads " - "in MessageBrokerSchedulePool (background_message_broker_schedule_pool_size). " - "See also https://clickhouse.com/docs/en/integrations/kafka#tuning-performance", max_consumers); - } - else if (num_consumers < 1) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Number of consumers can not be lower than 1"); - } - - if (kafka_settings->kafka_max_block_size.changed && kafka_settings->kafka_max_block_size.value < 1) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "kafka_max_block_size can not be lower than 1"); - } - - if (kafka_settings->kafka_poll_max_batch_size.changed && kafka_settings->kafka_poll_max_batch_size.value < 1) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "kafka_poll_max_batch_size can not be lower than 1"); - } - NamesAndTypesList supported_columns; - for (const auto & column : args.columns) - { - if (column.default_desc.kind == ColumnDefaultKind::Alias) - supported_columns.emplace_back(column.name, column.type); - if (column.default_desc.kind == ColumnDefaultKind::Default && !column.default_desc.expression) - supported_columns.emplace_back(column.name, column.type); - } - // Kafka engine allows only ordinary columns without default expression or alias columns. - if (args.columns.getAll() != supported_columns) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "KafkaEngine doesn't support DEFAULT/MATERIALIZED/EPHEMERAL expressions for columns. " - "See https://clickhouse.com/docs/en/engines/table-engines/integrations/kafka/#configuration"); - } - - if (kafka_settings->keeper_path.changed && !args.getLocalContext()->getSettingsRef().allow_experimental_kafka_store_offsets_in_keeper){ - - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Storing the Kafka offsets in Keeper is experimental. " - "Set `allow_experimental_kafka_store_offsets_in_keeper` setting to enable it"); - } - - return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); - }; - - factory.registerStorage("Kafka", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); -} - NamesAndTypesList StorageKafka::getVirtuals() const { auto result = NamesAndTypesList{ diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 9280809be0e..907923c587b 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -23,8 +23,7 @@ class Configuration; namespace DB { -class StorageSystemKafkaConsumers; - +template struct StorageKafkaInterceptors; using KafkaConsumerPtr = std::shared_ptr; @@ -35,7 +34,8 @@ using KafkaConsumerWeakPtr = std::weak_ptr; */ class StorageKafka final : public IStorage, WithContext { - friend struct StorageKafkaInterceptors; + using StorageKafkaInterceptors = StorageKafkaInterceptors; + friend StorageKafkaInterceptors; public: StorageKafka( diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp new file mode 100644 index 00000000000..7cb2a38067a --- /dev/null +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -0,0 +1,722 @@ +#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 +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "Storages/ColumnDefault.h" +#include "config_version.h" + +#include +#include +#if USE_KRB5 +# include +#endif // USE_KRB5 + +namespace CurrentMetrics +{ +extern const Metric KafkaBackgroundReads; +extern const Metric KafkaConsumersInUse; +extern const Metric KafkaWrites; +} + +namespace ProfileEvents +{ +extern const Event KafkaDirectReads; +extern const Event KafkaBackgroundReads; +extern const Event KafkaWrites; +} + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; + extern const int QUERY_NOT_ALLOWED; +} + +namespace +{ + const auto RESCHEDULE_MS = 500; + const auto CLEANUP_TIMEOUT_MS = 3000; + const auto MAX_THREAD_WORK_DURATION_MS = 60000; // once per minute leave do reschedule (we can't lock threads in pool forever) +} + +StorageKafka2::StorageKafka2( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + std::unique_ptr kafka_settings_, + const String & collection_name_) + : IStorage(table_id_) + , WithContext(context_->getGlobalContext()) + , kafka_settings(std::move(kafka_settings_)) + , macros_info{.table_id = table_id_} + , topics(parseTopics(getContext()->getMacros()->expand(kafka_settings->kafka_topic_list.value, macros_info))) + , brokers(getContext()->getMacros()->expand(kafka_settings->kafka_broker_list.value, macros_info)) + , group(getContext()->getMacros()->expand(kafka_settings->kafka_group_name.value, macros_info)) + , client_id( + kafka_settings->kafka_client_id.value.empty() + ? getDefaultClientId(table_id_) + : getContext()->getMacros()->expand(kafka_settings->kafka_client_id.value, macros_info)) + , format_name(getContext()->getMacros()->expand(kafka_settings->kafka_format.value)) + , max_rows_per_message(kafka_settings->kafka_max_rows_per_message.value) + , schema_name(getContext()->getMacros()->expand(kafka_settings->kafka_schema.value, macros_info)) + , num_consumers(kafka_settings->kafka_num_consumers.value) + , log(&Poco::Logger::get("StorageKafka (" + table_id_.table_name + ")")) + , semaphore(0, static_cast(num_consumers)) + , intermediate_commit(kafka_settings->kafka_commit_every_batch.value) + , settings_adjustments(createSettingsAdjustments()) + , thread_per_consumer(kafka_settings->kafka_thread_per_consumer.value) + , collection_name(collection_name_) +{ + if (kafka_settings->kafka_handle_error_mode == HandleKafkaErrorMode::STREAM) + { + kafka_settings->input_format_allow_errors_num = 0; + kafka_settings->input_format_allow_errors_ratio = 0; + } + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + setInMemoryMetadata(storage_metadata); + auto task_count = thread_per_consumer ? num_consumers : 1; + for (size_t i = 0; i < task_count; ++i) + { + auto task = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this, i] { threadFunc(i); }); + task->deactivate(); + tasks.emplace_back(std::make_shared(std::move(task))); + } +} + +SettingsChanges StorageKafka2::createSettingsAdjustments() +{ + SettingsChanges result; + // Needed for backward compatibility + if (!kafka_settings->input_format_skip_unknown_fields.changed) + { + // Always skip unknown fields regardless of the context (JSON or TSKV) + kafka_settings->input_format_skip_unknown_fields = true; + } + + if (!kafka_settings->input_format_allow_errors_ratio.changed) + { + kafka_settings->input_format_allow_errors_ratio = 0.; + } + + if (!kafka_settings->input_format_allow_errors_num.changed) + { + kafka_settings->input_format_allow_errors_num = kafka_settings->kafka_skip_broken_messages.value; + } + + if (!schema_name.empty()) + result.emplace_back("format_schema", schema_name); + + for (const auto & setting : *kafka_settings) + { + const auto & name = setting.getName(); + if (name.find("kafka_") == std::string::npos) + result.emplace_back(name, setting.getValue()); + } + return result; +} + +Names StorageKafka2::parseTopics(String topic_list) +{ + Names result; + boost::split(result, topic_list, [](char c) { return c == ','; }); + for (String & topic : result) + { + boost::trim(topic); + } + return result; +} + +String StorageKafka2::getDefaultClientId(const StorageID & table_id_) +{ + return fmt::format("{}-{}-{}-{}", VERSION_NAME, getFQDNOrHostName(), table_id_.database_name, table_id_.table_name); +} + + +Pipe StorageKafka2::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & /* query_info */, + ContextPtr local_context, + QueryProcessingStage::Enum /* processed_stage */, + size_t /* max_block_size */, + size_t /* num_streams */) +{ + if (num_created_consumers == 0) + return {}; + + if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) + throw Exception( + ErrorCodes::QUERY_NOT_ALLOWED, "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); + + if (mv_attached) + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageKafka with attached materialized views"); + + ProfileEvents::increment(ProfileEvents::KafkaDirectReads); + + /// Always use all consumers at once, otherwise SELECT may not read messages from all partitions. + Pipes pipes; + pipes.reserve(num_created_consumers); + auto modified_context = Context::createCopy(local_context); + modified_context->applySettingsChanges(settings_adjustments); + + // Claim as many consumers as requested, but don't block + for (size_t i = 0; i < num_created_consumers; ++i) + { + /// Use block size of 1, otherwise LIMIT won't work properly as it will buffer excess messages in the last block + /// TODO: probably that leads to awful performance. + /// FIXME: seems that doesn't help with extra reading and committing unprocessed messages. + pipes.emplace_back(std::make_shared( + *this, storage_snapshot, modified_context, column_names, log, 1, kafka_settings->kafka_commit_on_select)); + } + + LOG_DEBUG(log, "Starting reading {} streams", pipes.size()); + return Pipe::unitePipes(std::move(pipes)); +} + + +SinkToStoragePtr +StorageKafka2::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) +{ + auto modified_context = Context::createCopy(local_context); + modified_context->applySettingsChanges(settings_adjustments); + + CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaWrites}; + ProfileEvents::increment(ProfileEvents::KafkaWrites); + + if (topics.size() > 1) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Can't write to Kafka table with multiple topics!"); + + cppkafka::Configuration conf; + conf.set("metadata.broker.list", brokers); + conf.set("client.id", client_id); + conf.set("client.software.name", VERSION_NAME); + conf.set("client.software.version", VERSION_DESCRIBE); + // TODO: fill required settings + updateConfiguration(conf); + + const Settings & settings = getContext()->getSettingsRef(); + size_t poll_timeout = settings.stream_poll_timeout_ms.totalMilliseconds(); + const auto & header = metadata_snapshot->getSampleBlockNonMaterialized(); + + auto producer = std::make_unique( + std::make_shared(conf), topics[0], std::chrono::milliseconds(poll_timeout), shutdown_called, header); + + size_t max_rows = max_rows_per_message; + /// Need for backward compatibility. + if (format_name == "Avro" && local_context->getSettingsRef().output_format_avro_rows_in_file.changed) + max_rows = local_context->getSettingsRef().output_format_avro_rows_in_file.value; + return std::make_shared(header, getFormatName(), max_rows, std::move(producer), getName(), modified_context); +} + + +void StorageKafka2::startup() +{ + for (size_t i = 0; i < num_consumers; ++i) + { + try + { + pushConsumer(createConsumer(i)); + ++num_created_consumers; + } + catch (const cppkafka::Exception &) + { + tryLogCurrentException(log); + } + } + + // Start the reader thread + for (auto & task : tasks) + { + task->holder->activateAndSchedule(); + } +} + + +void StorageKafka2::shutdown() +{ + for (auto & task : tasks) + { + // Interrupt streaming thread + task->stream_cancelled = true; + + LOG_TRACE(log, "Waiting for cleanup"); + task->holder->deactivate(); + } + + LOG_TRACE(log, "Closing consumers"); + for (size_t i = 0; i < num_created_consumers; ++i) + auto consumer = popConsumer(); + LOG_TRACE(log, "Consumers closed"); + + rd_kafka_wait_destroyed(CLEANUP_TIMEOUT_MS); +} + + +void StorageKafka2::pushConsumer(KafkaConsumer2Ptr consumer) +{ + std::lock_guard lock(mutex); + consumers.push_back(consumer); + semaphore.set(); + CurrentMetrics::sub(CurrentMetrics::KafkaConsumersInUse, 1); +} + + +KafkaConsumer2Ptr StorageKafka2::popConsumer() +{ + return popConsumer(std::chrono::milliseconds::zero()); +} + + +KafkaConsumer2Ptr StorageKafka2::popConsumer(std::chrono::milliseconds timeout) +{ + // Wait for the first free buffer + if (timeout == std::chrono::milliseconds::zero()) + semaphore.wait(); + else + { + if (!semaphore.tryWait(timeout.count())) + return nullptr; + } + + // Take the first available buffer from the list + std::lock_guard lock(mutex); + auto consumer = consumers.back(); + consumers.pop_back(); + CurrentMetrics::add(CurrentMetrics::KafkaConsumersInUse, 1); + return consumer; +} + + +KafkaConsumer2Ptr StorageKafka2::createConsumer(size_t consumer_number) +{ + cppkafka::Configuration conf; + + conf.set("metadata.broker.list", brokers); + conf.set("group.id", group); + if (num_consumers > 1) + { + conf.set("client.id", fmt::format("{}-{}", client_id, consumer_number)); + } + else + { + conf.set("client.id", client_id); + } + conf.set("client.software.name", VERSION_NAME); + conf.set("client.software.version", VERSION_DESCRIBE); + conf.set("auto.offset.reset", "earliest"); // If no offset stored for this group, read all messages from the start + + // that allows to prevent fast draining of the librdkafka queue + // during building of single insert block. Improves performance + // significantly, but may lead to bigger memory consumption. + size_t default_queued_min_messages = 100000; // we don't want to decrease the default + conf.set("queued.min.messages", std::max(getMaxBlockSize(), default_queued_min_messages)); + + updateConfiguration(conf); + + // those settings should not be changed by users. + conf.set("enable.auto.commit", "false"); // We manually commit offsets after a stream successfully finished + conf.set("enable.auto.offset.store", "false"); // Update offset automatically - to commit them all at once. + conf.set("enable.partition.eof", "false"); // Ignore EOF messages + + // Create a consumer and subscribe to topics + auto consumer_impl = std::make_shared(conf); + consumer_impl->set_destroy_flags(RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE); + + /// NOTE: we pass |stream_cancelled| by reference here, so the buffers should not outlive the storage. + if (thread_per_consumer) + { + auto & stream_cancelled = tasks[consumer_number]->stream_cancelled; + return std::make_shared( + consumer_impl, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), intermediate_commit, stream_cancelled, topics); + } + return std::make_shared( + consumer_impl, + log, + getPollMaxBatchSize(), + getPollTimeoutMillisecond(), + intermediate_commit, + tasks.back()->stream_cancelled, + topics); +} + +size_t StorageKafka2::getMaxBlockSize() const +{ + return kafka_settings->kafka_max_block_size.changed ? kafka_settings->kafka_max_block_size.value + : (getContext()->getSettingsRef().max_insert_block_size.value / num_consumers); +} + +size_t StorageKafka2::getPollMaxBatchSize() const +{ + size_t batch_size = kafka_settings->kafka_poll_max_batch_size.changed ? kafka_settings->kafka_poll_max_batch_size.value + : getContext()->getSettingsRef().max_block_size.value; + + return std::min(batch_size, getMaxBlockSize()); +} + +size_t StorageKafka2::getPollTimeoutMillisecond() const +{ + return kafka_settings->kafka_poll_timeout_ms.changed ? kafka_settings->kafka_poll_timeout_ms.totalMilliseconds() + : getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); +} + +String StorageKafka2::getConfigPrefix() const +{ + if (!collection_name.empty()) + return "named_collections." + collection_name + "." + + String{KafkaConfigLoader::CONFIG_KAFKA_TAG}; /// Add one more level to separate librdkafka configuration. + return String{KafkaConfigLoader::CONFIG_KAFKA_TAG}; +} + +void StorageKafka2::updateConfiguration(cppkafka::Configuration & kafka_config) +{ + // Update consumer configuration from the configuration. Example: + // + // 250 + // 100000 + // + const auto & config = getContext()->getConfigRef(); + auto config_prefix = getConfigPrefix(); + if (config.has(config_prefix)) + KafkaConfigLoader::loadConfig(kafka_config, config, config_prefix); + +#if USE_KRB5 + if (kafka_config.has_property("sasl.kerberos.kinit.cmd")) + LOG_WARNING(log, "sasl.kerberos.kinit.cmd configuration parameter is ignored."); + + kafka_config.set("sasl.kerberos.kinit.cmd", ""); + kafka_config.set("sasl.kerberos.min.time.before.relogin", "0"); + + if (kafka_config.has_property("sasl.kerberos.keytab") && kafka_config.has_property("sasl.kerberos.principal")) + { + String keytab = kafka_config.get("sasl.kerberos.keytab"); + String principal = kafka_config.get("sasl.kerberos.principal"); + LOG_DEBUG(log, "Running KerberosInit"); + try + { + kerberosInit(keytab, principal); + } + catch (const Exception & e) + { + LOG_ERROR(log, "KerberosInit failure: {}", getExceptionMessage(e, false)); + } + LOG_DEBUG(log, "Finished KerberosInit"); + } +#else // USE_KRB5 + if (kafka_config.has_property("sasl.kerberos.keytab") || kafka_config.has_property("sasl.kerberos.principal")) + LOG_WARNING(log, "Ignoring Kerberos-related parameters because ClickHouse was built without krb5 library support."); +#endif // USE_KRB5 + + // Update consumer topic-specific configuration (legacy syntax, retained for compatibility). Example with topic "football": + // + // 250 + // 100000 + // + // The legacy syntax has the problem that periods in topic names (e.g. "sports.football") are not supported because the Poco + // configuration framework hierarchy is based on periods as level separators. Besides that, per-topic tags at the same level + // as are ugly. + for (const auto & topic : topics) + { + const auto topic_config_key = config_prefix + "_" + topic; + if (config.has(topic_config_key)) + KafkaConfigLoader::loadConfig(kafka_config, config, topic_config_key); + } + + // Update consumer topic-specific configuration (new syntax). Example with topics "football" and "baseball": + // + // + // football + // 250 + // 5000 + // + // + // baseball + // 300 + // 2000 + // + // + // Advantages: The period restriction no longer applies (e.g. sports.football will work), everything + // Kafka-related is below . + for (const auto & topic : topics) + if (config.has(config_prefix)) + KafkaConfigLoader::loadTopicConfig(kafka_config, config, config_prefix, topic); + + // No need to add any prefix, messages can be distinguished + kafka_config.set_log_callback( + [this](cppkafka::KafkaHandleBase &, int level, const std::string & facility, const std::string & message) + { + auto [poco_level, client_logs_level] = parseSyslogLevel(level); + LOG_IMPL(log, client_logs_level, poco_level, "[rdk:{}] {}", facility, message); + }); + + // Configure interceptor to change thread name + // + // TODO: add interceptors support into the cppkafka. + // XXX: rdkafka uses pthread_set_name_np(), but glibc-compatibliity overrides it to noop. + { + // This should be safe, since we wait the rdkafka object anyway. + void * self = static_cast(this); + + int status; + + status = rd_kafka_conf_interceptor_add_on_new(kafka_config.get_handle(), "init", StorageKafkaInterceptors::rdKafkaOnNew, self); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(log, "Cannot set new interceptor due to {} error", status); + + // cppkafka always copy the configuration + status = rd_kafka_conf_interceptor_add_on_conf_dup( + kafka_config.get_handle(), "init", StorageKafkaInterceptors::rdKafkaOnConfDup, self); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(log, "Cannot set dup conf interceptor due to {} error", status); + } +} + +bool StorageKafka2::checkDependencies(const StorageID & table_id) +{ + // Check if all dependencies are attached + auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); + if (view_ids.empty()) + return true; + + // Check the dependencies are ready? + for (const auto & view_id : view_ids) + { + auto view = DatabaseCatalog::instance().tryGetTable(view_id, getContext()); + if (!view) + return false; + + // If it materialized view, check it's target table + auto * materialized_view = dynamic_cast(view.get()); + if (materialized_view && !materialized_view->tryGetTargetTable()) + return false; + + // Check all its dependencies + if (!checkDependencies(view_id)) + return false; + } + + return true; +} + +void StorageKafka2::threadFunc(size_t idx) +{ + assert(idx < tasks.size()); + auto task = tasks[idx]; + try + { + auto table_id = getStorageID(); + // Check if at least one direct dependency is attached + size_t num_views = DatabaseCatalog::instance().getDependentViews(table_id).size(); + if (num_views) + { + auto start_time = std::chrono::steady_clock::now(); + + mv_attached.store(true); + + // Keep streaming as long as there are attached views and streaming is not cancelled + while (!task->stream_cancelled && num_created_consumers > 0) + { + if (!checkDependencies(table_id)) + break; + + LOG_DEBUG(log, "Started streaming to {} attached views", num_views); + + // Exit the loop & reschedule if some stream stalled + auto some_stream_is_stalled = streamToViews(); + if (some_stream_is_stalled) + { + LOG_TRACE(log, "Stream(s) stalled. Reschedule."); + break; + } + + auto ts = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(ts - start_time); + if (duration.count() > MAX_THREAD_WORK_DURATION_MS) + { + LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); + break; + } + } + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + mv_attached.store(false); + + // Wait for attached views + if (!task->stream_cancelled) + task->holder->scheduleAfter(RESCHEDULE_MS); +} + + +bool StorageKafka2::streamToViews() +{ + Stopwatch watch; + + auto table_id = getStorageID(); + auto table = DatabaseCatalog::instance().getTable(table_id, getContext()); + if (!table) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Engine table {} doesn't exist.", table_id.getNameForLogs()); + + CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaBackgroundReads}; + ProfileEvents::increment(ProfileEvents::KafkaBackgroundReads); + + auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); + + // Create an INSERT query for streaming data + auto insert = std::make_shared(); + insert->table_id = table_id; + + size_t block_size = getMaxBlockSize(); + + auto kafka_context = Context::createCopy(getContext()); + kafka_context->makeQueryContext(); + kafka_context->applySettingsChanges(settings_adjustments); + + // Create a stream for each consumer and join them in a union stream + // Only insert into dependent views and expect that input blocks contain virtual columns + InterpreterInsertQuery interpreter(insert, kafka_context, false, true, true); + auto block_io = interpreter.execute(); + + // Create a stream for each consumer and join them in a union stream + std::vector> sources; + Pipes pipes; + + auto stream_count = thread_per_consumer ? 1 : num_created_consumers; + sources.reserve(stream_count); + pipes.reserve(stream_count); + for (size_t i = 0; i < stream_count; ++i) + { + auto source = std::make_shared( + *this, storage_snapshot, kafka_context, block_io.pipeline.getHeader().getNames(), log, block_size, false); + sources.emplace_back(source); + pipes.emplace_back(source); + + // Limit read batch to maximum block size to allow DDL + StreamLocalLimits limits; + + Poco::Timespan max_execution_time = kafka_settings->kafka_flush_interval_ms.changed + ? kafka_settings->kafka_flush_interval_ms + : getContext()->getSettingsRef().stream_flush_interval_ms; + + source->setTimeLimit(max_execution_time); + } + + auto pipe = Pipe::unitePipes(std::move(pipes)); + + // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. + // It will be cancelled on underlying layer (kafka buffer) + + std::atomic_size_t rows = 0; + { + block_io.pipeline.complete(std::move(pipe)); + + // we need to read all consumers in parallel (sequential read may lead to situation + // when some of consumers are not used, and will break some Kafka consumer invariants) + block_io.pipeline.setNumThreads(stream_count); + + block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); + CompletedPipelineExecutor executor(block_io.pipeline); + executor.execute(); + } + + bool some_stream_is_stalled = false; + for (auto & source : sources) + { + some_stream_is_stalled = some_stream_is_stalled || source->isStalled(); + source->commit(); + } + + UInt64 milliseconds = watch.elapsedMilliseconds(); + LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); + + return some_stream_is_stalled; +} + +NamesAndTypesList StorageKafka2::getVirtuals() const +{ + auto result = NamesAndTypesList{ + {"_topic", std::make_shared(std::make_shared())}, + {"_key", std::make_shared()}, + {"_offset", std::make_shared()}, + {"_partition", std::make_shared()}, + {"_timestamp", std::make_shared(std::make_shared())}, + {"_timestamp_ms", std::make_shared(std::make_shared(3))}, + {"_headers.name", std::make_shared(std::make_shared())}, + {"_headers.value", std::make_shared(std::make_shared())}}; + if (kafka_settings->kafka_handle_error_mode == HandleKafkaErrorMode::STREAM) + { + result.push_back({"_raw_message", std::make_shared()}); + result.push_back({"_error", std::make_shared()}); + } + return result; +} + +Names StorageKafka2::getVirtualColumnNames() const +{ + auto result = Names{ + "_topic", + "_key", + "_offset", + "_partition", + "_timestamp", + "_timestamp_ms", + "_headers.name", + "_headers.value", + }; + if (kafka_settings->kafka_handle_error_mode == HandleKafkaErrorMode::STREAM) + { + result.push_back({"_raw_message"}); + result.push_back({"_error"}); + } + return result; +} + +} diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h new file mode 100644 index 00000000000..d0bc5cc78b7 --- /dev/null +++ b/src/Storages/Kafka/StorageKafka2.h @@ -0,0 +1,149 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include + +namespace cppkafka +{ + +class Configuration; + +} + +namespace DB +{ + +template +struct StorageKafkaInterceptors; + +using KafkaConsumer2Ptr = std::shared_ptr; + +/** Implements a Kafka queue table engine that can be used as a persistent queue / buffer, + * or as a basic building block for creating pipelines with a continuous insertion / ETL. + */ +class StorageKafka2 final : public IStorage, WithContext +{ + using StorageKafkaInterceptors = StorageKafkaInterceptors; + friend StorageKafkaInterceptors; + +public: + StorageKafka2( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + std::unique_ptr kafka_settings_, + const String & collection_name_); + + std::string getName() const override { return "Kafka"; } + + bool noPushingToViews() const override { return true; } + + void startup() override; + void shutdown() override; + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + SinkToStoragePtr write( + const ASTPtr & query, + const StorageMetadataPtr & /*metadata_snapshot*/, + ContextPtr context, + bool async_insert) override; + + /// We want to control the number of rows in a chunk inserted into Kafka + bool prefersLargeBlocks() const override { return false; } + + void pushConsumer(KafkaConsumer2Ptr consumer); + KafkaConsumer2Ptr popConsumer(); + KafkaConsumer2Ptr popConsumer(std::chrono::milliseconds timeout); + + const auto & getFormatName() const { return format_name; } + + NamesAndTypesList getVirtuals() const override; + Names getVirtualColumnNames() const; + HandleKafkaErrorMode getHandleKafkaErrorMode() const { return kafka_settings->kafka_handle_error_mode; } + +private: + // Configuration and state + std::unique_ptr kafka_settings; + Macros::MacroExpansionInfo macros_info; + const Names topics; + const String brokers; + const String group; + const String client_id; + const String format_name; + const size_t max_rows_per_message; + const String schema_name; + const size_t num_consumers; /// total number of consumers + Poco::Logger * log; + Poco::Semaphore semaphore; + const bool intermediate_commit; + const SettingsChanges settings_adjustments; + + std::atomic mv_attached = false; + + /// Can differ from num_consumers in case of exception in startup() (or if startup() hasn't been called). + /// In this case we still need to be able to shutdown() properly. + size_t num_created_consumers = 0; /// number of actually created consumers. + + std::vector consumers; /// available consumers + + std::mutex mutex; + + // Stream thread + struct TaskContext + { + BackgroundSchedulePool::TaskHolder holder; + std::atomic stream_cancelled {false}; + explicit TaskContext(BackgroundSchedulePool::TaskHolder&& task_) : holder(std::move(task_)) + { + } + }; + std::vector> tasks; + bool thread_per_consumer = false; + + /// For memory accounting in the librdkafka threads. + std::mutex thread_statuses_mutex; + std::list> thread_statuses; + + SettingsChanges createSettingsAdjustments(); + KafkaConsumer2Ptr createConsumer(size_t consumer_number); + + /// If named_collection is specified. + String collection_name; + + std::atomic shutdown_called = false; + + // Update Kafka configuration with values from CH user configuration. + void updateConfiguration(cppkafka::Configuration & kafka_config); + String getConfigPrefix() const; + void threadFunc(size_t idx); + + size_t getPollMaxBatchSize() const; + size_t getMaxBlockSize() const; + size_t getPollTimeoutMillisecond() const; + + static Names parseTopics(String topic_list); + static String getDefaultClientId(const StorageID & table_id_); + + bool streamToViews(); + bool checkDependencies(const StorageID & table_id); +}; + +} diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp new file mode 100644 index 00000000000..3a35272ac74 --- /dev/null +++ b/src/Storages/Kafka/StorageKafkaCommon.cpp @@ -0,0 +1,386 @@ +#include + + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + + +namespace CurrentMetrics +{ +extern const Metric KafkaLibrdkafkaThreads; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int SUPPORT_IS_DISABLED; +} + +template +rd_kafka_resp_err_t +StorageKafkaInterceptors::rdKafkaOnThreadStart(rd_kafka_t *, rd_kafka_thread_type_t thread_type, const char *, void * ctx) +{ + TStorageKafka * self = reinterpret_cast(ctx); + CurrentMetrics::add(CurrentMetrics::KafkaLibrdkafkaThreads, 1); + + const auto & storage_id = self->getStorageID(); + const auto & table = storage_id.getTableName(); + + switch (thread_type) + { + case RD_KAFKA_THREAD_MAIN: + setThreadName(("rdk:m/" + table.substr(0, 9)).c_str()); + break; + case RD_KAFKA_THREAD_BACKGROUND: + setThreadName(("rdk:bg/" + table.substr(0, 8)).c_str()); + break; + case RD_KAFKA_THREAD_BROKER: + setThreadName(("rdk:b/" + table.substr(0, 9)).c_str()); + break; + } + + /// Create ThreadStatus to track memory allocations from librdkafka threads. + // + /// And store them in a separate list (thread_statuses) to make sure that they will be destroyed, + /// regardless how librdkafka calls the hooks. + /// But this can trigger use-after-free if librdkafka will not destroy threads after rd_kafka_wait_destroyed() + auto thread_status = std::make_shared(); + std::lock_guard lock(self->thread_statuses_mutex); + self->thread_statuses.emplace_back(std::move(thread_status)); + + return RD_KAFKA_RESP_ERR_NO_ERROR; +} + +template +rd_kafka_resp_err_t +StorageKafkaInterceptors::rdKafkaOnThreadExit(rd_kafka_t *, rd_kafka_thread_type_t, const char *, void * ctx) +{ + TStorageKafka * self = reinterpret_cast(ctx); + CurrentMetrics::sub(CurrentMetrics::KafkaLibrdkafkaThreads, 1); + + std::lock_guard lock(self->thread_statuses_mutex); + const auto it = std::find_if( + self->thread_statuses.begin(), + self->thread_statuses.end(), + [](const auto & thread_status_ptr) { return thread_status_ptr.get() == current_thread; }); + if (it == self->thread_statuses.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread status for this librdkafka thread."); + + self->thread_statuses.erase(it); + + return RD_KAFKA_RESP_ERR_NO_ERROR; +} + +template +rd_kafka_resp_err_t StorageKafkaInterceptors::rdKafkaOnNew( + rd_kafka_t * rk, const rd_kafka_conf_t *, void * ctx, char * /*errstr*/, size_t /*errstr_size*/) +{ + TStorageKafka * self = reinterpret_cast(ctx); + rd_kafka_resp_err_t status; + + status = rd_kafka_interceptor_add_on_thread_start(rk, "init-thread", rdKafkaOnThreadStart, ctx); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + { + LOG_ERROR(self->log, "Cannot set on thread start interceptor due to {} error", status); + return status; + } + + status = rd_kafka_interceptor_add_on_thread_exit(rk, "exit-thread", rdKafkaOnThreadExit, ctx); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(self->log, "Cannot set on thread exit interceptor due to {} error", status); + + return status; +} + +template +rd_kafka_resp_err_t StorageKafkaInterceptors::rdKafkaOnConfDup( + rd_kafka_conf_t * new_conf, const rd_kafka_conf_t * /*old_conf*/, size_t /*filter_cnt*/, const char ** /*filter*/, void * ctx) +{ + TStorageKafka * self = reinterpret_cast(ctx); + rd_kafka_resp_err_t status; + + // cppkafka copies configuration multiple times + status = rd_kafka_conf_interceptor_add_on_conf_dup(new_conf, "init", rdKafkaOnConfDup, ctx); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + { + LOG_ERROR(self->log, "Cannot set on conf dup interceptor due to {} error", status); + return status; + } + + status = rd_kafka_conf_interceptor_add_on_new(new_conf, "init", rdKafkaOnNew, ctx); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(self->log, "Cannot set on conf new interceptor due to {} error", status); + + return status; +} + +void KafkaConfigLoader::loadConfig( + cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) +{ + /// Read all tags one level below + Poco::Util::AbstractConfiguration::Keys tags; + config.keys(config_prefix, tags); + + for (const auto & tag : tags) + { + if (tag.starts_with(CONFIG_KAFKA_TOPIC_TAG)) /// multiple occurrences given as "kafka_topic", "kafka_topic[1]", etc. + continue; /// used by new per-topic configuration, ignore + + const String setting_path = config_prefix + "." + tag; + const String setting_value = config.getString(setting_path); + + /// "log_level" has valid underscore, the remaining librdkafka setting use dot.separated.format which isn't acceptable for XML. + /// See https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md + const String setting_name_in_kafka_config = (tag == "log_level") ? tag : boost::replace_all_copy(tag, "_", "."); + kafka_config.set(setting_name_in_kafka_config, setting_value); + } +} + +void KafkaConfigLoader::loadTopicConfig( + cppkafka::Configuration & kafka_config, + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + const String & topic) +{ + /// Read all tags one level below + Poco::Util::AbstractConfiguration::Keys tags; + config.keys(config_prefix, tags); + + for (const auto & tag : tags) + { + /// Only consider tag . Multiple occurrences given as "kafka_topic", "kafka_topic[1]", etc. + if (!tag.starts_with(CONFIG_KAFKA_TOPIC_TAG)) + continue; + + /// Read topic name between ... + const String kafka_topic_path = config_prefix + "." + tag; + const String kafpa_topic_name_path = kafka_topic_path + "." + String{CONFIG_NAME_TAG}; + + const String topic_name = config.getString(kafpa_topic_name_path); + if (topic_name == topic) + { + /// Found it! Now read the per-topic configuration into cppkafka. + Poco::Util::AbstractConfiguration::Keys inner_tags; + config.keys(kafka_topic_path, inner_tags); + for (const auto & inner_tag : inner_tags) + { + if (inner_tag == CONFIG_NAME_TAG) + continue; // ignore + + /// "log_level" has valid underscore, the remaining librdkafka setting use dot.separated.format which isn't acceptable for XML. + /// See https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md + const String setting_path = kafka_topic_path + "." + inner_tag; + const String setting_value = config.getString(setting_path); + + const String setting_name_in_kafka_config + = (inner_tag == "log_level") ? inner_tag : boost::replace_all_copy(inner_tag, "_", "."); + kafka_config.set(setting_name_in_kafka_config, setting_value); + } + } + } +} + + +void registerStorageKafka(StorageFactory & factory) +{ + auto creator_fn = [](const StorageFactory::Arguments & args) -> std::shared_ptr + { + ASTs & engine_args = args.engine_args; + size_t args_count = engine_args.size(); + const bool has_settings = args.storage_def->settings; + + auto kafka_settings = std::make_unique(); + String collection_name; + if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args, args.getLocalContext())) + { + for (const auto & setting : kafka_settings->all()) + { + const auto & setting_name = setting.getName(); + if (named_collection->has(setting_name)) + kafka_settings->set(setting_name, named_collection->get(setting_name)); + } + collection_name = assert_cast(args.engine_args[0].get())->name(); + } + + if (has_settings) + { + kafka_settings->loadFromQuery(*args.storage_def); + } + +// Check arguments and settings +#define CHECK_KAFKA_STORAGE_ARGUMENT(ARG_NUM, PAR_NAME, EVAL) \ + /* One of the four required arguments is not specified */ \ + if (args_count < (ARG_NUM) && (ARG_NUM) <= 4 && !kafka_settings->PAR_NAME.changed) \ + { \ + throw Exception( \ + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, \ + "Required parameter '{}' " \ + "for storage Kafka not specified", \ + #PAR_NAME); \ + } \ + if (args_count >= (ARG_NUM)) \ + { \ + /* The same argument is given in two places */ \ + if (has_settings && kafka_settings->PAR_NAME.changed) \ + { \ + throw Exception( \ + ErrorCodes::BAD_ARGUMENTS, \ + "The argument №{} of storage Kafka " \ + "and the parameter '{}' " \ + "in SETTINGS cannot be specified at the same time", \ + #ARG_NUM, \ + #PAR_NAME); \ + } \ + /* move engine args to settings */ \ + else \ + { \ + if constexpr ((EVAL) == 1) \ + { \ + engine_args[(ARG_NUM)-1] = evaluateConstantExpressionAsLiteral(engine_args[(ARG_NUM)-1], args.getLocalContext()); \ + } \ + if constexpr ((EVAL) == 2) \ + { \ + engine_args[(ARG_NUM)-1] \ + = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[(ARG_NUM)-1], args.getLocalContext()); \ + } \ + kafka_settings->PAR_NAME = engine_args[(ARG_NUM)-1]->as().value; \ + } \ + } + + /** Arguments of engine is following: + * - Kafka broker list + * - List of topics + * - Group ID (may be a constant expression with a string result) + * - Message format (string) + * - Row delimiter + * - Schema (optional, if the format supports it) + * - Number of consumers + * - Max block size for background consumption + * - Skip (at least) unreadable messages number + * - Do intermediate commits when the batch consumed and handled + */ + + /* 0 = raw, 1 = evaluateConstantExpressionAsLiteral, 2=evaluateConstantExpressionOrIdentifierAsLiteral */ + /// In case of named collection we already validated the arguments. + if (collection_name.empty()) + { + CHECK_KAFKA_STORAGE_ARGUMENT(1, kafka_broker_list, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(2, kafka_topic_list, 1) + CHECK_KAFKA_STORAGE_ARGUMENT(3, kafka_group_name, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(4, kafka_format, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(5, kafka_row_delimiter, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(6, kafka_schema, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(7, kafka_num_consumers, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(10, kafka_commit_every_batch, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(11, kafka_client_id, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(12, kafka_poll_timeout_ms, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(13, kafka_flush_interval_ms, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(14, kafka_thread_per_consumer, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(15, kafka_handle_error_mode, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(16, kafka_commit_on_select, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(17, kafka_max_rows_per_message, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(18, kafka_keeper_path, 0) + } + +#undef CHECK_KAFKA_STORAGE_ARGUMENT + + auto num_consumers = kafka_settings->kafka_num_consumers.value; + auto max_consumers = std::max(getNumberOfPhysicalCPUCores(), 16); + + if (!args.getLocalContext()->getSettingsRef().kafka_disable_num_consumers_limit && num_consumers > max_consumers) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "The number of consumers can not be bigger than {}. " + "A single consumer can read any number of partitions. " + "Extra consumers are relatively expensive, " + "and using a lot of them can lead to high memory and CPU usage. " + "To achieve better performance " + "of getting data from Kafka, consider using a setting kafka_thread_per_consumer=1, " + "and ensure you have enough threads " + "in MessageBrokerSchedulePool (background_message_broker_schedule_pool_size). " + "See also https://clickhouse.com/docs/integrations/kafka/kafka-table-engine#tuning-performance", + max_consumers); + } + else if (num_consumers < 1) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Number of consumers can not be lower than 1"); + } + + if (kafka_settings->kafka_max_block_size.changed && kafka_settings->kafka_max_block_size.value < 1) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "kafka_max_block_size can not be lower than 1"); + } + + if (kafka_settings->kafka_poll_max_batch_size.changed && kafka_settings->kafka_poll_max_batch_size.value < 1) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "kafka_poll_max_batch_size can not be lower than 1"); + } + NamesAndTypesList supported_columns; + for (const auto & column : args.columns) + { + if (column.default_desc.kind == ColumnDefaultKind::Alias) + supported_columns.emplace_back(column.name, column.type); + if (column.default_desc.kind == ColumnDefaultKind::Default && !column.default_desc.expression) + supported_columns.emplace_back(column.name, column.type); + } + // Kafka engine allows only ordinary columns without default expression or alias columns. + if (args.columns.getAll() != supported_columns) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "KafkaEngine doesn't support DEFAULT/MATERIALIZED/EPHEMERAL expressions for columns. " + "See https://clickhouse.com/docs/en/engines/table-engines/integrations/kafka/#configuration"); + } + + if (kafka_settings->kafka_keeper_path.changed) + { + if (!args.getLocalContext()->getSettingsRef().allow_experimental_kafka_store_offsets_in_keeper) + + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "Storing the Kafka offsets in Keeper is experimental. " + "Set `allow_experimental_kafka_store_offsets_in_keeper` setting to enable it"); + + return std::make_shared( + args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); + } + + return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); + }; + + factory.registerStorage( + "Kafka", + creator_fn, + StorageFactory::StorageFeatures{ + .supports_settings = true, + }); +} + +template struct StorageKafkaInterceptors; +template struct StorageKafkaInterceptors; + +} diff --git a/src/Storages/Kafka/StorageKafkaCommon.h b/src/Storages/Kafka/StorageKafkaCommon.h new file mode 100644 index 00000000000..108dcbf19e2 --- /dev/null +++ b/src/Storages/Kafka/StorageKafkaCommon.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include + +namespace Poco +{ +namespace Util +{ + class AbstractConfiguration; +} +} + +namespace DB +{ +template +struct StorageKafkaInterceptors +{ + static rd_kafka_resp_err_t rdKafkaOnThreadStart(rd_kafka_t *, rd_kafka_thread_type_t thread_type, const char *, void * ctx); + + static rd_kafka_resp_err_t rdKafkaOnThreadExit(rd_kafka_t *, rd_kafka_thread_type_t, const char *, void * ctx); + + static rd_kafka_resp_err_t + rdKafkaOnNew(rd_kafka_t * rk, const rd_kafka_conf_t *, void * ctx, char * /*errstr*/, size_t /*errstr_size*/); + + static rd_kafka_resp_err_t rdKafkaOnConfDup( + rd_kafka_conf_t * new_conf, const rd_kafka_conf_t * /*old_conf*/, size_t /*filter_cnt*/, const char ** /*filter*/, void * ctx); +}; + +struct KafkaConfigLoader +{ + static constexpr std::string_view CONFIG_KAFKA_TAG = "kafka"; + static constexpr std::string_view CONFIG_KAFKA_TOPIC_TAG = "kafka_topic"; + static constexpr std::string_view CONFIG_NAME_TAG = "name"; + + /// Read server configuration into cppkafka configuration, used by global configuration and by legacy per-topic configuration + static void loadConfig( + cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & config_prefix); + + /// Read server configuration into cppkafa configuration, used by new per-topic configuration + static void loadTopicConfig( + cppkafka::Configuration & kafka_config, + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + const String & topic); +}; +} From 86fab063382b8576292227b55a89a2363de3e1bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 1 Sep 2023 14:22:37 +0000 Subject: [PATCH 004/644] Very rudimentary version that works with single thread and consumer --- src/Common/ZooKeeper/ZooKeeper.cpp | 2 +- src/Storages/Kafka/KafkaConsumer2.cpp | 458 +++++++--------- src/Storages/Kafka/KafkaConsumer2.h | 97 +++- src/Storages/Kafka/KafkaSettings.h | 1 + src/Storages/Kafka/KafkaSource2.cpp | 303 ----------- src/Storages/Kafka/KafkaSource2.h | 64 --- src/Storages/Kafka/StorageKafka2.cpp | 627 +++++++++++++++++----- src/Storages/Kafka/StorageKafka2.h | 80 ++- src/Storages/Kafka/StorageKafkaCommon.cpp | 2 +- 9 files changed, 835 insertions(+), 799 deletions(-) delete mode 100644 src/Storages/Kafka/KafkaSource2.cpp delete mode 100644 src/Storages/Kafka/KafkaSource2.h diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 436a4e14f14..22e99c9120c 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -853,7 +853,7 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & /// method is called. do { - /// Use getData insteand of exists to avoid watch leak. + /// Use getData instead of exists to avoid watch leak. impl->get(path, callback, std::make_shared(watch)); if (!state->event.tryWait(1000)) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index ec32248af46..61b6f801e9e 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -1,34 +1,39 @@ // Needs to go first because its partial specialization of fmt::formatter // should be defined before any instantiation +#include +#include +#include #include -#include #include +#include #include -#include -#include #include +#include +#include +#include #include #include +#include "base/scope_guard.h" namespace CurrentMetrics { - extern const Metric KafkaAssignedPartitions; - extern const Metric KafkaConsumersWithAssignment; +extern const Metric KafkaAssignedPartitions; +extern const Metric KafkaConsumersWithAssignment; } namespace ProfileEvents { - extern const Event KafkaRebalanceRevocations; - extern const Event KafkaRebalanceAssignments; - extern const Event KafkaRebalanceErrors; - extern const Event KafkaMessagesPolled; - extern const Event KafkaCommitFailures; - extern const Event KafkaCommits; - extern const Event KafkaConsumerErrors; +extern const Event KafkaRebalanceRevocations; +extern const Event KafkaRebalanceAssignments; +extern const Event KafkaRebalanceErrors; +extern const Event KafkaMessagesPolled; +extern const Event KafkaCommitFailures; +extern const Event KafkaCommits; +extern const Event KafkaConsumerErrors; } namespace DB @@ -45,6 +50,12 @@ const std::size_t POLL_TIMEOUT_WO_ASSIGNMENT_MS = 50; const auto DRAIN_TIMEOUT_MS = 5000ms; +bool KafkaConsumer2::TopicPartition::operator<(const TopicPartition & other) const +{ + return std::tie(topic, partition_id, offset) < std::tie(other.topic, other.partition_id, other.offset); +} + + KafkaConsumer2::KafkaConsumer2( ConsumerPtr consumer_, Poco::Logger * log_, @@ -63,68 +74,87 @@ KafkaConsumer2::KafkaConsumer2( , topics(_topics) { // called (synchronously, during poll) when we enter the consumer group - consumer->set_assignment_callback([this](const cppkafka::TopicPartitionList & topic_partitions) - { - CurrentMetrics::add(CurrentMetrics::KafkaAssignedPartitions, topic_partitions.size()); - ProfileEvents::increment(ProfileEvents::KafkaRebalanceAssignments); - - if (topic_partitions.empty()) + consumer->set_assignment_callback( + [this](const cppkafka::TopicPartitionList & topic_partitions) { - LOG_INFO(log, "Got empty assignment: Not enough partitions in the topic for all consumers?"); - } - else - { - LOG_TRACE(log, "Topics/partitions assigned: {}", topic_partitions); - CurrentMetrics::add(CurrentMetrics::KafkaConsumersWithAssignment, 1); - } + CurrentMetrics::add(CurrentMetrics::KafkaAssignedPartitions, topic_partitions.size()); + ProfileEvents::increment(ProfileEvents::KafkaRebalanceAssignments); - assignment = topic_partitions; - }); + if (topic_partitions.empty()) + { + LOG_INFO(log, "Got empty assignment: Not enough partitions in the topic for all consumers?"); + } + else + { + LOG_TRACE(log, "Topics/partitions assigned: {}", topic_partitions); + CurrentMetrics::add(CurrentMetrics::KafkaConsumersWithAssignment, 1); + } + + chassert(!assignment.has_value()); + + assignment.emplace(); + assignment->reserve(topic_partitions.size()); + needs_offset_update = true; + for (const auto & topic_partition : topic_partitions) + { + assignment->push_back(TopicPartition{topic_partition.get_topic(), topic_partition.get_partition(), INVALID_OFFSET}); + } + std::sort(assignment->begin(), assignment->end()); + + updateOffsets(topic_partitions); + }); // called (synchronously, during poll) when we leave the consumer group - consumer->set_revocation_callback([this](const cppkafka::TopicPartitionList & topic_partitions) - { - CurrentMetrics::sub(CurrentMetrics::KafkaAssignedPartitions, topic_partitions.size()); - ProfileEvents::increment(ProfileEvents::KafkaRebalanceRevocations); - - // Rebalance is happening now, and now we have a chance to finish the work - // with topics/partitions we were working with before rebalance - LOG_TRACE(log, "Rebalance initiated. Revoking partitions: {}", topic_partitions); - - if (!topic_partitions.empty()) + consumer->set_revocation_callback( + [this](const cppkafka::TopicPartitionList & topic_partitions) { - CurrentMetrics::sub(CurrentMetrics::KafkaConsumersWithAssignment, 1); - } + // TODO(antaljanosbenjamin): deal with revocation + CurrentMetrics::sub(CurrentMetrics::KafkaAssignedPartitions, topic_partitions.size()); + ProfileEvents::increment(ProfileEvents::KafkaRebalanceRevocations); - // we can not flush data to target from that point (it is pulled, not pushed) - // so the best we can now it to - // 1) repeat last commit in sync mode (async could be still in queue, we need to be sure is is properly committed before rebalance) - // 2) stop / brake the current reading: - // * clean buffered non-commited messages - // * set flag / flush + // Rebalance is happening now, and now we have a chance to finish the work + // with topics/partitions we were working with before rebalance + LOG_TRACE(log, "Rebalance initiated. Revoking partitions: {}", topic_partitions); - cleanUnprocessed(); + if (!topic_partitions.empty()) + { + CurrentMetrics::sub(CurrentMetrics::KafkaConsumersWithAssignment, 1); + } - stalled_status = REBALANCE_HAPPENED; - assignment.reset(); - waited_for_assignment = 0; + // we can not flush data to target from that point (it is pulled, not pushed) + // so the best we can now it to + // 1) repeat last commit in sync mode (async could be still in queue, we need to be sure is is properly committed before rebalance) + // 2) stop / brake the current reading: + // * clean buffered non-commited messages + // * set flag / flush - // for now we use slower (but reliable) sync commit in main loop, so no need to repeat - // try - // { - // consumer->commit(); - // } - // catch (cppkafka::HandleException & e) - // { - // LOG_WARNING(log, "Commit error: {}", e.what()); - // } - }); + cleanUnprocessed(); - consumer->set_rebalance_error_callback([this](cppkafka::Error err) - { - LOG_ERROR(log, "Rebalance error: {}", err); - ProfileEvents::increment(ProfileEvents::KafkaRebalanceErrors); - }); + stalled_status = StalledStatus::REBALANCE_HAPPENED; + assignment.reset(); + queues.clear(); + needs_offset_update = true; + waited_for_assignment = 0; + + // for now we use slower (but reliable) sync commit in main loop, so no need to repeat + // try + // { + // consumer->commit(); + // } + // catch (cppkafka::HandleException & e) + // { + // LOG_WARNING(log, "Commit error: {}", e.what()); + // } + }); + + consumer->set_rebalance_error_callback( + [this](cppkafka::Error err) + { + LOG_ERROR(log, "Rebalance error: {}", err); + ProfileEvents::increment(ProfileEvents::KafkaRebalanceErrors); + }); + + consumer->subscribe(topics); } KafkaConsumer2::~KafkaConsumer2() @@ -148,7 +178,6 @@ KafkaConsumer2::~KafkaConsumer2() { LOG_ERROR(log, "Error while destructing consumer: {}", e.what()); } - } // Needed to drain rest of the messages / queued callback calls from the consumer @@ -160,6 +189,9 @@ void KafkaConsumer2::drain() auto start_time = std::chrono::steady_clock::now(); cppkafka::Error last_error(RD_KAFKA_RESP_ERR_NO_ERROR); + for (auto & [tp, queue] : queues) + queue.forward_to_queue(consumer->get_consumer_queue()); + while (true) { auto msg = consumer->poll(100ms); @@ -185,7 +217,7 @@ void KafkaConsumer2::drain() last_error = error; auto ts = std::chrono::steady_clock::now(); - if (std::chrono::duration_cast(ts-start_time) > DRAIN_TIMEOUT_MS) + if (std::chrono::duration_cast(ts - start_time) > DRAIN_TIMEOUT_MS) { LOG_ERROR(log, "Timeout during draining."); break; @@ -193,139 +225,6 @@ void KafkaConsumer2::drain() } } - -void KafkaConsumer2::commit() -{ - auto print_offsets = [this] (const char * prefix, const cppkafka::TopicPartitionList & offsets) - { - for (const auto & topic_part : offsets) - { - auto print_special_offset = [&topic_part] - { - switch (topic_part.get_offset()) - { - case cppkafka::TopicPartition::OFFSET_BEGINNING: return "BEGINNING"; - case cppkafka::TopicPartition::OFFSET_END: return "END"; - case cppkafka::TopicPartition::OFFSET_STORED: return "STORED"; - case cppkafka::TopicPartition::OFFSET_INVALID: return "INVALID"; - default: return ""; - } - }; - - if (topic_part.get_offset() < 0) - { - LOG_TRACE(log, "{} {} (topic: {}, partition: {})", prefix, print_special_offset(), topic_part.get_topic(), topic_part.get_partition()); - } - else - { - LOG_TRACE(log, "{} {} (topic: {}, partition: {})", prefix, topic_part.get_offset(), topic_part.get_topic(), topic_part.get_partition()); - } - } - }; - - print_offsets("Polled offset", consumer->get_offsets_position(consumer->get_assignment())); - - if (hasMorePolledMessages()) - { - LOG_WARNING(log, "Logical error. Non all polled messages were processed."); - } - - if (offsets_stored > 0) - { - // if we will do async commit here (which is faster) - // we may need to repeat commit in sync mode in revocation callback, - // but it seems like existing API doesn't allow us to to that - // in a controlled manner (i.e. we don't know the offsets to commit then) - - size_t max_retries = 5; - bool committed = false; - - while (!committed && max_retries > 0) - { - try - { - // See https://github.com/edenhill/librdkafka/issues/1470 - // broker may reject commit if during offsets.commit.timeout.ms (5000 by default), - // there were not enough replicas available for the __consumer_offsets topic. - // also some other temporary issues like client-server connectivity problems are possible - consumer->commit(); - committed = true; - print_offsets("Committed offset", consumer->get_offsets_committed(consumer->get_assignment())); - } - catch (const cppkafka::HandleException & e) - { - // If there were actually no offsets to commit, return. Retrying won't solve - // anything here - if (e.get_error() == RD_KAFKA_RESP_ERR__NO_OFFSET) - committed = true; - else - LOG_ERROR(log, "Exception during commit attempt: {}", e.what()); - } - --max_retries; - } - - if (!committed) - { - // TODO: insert atomicity / transactions is needed here (possibility to rollback, on 2 phase commits) - ProfileEvents::increment(ProfileEvents::KafkaCommitFailures); - throw Exception(ErrorCodes::CANNOT_COMMIT_OFFSET, - "All commit attempts failed. Last block was already written to target table(s), " - "but was not committed to Kafka."); - } - else - { - ProfileEvents::increment(ProfileEvents::KafkaCommits); - } - - } - else - { - LOG_TRACE(log, "Nothing to commit."); - } - - offsets_stored = 0; -} - -void KafkaConsumer2::subscribe() -{ - LOG_TRACE(log, "Already subscribed to topics: [{}]", boost::algorithm::join(consumer->get_subscription(), ", ")); - - if (assignment.has_value()) - { - LOG_TRACE(log, "Already assigned to: {}", assignment.value()); - } - else - { - LOG_TRACE(log, "No assignment"); - } - - - size_t max_retries = 5; - - while (consumer->get_subscription().empty()) - { - --max_retries; - try - { - consumer->subscribe(topics); - // FIXME: if we failed to receive "subscribe" response while polling and destroy consumer now, then we may hang up. - // see https://github.com/edenhill/librdkafka/issues/2077 - } - catch (cppkafka::HandleException & e) - { - if (max_retries > 0 && e.get_error() == RD_KAFKA_RESP_ERR__TIMED_OUT) - continue; - throw; - } - } - - cleanUnprocessed(); - - // we can reset any flags (except of CONSUMER_STOPPED) before attempt of reading new block of data - if (stalled_status != CONSUMER_STOPPED) - stalled_status = NO_MESSAGES_RETURNED; -} - void KafkaConsumer2::cleanUnprocessed() { messages.clear(); @@ -333,59 +232,100 @@ void KafkaConsumer2::cleanUnprocessed() offsets_stored = 0; } -void KafkaConsumer2::unsubscribe() +void KafkaConsumer2::pollEvents() { - LOG_TRACE(log, "Re-joining claimed consumer after failure"); - cleanUnprocessed(); + // All the partition queues are detached, so the consumer shouldn't be able to poll any messages + auto msg = consumer->poll(10ms); + chassert(!msg && "Consumer returned a message when it was not expected"); +}; - // it should not raise exception as used in destructor +KafkaConsumer2::TopicPartitionCounts KafkaConsumer2::getPartitionCounts() const +{ + TopicPartitionCounts result; try { - // From docs: Any previous subscription will be unassigned and unsubscribed first. - consumer->subscribe(topics); + auto metadata = consumer->get_metadata(); + auto topic_metadatas = metadata.get_topics(); - // I wanted to avoid explicit unsubscribe as it requires draining the messages - // to close the consumer safely after unsubscribe - // see https://github.com/edenhill/librdkafka/issues/2077 - // https://github.com/confluentinc/confluent-kafka-go/issues/189 etc. + for (auto & topic_metadata : topic_metadatas) + { + if (const auto it = std::find(topics.begin(), topics.end(), topic_metadata.get_name()); it != topics.end()) + { + result.push_back({topic_metadata.get_name(), topic_metadata.get_partitions().size()}); + } + } } - catch (const cppkafka::HandleException & e) + catch (cppkafka::HandleException & e) { - LOG_ERROR(log, "Exception from KafkaConsumer2::unsubscribe: {}", e.what()); + chassert(e.what() != nullptr); } - + return result; } - -void KafkaConsumer2::resetToLastCommitted(const char * msg) +bool KafkaConsumer2::polledDataUnusable(const TopicPartition & topic_partition) const { - if (!assignment.has_value() || assignment->empty()) + const auto consumer_in_wrong_state + = (stalled_status != StalledStatus::NOT_STALLED) && (stalled_status != StalledStatus::NO_MESSAGES_RETURNED); + const auto different_topic_partition = current == messages.end() + ? false + : (current->get_topic() != topic_partition.topic || current->get_partition() != topic_partition.partition_id); + return consumer_in_wrong_state || different_topic_partition; +} + +KafkaConsumer2::TopicPartitions const * KafkaConsumer2::getAssignment() const +{ + if (assignment.has_value()) { - LOG_TRACE(log, "Not assignned. Can't reset to last committed position."); - return; + return &*assignment; } - auto committed_offset = consumer->get_offsets_committed(consumer->get_assignment()); - consumer->assign(committed_offset); - LOG_TRACE(log, "{} Returned to committed position: {}", msg, committed_offset); + + return nullptr; +} + +void KafkaConsumer2::updateOffsets(const TopicPartitions & topic_partitions) +{ + // TODO(antaljanosbenjamin): Make sure topic_partitions and assignment is in sync. + cppkafka::TopicPartitionList original_topic_partitions; + original_topic_partitions.reserve(topic_partitions.size()); + std::transform( + topic_partitions.begin(), + topic_partitions.end(), + std::back_inserter(original_topic_partitions), + [](const TopicPartition & tp) { + return cppkafka::TopicPartition{tp.topic, tp.partition_id, tp.offset}; + }); + updateOffsets(original_topic_partitions); + needs_offset_update = false; + stalled_status = StalledStatus::NOT_STALLED; +} + +void KafkaConsumer2::updateOffsets(const cppkafka::TopicPartitionList & topic_partitions) +{ + queues.clear(); + // cppkafka itself calls assign(), but in order to detach the queues here we have to do the assignment manually. Later on we have to reassign the topic partitions with correct offsets. + consumer->assign(topic_partitions); + for (const auto & topic_partition : topic_partitions) + // This will also detach the partition queues from the consumer, thus the messages won't be forwarded without attaching them manually + queues.emplace( + TopicPartition{topic_partition.get_topic(), topic_partition.get_partition(), topic_partition.get_offset()}, + consumer->get_partition_queue(topic_partition)); } // it do the poll when needed -ReadBufferPtr KafkaConsumer2::consume() +ReadBufferPtr KafkaConsumer2::consume(const TopicPartition & topic_partition) { resetIfStopped(); - if (polledDataUnusable()) + if (polledDataUnusable(topic_partition)) return nullptr; if (hasMorePolledMessages()) return getNextMessage(); - if (intermediate_commit) - commit(); while (true) { - stalled_status = NO_MESSAGES_RETURNED; + stalled_status = StalledStatus::NO_MESSAGES_RETURNED; // we already wait enough for assignment in the past, // let's make polls shorter and not block other consumer @@ -393,27 +333,23 @@ ReadBufferPtr KafkaConsumer2::consume() // POLL_TIMEOUT_WO_ASSIGNMENT_MS (50ms) is 100% enough just to check if we got assignment // (see https://github.com/ClickHouse/ClickHouse/issues/11218) auto actual_poll_timeout_ms = (waited_for_assignment >= MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS) - ? std::min(POLL_TIMEOUT_WO_ASSIGNMENT_MS,poll_timeout) - : poll_timeout; + ? std::min(POLL_TIMEOUT_WO_ASSIGNMENT_MS, poll_timeout) + : poll_timeout; + + auto & queue_to_poll_from = queues[topic_partition]; + queue_to_poll_from.forward_to_queue(consumer->get_consumer_queue()); + SCOPE_EXIT({ queue_to_poll_from.disable_queue_forwarding(); }); /// Don't drop old messages immediately, since we may need them for virtual columns. - auto new_messages = consumer->poll_batch(batch_size, - std::chrono::milliseconds(actual_poll_timeout_ms)); + auto new_messages = consumer->poll_batch(batch_size, std::chrono::milliseconds(actual_poll_timeout_ms)); resetIfStopped(); - if (stalled_status == CONSUMER_STOPPED) + if (stalled_status == StalledStatus::CONSUMER_STOPPED) { return nullptr; } - else if (stalled_status == REBALANCE_HAPPENED) + else if (stalled_status == StalledStatus::REBALANCE_HAPPENED) { - if (!new_messages.empty()) - { - // we have polled something just after rebalance. - // we will not use current batch, so we need to return to last committed position - // otherwise we will continue polling from that position - resetToLastCommitted("Rewind last poll after rebalance."); - } return nullptr; } @@ -431,7 +367,7 @@ ReadBufferPtr KafkaConsumer2::consume() else { LOG_WARNING(log, "Can't get assignment. Will keep trying."); - stalled_status = NO_ASSIGNMENT; + stalled_status = StalledStatus::NO_ASSIGNMENT; return nullptr; } } @@ -450,8 +386,11 @@ ReadBufferPtr KafkaConsumer2::consume() { messages = std::move(new_messages); current = messages.begin(); - LOG_TRACE(log, "Polled batch of {} messages. Offsets position: {}", - messages.size(), consumer->get_offsets_position(consumer->get_assignment())); + LOG_TRACE( + log, + "Polled batch of {} messages. Offsets position: {}", + messages.size(), + consumer->get_offsets_position(consumer->get_assignment())); break; } } @@ -460,13 +399,13 @@ ReadBufferPtr KafkaConsumer2::consume() if (current == messages.end()) { LOG_ERROR(log, "Only errors left"); - stalled_status = ERRORS_RETURNED; + stalled_status = StalledStatus::ERRORS_RETURNED; return nullptr; } ProfileEvents::increment(ProfileEvents::KafkaMessagesPolled, messages.size()); - stalled_status = NOT_STALLED; + stalled_status = StalledStatus::NOT_STALLED; return getNextMessage(); } @@ -489,16 +428,18 @@ size_t KafkaConsumer2::filterMessageErrors() { assert(current == messages.begin()); - size_t skipped = std::erase_if(messages, [this](auto & message) - { - if (auto error = message.get_error()) + size_t skipped = std::erase_if( + messages, + [this](auto & message) { - ProfileEvents::increment(ProfileEvents::KafkaConsumerErrors); - LOG_ERROR(log, "Consumer error: {}", error); - return true; - } - return false; - }); + if (auto error = message.get_error()) + { + ProfileEvents::increment(ProfileEvents::KafkaConsumerErrors); + LOG_ERROR(log, "Consumer error: {}", error); + return true; + } + return false; + }); if (skipped) LOG_ERROR(log, "There were {} messages with an error", skipped); @@ -512,19 +453,8 @@ void KafkaConsumer2::resetIfStopped() // after block is formed (i.e. during copying data to MV / committing) we ignore stop attempts if (stopped) { - stalled_status = CONSUMER_STOPPED; + stalled_status = StalledStatus::CONSUMER_STOPPED; cleanUnprocessed(); } } - - -void KafkaConsumer2::storeLastReadMessageOffset() -{ - if (!isStalled()) - { - consumer->store_offset(*(current - 1)); - ++offsets_stored; - } -} - } diff --git a/src/Storages/Kafka/KafkaConsumer2.h b/src/Storages/Kafka/KafkaConsumer2.h index 6562a65a9b0..3341dc2c42f 100644 --- a/src/Storages/Kafka/KafkaConsumer2.h +++ b/src/Storages/Kafka/KafkaConsumer2.h @@ -1,11 +1,16 @@ #pragma once #include -#include #include - -#include #include +#include + +#include +#include +#include +#include + +#include namespace CurrentMetrics { @@ -25,6 +30,54 @@ using ConsumerPtr = std::shared_ptr; class KafkaConsumer2 { public: + static inline constexpr int INVALID_OFFSET = RD_KAFKA_OFFSET_INVALID; + static inline constexpr int BEGINNING_OFFSET = RD_KAFKA_OFFSET_BEGINNING; + static inline constexpr int END_OFFSET = RD_KAFKA_OFFSET_END; + + struct TopicPartition + { + String topic; + int32_t partition_id; + int64_t offset{INVALID_OFFSET}; + + bool operator==(const TopicPartition&) const = default; + bool operator<(const TopicPartition& other) const; + }; + + using TopicPartitions = std::vector; + + struct OnlyTopicNameAndPartitionIdHash + { + std::size_t operator()(const TopicPartition & tp) const + { + SipHash s; + s.update(tp.topic); + s.update(tp.partition_id); + return s.get64(); + } + }; + + struct OnlyTopicNameAndPartitionIdEquality + { + bool operator()(const TopicPartition & lhs, const TopicPartition & rhs) const + { + return lhs.topic == rhs.topic && lhs.partition_id == rhs.partition_id; + } + }; + + struct TopicPartitionCount + { + String topic; + size_t partition_count; + }; + + using TopicPartitionCounts = std::vector; + + // struct AssignmentChanges { + // TopicPartitions revoked_partitions; + // TopicPartitions new_partitions; + // }; + KafkaConsumer2( ConsumerPtr consumer_, Poco::Logger * log_, @@ -36,30 +89,34 @@ public: ); ~KafkaConsumer2(); - void commit(); // Commit all processed messages. - void subscribe(); // Subscribe internal consumer to topics. - void unsubscribe(); // Unsubscribe internal consumer in case of failure. + + void pollEvents(); + + TopicPartitionCounts getPartitionCounts() const; auto pollTimeout() const { return poll_timeout; } inline bool hasMorePolledMessages() const { - return (stalled_status == NOT_STALLED) && (current != messages.end()); + return (stalled_status == StalledStatus::NOT_STALLED) && (current != messages.end()); } - inline bool polledDataUnusable() const - { - return (stalled_status != NOT_STALLED) && (stalled_status != NO_MESSAGES_RETURNED); - } + inline bool isStalled() const { return stalled_status != StalledStatus::NOT_STALLED; } - inline bool isStalled() const { return stalled_status != NOT_STALLED; } + bool polledDataUnusable(const TopicPartition & topic_partition) const; - void storeLastReadMessageOffset(); - void resetToLastCommitted(const char * msg); + TopicPartitions const * getAssignment() const; + + // As the main source of offsets is not Kafka, the offsets needs to pushed to the consumer from outside + bool needsOffsetUpdate() const { return needs_offset_update; } + + // Returns true if it received new assignment and could update the internal state accordingly, false otherwise + void updateOffsets(const TopicPartitions & topic_partitions); /// Polls batch of messages from Kafka and returns read buffer containing the next message or /// nullptr when there are no messages to process. - ReadBufferPtr consume(); + /// TODO(antaljanosbenjamin): add batch size param + ReadBufferPtr consume(const TopicPartition & topic_partition); // Return values for the message that's being read. String currentTopic() const { return current[-1].get_topic(); } @@ -74,7 +131,7 @@ private: using Messages = std::vector; CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaConsumers}; - enum StalledStatus + enum class StalledStatus { NOT_STALLED, NO_MESSAGES_RETURNED, @@ -90,7 +147,7 @@ private: const size_t poll_timeout = 0; size_t offsets_stored = 0; - StalledStatus stalled_status = NO_MESSAGES_RETURNED; + StalledStatus stalled_status = StalledStatus::NO_MESSAGES_RETURNED; bool intermediate_commit = true; size_t waited_for_assignment = 0; @@ -102,7 +159,9 @@ private: Messages::const_iterator current; // order is important, need to be destructed before consumer - std::optional assignment; + std::optional assignment; + bool needs_offset_update{false}; + std::unordered_map queues; const Names topics; void drain(); @@ -111,6 +170,8 @@ private: /// Return number of messages with an error. size_t filterMessageErrors(); ReadBufferPtr getNextMessage(); + + void updateOffsets(const cppkafka::TopicPartitionList & topic_partitions); }; } diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 748090165d7..3826857c24e 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -31,6 +31,7 @@ class ASTStorage; M(StreamingHandleErrorMode, kafka_handle_error_mode, StreamingHandleErrorMode::DEFAULT, "How to handle errors for Kafka engine. Possible values: default (throw an exception after rabbitmq_skip_broken_messages broken messages), stream (save broken messages and errors in virtual columns _raw_message, _error).", 0) \ M(Bool, kafka_commit_on_select, false, "Commit messages when select query is made", 0) \ M(UInt64, kafka_max_rows_per_message, 1, "The maximum number of rows produced in one kafka message for row-based formats.", 0) \ + /* TODO(antaljanosbenjamin): Probably this shouldn't be here, but only read as an argument */ \ M(String, kafka_keeper_path, "", "TODO(antaljanosbenjamin)", 0) \ #define OBSOLETE_KAFKA_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/Kafka/KafkaSource2.cpp b/src/Storages/Kafka/KafkaSource2.cpp deleted file mode 100644 index 3e14c57e8e2..00000000000 --- a/src/Storages/Kafka/KafkaSource2.cpp +++ /dev/null @@ -1,303 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include - -#include - -namespace ProfileEvents -{ - extern const Event KafkaMessagesRead; - extern const Event KafkaMessagesFailed; - extern const Event KafkaRowsRead; - extern const Event KafkaRowsRejected; -} - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -// with default poll timeout (500ms) it will give about 5 sec delay for doing 10 retries -// when selecting from empty topic -const auto MAX_FAILED_POLL_ATTEMPTS = 10; - -KafkaSource2::KafkaSource2( - StorageKafka2 & storage_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - const Names & columns, - Poco::Logger * log_, - size_t max_block_size_, - bool commit_in_suffix_) - : ISource(storage_snapshot_->getSampleBlockForColumns(columns)) - , storage(storage_) - , storage_snapshot(storage_snapshot_) - , context(context_) - , column_names(columns) - , log(log_) - , max_block_size(max_block_size_) - , commit_in_suffix(commit_in_suffix_) - , non_virtual_header(storage_snapshot->metadata->getSampleBlockNonMaterialized()) - , virtual_header(storage_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames())) - , handle_error_mode(storage.getHandleKafkaErrorMode()) -{ -} - -KafkaSource2::~KafkaSource2() -{ - if (!consumer) - return; - - if (broken) - consumer->unsubscribe(); - - storage.pushConsumer(consumer); -} - -bool KafkaSource2::checkTimeLimit() const -{ - if (max_execution_time != 0) - { - auto elapsed_ns = total_stopwatch.elapsed(); - - if (elapsed_ns > static_cast(max_execution_time.totalMicroseconds()) * 1000) - return false; - } - - return true; -} - -Chunk KafkaSource2::generateImpl() -{ - if (!consumer) - { - auto timeout = std::chrono::milliseconds(context->getSettingsRef().kafka_max_wait_ms.totalMilliseconds()); - consumer = storage.popConsumer(timeout); - - if (!consumer) - return {}; - - consumer->subscribe(); - - broken = true; - } - - if (is_finished) - return {}; - - is_finished = true; - // now it's one-time usage InputStream - // one block of the needed size (or with desired flush timeout) is formed in one internal iteration - // otherwise external iteration will reuse that and logic will became even more fuzzy - MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); - - auto put_error_to_stream = handle_error_mode == HandleKafkaErrorMode::STREAM; - - EmptyReadBuffer empty_buf; - auto input_format = FormatFactory::instance().getInput( - storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size, std::nullopt, 1); - - std::optional exception_message; - size_t total_rows = 0; - size_t failed_poll_attempts = 0; - - auto on_error = [&](const MutableColumns & result_columns, Exception & e) - { - ProfileEvents::increment(ProfileEvents::KafkaMessagesFailed); - - if (put_error_to_stream) - { - exception_message = e.message(); - for (const auto & column : result_columns) - { - // read_kafka_message could already push some rows to result_columns - // before exception, we need to fix it. - auto cur_rows = column->size(); - if (cur_rows > total_rows) - column->popBack(cur_rows - total_rows); - - // all data columns will get default value in case of error - column->insertDefault(); - } - - return 1; - } - else - { - e.addMessage("while parsing Kafka message (topic: {}, partition: {}, offset: {})'", - consumer->currentTopic(), consumer->currentPartition(), consumer->currentOffset()); - throw std::move(e); - } - }; - - StreamingFormatExecutor executor(non_virtual_header, input_format, std::move(on_error)); - - while (true) - { - size_t new_rows = 0; - exception_message.reset(); - if (auto buf = consumer->consume()) - { - ProfileEvents::increment(ProfileEvents::KafkaMessagesRead); - new_rows = executor.execute(*buf); - } - - if (new_rows) - { - // In read_kafka_message(), KafkaConsumer::nextImpl() - // will be called, that may make something unusable, i.e. clean - // KafkaConsumer::messages, which is accessed from - // KafkaConsumer::currentTopic() (and other helpers). - if (consumer->isStalled()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Polled messages became unusable"); - - ProfileEvents::increment(ProfileEvents::KafkaRowsRead, new_rows); - - consumer->storeLastReadMessageOffset(); - - auto topic = consumer->currentTopic(); - auto key = consumer->currentKey(); - auto offset = consumer->currentOffset(); - auto partition = consumer->currentPartition(); - auto timestamp_raw = consumer->currentTimestamp(); - auto header_list = consumer->currentHeaderList(); - - Array headers_names; - Array headers_values; - - if (!header_list.empty()) - { - headers_names.reserve(header_list.size()); - headers_values.reserve(header_list.size()); - for (const auto & header : header_list) - { - headers_names.emplace_back(header.get_name()); - headers_values.emplace_back(static_cast(header.get_value())); - } - } - - for (size_t i = 0; i < new_rows; ++i) - { - virtual_columns[0]->insert(topic); - virtual_columns[1]->insert(key); - virtual_columns[2]->insert(offset); - virtual_columns[3]->insert(partition); - if (timestamp_raw) - { - auto ts = timestamp_raw->get_timestamp(); - virtual_columns[4]->insert(std::chrono::duration_cast(ts).count()); - virtual_columns[5]->insert(DecimalField(std::chrono::duration_cast(ts).count(),3)); - } - else - { - virtual_columns[4]->insertDefault(); - virtual_columns[5]->insertDefault(); - } - virtual_columns[6]->insert(headers_names); - virtual_columns[7]->insert(headers_values); - if (put_error_to_stream) - { - if (exception_message) - { - auto payload = consumer->currentPayload(); - virtual_columns[8]->insert(payload); - virtual_columns[9]->insert(*exception_message); - } - else - { - virtual_columns[8]->insertDefault(); - virtual_columns[9]->insertDefault(); - } - } - } - - total_rows = total_rows + new_rows; - } - else if (consumer->polledDataUnusable()) - { - break; - } - else if (consumer->isStalled()) - { - ++failed_poll_attempts; - } - else - { - // We came here in case of tombstone (or sometimes zero-length) messages, and it is not something abnormal - // TODO: it seems like in case of put_error_to_stream=true we may need to process those differently - // currently we just skip them with note in logs. - consumer->storeLastReadMessageOffset(); - LOG_DEBUG(log, "Parsing of message (topic: {}, partition: {}, offset: {}) return no rows.", consumer->currentTopic(), consumer->currentPartition(), consumer->currentOffset()); - } - - if (!consumer->hasMorePolledMessages() - && (total_rows >= max_block_size || !checkTimeLimit() || failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS)) - { - break; - } - } - - if (total_rows == 0) - { - return {}; - } - else if (consumer->polledDataUnusable()) - { - // the rows were counted already before by KafkaRowsRead, - // so let's count the rows we ignore separately - // (they will be retried after the rebalance) - ProfileEvents::increment(ProfileEvents::KafkaRowsRejected, total_rows); - return {}; - } - - /// MATERIALIZED columns can be added here, but I think - // they are not needed here: - // and it's misleading to use them here, - // as columns 'materialized' that way stays 'ephemeral' - // i.e. will not be stored anythere - // If needed any extra columns can be added using DEFAULT they can be added at MV level if needed. - - auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns()); - auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); - - for (const auto & column : virtual_block.getColumnsWithTypeAndName()) - result_block.insert(column); - - auto converting_dag = ActionsDAG::makeConvertingActions( - result_block.cloneEmpty().getColumnsWithTypeAndName(), - getPort().getHeader().getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name); - - auto converting_actions = std::make_shared(std::move(converting_dag)); - converting_actions->execute(result_block); - - return Chunk(result_block.getColumns(), result_block.rows()); -} - -Chunk KafkaSource2::generate() -{ - auto chunk = generateImpl(); - if (!chunk && commit_in_suffix) - commit(); - - return chunk; -} - -void KafkaSource2::commit() -{ - if (!consumer) - return; - - consumer->commit(); - - broken = false; -} - -} diff --git a/src/Storages/Kafka/KafkaSource2.h b/src/Storages/Kafka/KafkaSource2.h deleted file mode 100644 index 0a49001a686..00000000000 --- a/src/Storages/Kafka/KafkaSource2.h +++ /dev/null @@ -1,64 +0,0 @@ -#pragma once - -#include - -#include -#include -#include - - -namespace Poco -{ - class Logger; -} -namespace DB -{ - -class KafkaSource2 : public ISource -{ -public: - KafkaSource2( - StorageKafka2 & storage_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - const Names & columns, - Poco::Logger * log_, - size_t max_block_size_, - bool commit_in_suffix = false); - ~KafkaSource2() override; - - String getName() const override { return storage.getName(); } - - Chunk generate() override; - - void commit(); - bool isStalled() const { return !consumer || consumer->isStalled(); } - - void setTimeLimit(Poco::Timespan max_execution_time_) { max_execution_time = max_execution_time_; } - -private: - StorageKafka2 & storage; - StorageSnapshotPtr storage_snapshot; - ContextPtr context; - Names column_names; - Poco::Logger * log; - UInt64 max_block_size; - - KafkaConsumer2Ptr consumer; - bool broken = true; - bool is_finished = false; - bool commit_in_suffix; - - const Block non_virtual_header; - const Block virtual_header; - const HandleKafkaErrorMode handle_error_mode; - - Poco::Timespan max_execution_time = 0; - Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; - - bool checkTimeLimit() const; - - Chunk generateImpl(); -}; - -} diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 7cb2a38067a..a1e7a81b792 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -9,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -18,39 +18,47 @@ #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 #include #include - -#include "Storages/ColumnDefault.h" +#include "Storages/Kafka/KafkaConsumer2.h" #include "config_version.h" -#include -#include #if USE_KRB5 # include #endif // USE_KRB5 +#include +#include +#include +#include + +#include + namespace CurrentMetrics { extern const Metric KafkaBackgroundReads; @@ -62,6 +70,10 @@ namespace ProfileEvents { extern const Event KafkaDirectReads; extern const Event KafkaBackgroundReads; +extern const Event KafkaMessagesRead; +extern const Event KafkaMessagesFailed; +extern const Event KafkaRowsRead; +extern const Event KafkaRowsRejected; extern const Event KafkaWrites; } @@ -78,9 +90,11 @@ namespace ErrorCodes namespace { - const auto RESCHEDULE_MS = 500; - const auto CLEANUP_TIMEOUT_MS = 3000; - const auto MAX_THREAD_WORK_DURATION_MS = 60000; // once per minute leave do reschedule (we can't lock threads in pool forever) + constexpr auto RESCHEDULE_MS = 500; + // const auto CLEANUP_TIMEOUT_MS = 3000; + constexpr auto MAX_THREAD_WORK_DURATION_MS = 60000; // once per minute leave do reschedule (we can't lock threads in pool forever) + + constexpr auto MAX_FAILED_POLL_ATTEMPTS = 10; } StorageKafka2::StorageKafka2( @@ -91,6 +105,7 @@ StorageKafka2::StorageKafka2( const String & collection_name_) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) + , keeper(getContext()->getZooKeeper()) , kafka_settings(std::move(kafka_settings_)) , macros_info{.table_id = table_id_} , topics(parseTopics(getContext()->getMacros()->expand(kafka_settings->kafka_topic_list.value, macros_info))) @@ -111,6 +126,12 @@ StorageKafka2::StorageKafka2( , thread_per_consumer(kafka_settings->kafka_thread_per_consumer.value) , collection_name(collection_name_) { + if (kafka_settings->kafka_num_consumers != 1) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Multiple consumers not yet implemented!"); + + if (thread_per_consumer) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "The new Kafka storage cannot use multiple threads yet!"); + if (kafka_settings->kafka_handle_error_mode == HandleKafkaErrorMode::STREAM) { kafka_settings->input_format_allow_errors_num = 0; @@ -119,6 +140,7 @@ StorageKafka2::StorageKafka2( StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); + auto task_count = thread_per_consumer ? num_consumers : 1; for (size_t i = 0; i < task_count; ++i) { @@ -178,44 +200,15 @@ String StorageKafka2::getDefaultClientId(const StorageID & table_id_) Pipe StorageKafka2::read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, + const Names & /*column_names */, + const StorageSnapshotPtr & /* storage_snapshot */, SelectQueryInfo & /* query_info */, - ContextPtr local_context, + ContextPtr /* local_context */, QueryProcessingStage::Enum /* processed_stage */, size_t /* max_block_size */, size_t /* num_streams */) { - if (num_created_consumers == 0) - return {}; - - if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) - throw Exception( - ErrorCodes::QUERY_NOT_ALLOWED, "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); - - if (mv_attached) - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageKafka with attached materialized views"); - - ProfileEvents::increment(ProfileEvents::KafkaDirectReads); - - /// Always use all consumers at once, otherwise SELECT may not read messages from all partitions. - Pipes pipes; - pipes.reserve(num_created_consumers); - auto modified_context = Context::createCopy(local_context); - modified_context->applySettingsChanges(settings_adjustments); - - // Claim as many consumers as requested, but don't block - for (size_t i = 0; i < num_created_consumers; ++i) - { - /// Use block size of 1, otherwise LIMIT won't work properly as it will buffer excess messages in the last block - /// TODO: probably that leads to awful performance. - /// FIXME: seems that doesn't help with extra reading and committing unprocessed messages. - pipes.emplace_back(std::make_shared( - *this, storage_snapshot, modified_context, column_names, log, 1, kafka_settings->kafka_commit_on_select)); - } - - LOG_DEBUG(log, "Starting reading {} streams", pipes.size()); - return Pipe::unitePipes(std::move(pipes)); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "You cannot read from the new Kafka storage!"); } @@ -260,7 +253,7 @@ void StorageKafka2::startup() { try { - pushConsumer(createConsumer(i)); + consumers.push_back(ConsumerAndAssignmentInfo{.consumer = createConsumer(i)}); ++num_created_consumers; } catch (const cppkafka::Exception &) @@ -269,6 +262,7 @@ void StorageKafka2::startup() } } + createKeeperNodes(consumers.front().consumer); // Start the reader thread for (auto & task : tasks) { @@ -289,49 +283,15 @@ void StorageKafka2::shutdown() } LOG_TRACE(log, "Closing consumers"); - for (size_t i = 0; i < num_created_consumers; ++i) - auto consumer = popConsumer(); + consumers.clear(); LOG_TRACE(log, "Consumers closed"); - - rd_kafka_wait_destroyed(CLEANUP_TIMEOUT_MS); } - -void StorageKafka2::pushConsumer(KafkaConsumer2Ptr consumer) +void StorageKafka2::drop() { - std::lock_guard lock(mutex); - consumers.push_back(consumer); - semaphore.set(); - CurrentMetrics::sub(CurrentMetrics::KafkaConsumersInUse, 1); + getZooKeeper().removeRecursive(kafka_settings->kafka_keeper_path); } - -KafkaConsumer2Ptr StorageKafka2::popConsumer() -{ - return popConsumer(std::chrono::milliseconds::zero()); -} - - -KafkaConsumer2Ptr StorageKafka2::popConsumer(std::chrono::milliseconds timeout) -{ - // Wait for the first free buffer - if (timeout == std::chrono::milliseconds::zero()) - semaphore.wait(); - else - { - if (!semaphore.tryWait(timeout.count())) - return nullptr; - } - - // Take the first available buffer from the list - std::lock_guard lock(mutex); - auto consumer = consumers.back(); - consumers.pop_back(); - CurrentMetrics::add(CurrentMetrics::KafkaConsumersInUse, 1); - return consumer; -} - - KafkaConsumer2Ptr StorageKafka2::createConsumer(size_t consumer_number) { cppkafka::Configuration conf; @@ -350,9 +310,7 @@ KafkaConsumer2Ptr StorageKafka2::createConsumer(size_t consumer_number) conf.set("client.software.version", VERSION_DESCRIBE); conf.set("auto.offset.reset", "earliest"); // If no offset stored for this group, read all messages from the start - // that allows to prevent fast draining of the librdkafka queue - // during building of single insert block. Improves performance - // significantly, but may lead to bigger memory consumption. + // that allows to prevent fast draining of the librdkafka queue during building of single insert block. Improves performance significantly, but may lead to bigger memory consumption. size_t default_queued_min_messages = 100000; // we don't want to decrease the default conf.set("queued.min.messages", std::max(getMaxBlockSize(), default_queued_min_messages)); @@ -370,10 +328,12 @@ KafkaConsumer2Ptr StorageKafka2::createConsumer(size_t consumer_number) /// NOTE: we pass |stream_cancelled| by reference here, so the buffers should not outlive the storage. if (thread_per_consumer) { + // call subscribe; auto & stream_cancelled = tasks[consumer_number]->stream_cancelled; return std::make_shared( consumer_impl, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), intermediate_commit, stream_cancelled, topics); } + return std::make_shared( consumer_impl, log, @@ -487,16 +447,16 @@ void StorageKafka2::updateConfiguration(cppkafka::Configuration & kafka_config) // No need to add any prefix, messages can be distinguished kafka_config.set_log_callback( - [this](cppkafka::KafkaHandleBase &, int level, const std::string & facility, const std::string & message) + [this](cppkafka::KafkaHandleBase &, int /*level*/, const std::string & facility, const std::string & message) { - auto [poco_level, client_logs_level] = parseSyslogLevel(level); + auto [poco_level, client_logs_level] = parseSyslogLevel(1); LOG_IMPL(log, client_logs_level, poco_level, "[rdk:{}] {}", facility, message); }); // Configure interceptor to change thread name // // TODO: add interceptors support into the cppkafka. - // XXX: rdkafka uses pthread_set_name_np(), but glibc-compatibliity overrides it to noop. + // XXX: rdkafka uses pthread_set_name_np(), but glibc-compatiblity overrides it to noop. { // This should be safe, since we wait the rdkafka object anyway. void * self = static_cast(this); @@ -542,6 +502,343 @@ bool StorageKafka2::checkDependencies(const StorageID & table_id) return true; } +namespace +{ + const std::string lock_file_name{"lock"}; + const std::string commit_file_name{"committed"}; + const std::string intent_file_name{"intention"}; + + std::optional getNumber(zkutil::ZooKeeper & keeper, const std::string & path) + { + std::string result; + if (!keeper.tryGet(path, result)) + return std::nullopt; + + return DB::parse(result); + } +} + +void StorageKafka2::createKeeperNodes(const KafkaConsumer2Ptr & consumer) +{ + // TODO(antaljanosbenjamin): check config with other StorageKafkas + const auto & keeper_path = kafka_settings->kafka_keeper_path.value; + + auto & keeper_ref = getZooKeeper(); + + if (keeper_ref.exists(keeper_path)) + { + return; + } + + keeper_ref.createAncestors(keeper_path); + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest(keeper_path, "", zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest(keeper_path + "/topics", "", zkutil::CreateMode::Persistent)); + + const auto topics_prefix = keeper_path + "/topics/"; + + const auto topic_partition_counts = consumer->getPartitionCounts(); + for (const auto & topic_partition_count : topic_partition_counts) + { + ops.emplace_back(zkutil::makeCreateRequest(topics_prefix + topic_partition_count.topic, "", zkutil::CreateMode::Persistent)); + ops.emplace_back( + zkutil::makeCreateRequest(topics_prefix + topic_partition_count.topic + "/partitions", "", zkutil::CreateMode::Persistent)); + const auto partitions_prefix = topics_prefix + topic_partition_count.topic + "/partitions/"; + // TODO(antaljanosbenjamin): handle changing number of partitions + for (auto partition_id{0U}; partition_id < topic_partition_count.partition_count; ++partition_id) + ops.emplace_back(zkutil::makeCreateRequest(partitions_prefix + toString(partition_id), "", zkutil::CreateMode::Persistent)); + } + + + Coordination::Responses responses; + const auto code = keeper_ref.tryMulti(ops, responses); + if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNODEEXISTS) + { + zkutil::KeeperMultiException::check(code, ops, responses); + } +} + +std::optional StorageKafka2::lockTopicPartitions(const TopicPartitions & topic_partitions) +{ + // TODO(antaljanosbenjamin): Review this function with somebody who know keeper better than me + const auto uuid_as_string = toString(uuid); + + std::vector topic_partition_paths; + topic_partition_paths.reserve(topic_partitions.size()); + for (const auto & topic_partition : topic_partitions) + { + topic_partition_paths.emplace_back(getTopicPartitionPath(topic_partition)); + } + + Coordination::Requests ops; + + // for (const auto & topic_partition_path : topic_partition_paths) + // ops.push_back(zkutil::makeCheckRequest(topic_partition_path + lock_file_name, -1)); + + for (const auto & topic_partition_path : topic_partition_paths) + ops.push_back(zkutil::makeCreateRequest(topic_partition_path + lock_file_name, uuid_as_string, zkutil::CreateMode::Ephemeral)); + + bool success = false; + for (auto try_count{0}; try_count < 10; ++try_count) + { + Coordination::Responses responses; + // TODO(antaljanosbenjamin): this can go wrong if we start a new session simultaneously from multiple threads. + auto & keeper_ref = getZooKeeper(); + + if (const auto code = keeper_ref.tryMulti(ops, responses); code == Coordination::Error::ZOK) + { + success = true; + break; + } + else + { + zkutil::KeeperMultiException::check(code, ops, responses); + } + + // TODO(antaljanosbenjamin): Probably handle the most common errors + for (const auto & topic_partition_path : topic_partition_paths) + keeper_ref.waitForDisappear(topic_partition_path + lock_file_name); + } + + if (!success) + return std::nullopt; + + + // We have the locks + TopicPartitionLocks locks; + { + auto & keeper_ref = getZooKeeper(); + auto tp_it = topic_partitions.begin(); + auto path_it = topic_partition_paths.begin(); + for (; tp_it != topic_partitions.end(); ++tp_it, ++path_it) + { + using zkutil::EphemeralNodeHolder; + LockedTopicPartitionInfo lock_info{.lock = EphemeralNodeHolder::existing(*path_it + lock_file_name, keeper_ref)}; + + lock_info.committed_offset = getNumber(keeper_ref, *path_it + commit_file_name); + lock_info.intent_size = getNumber(keeper_ref, *path_it + intent_file_name); + + + locks.emplace(TopicPartition(*tp_it), std::move(lock_info)); + } + } + + return locks; +} + + +void StorageKafka2::saveCommittedOffset(const TopicPartition & topic_partition, int64_t committed_offset) +{ + const auto partition_prefix = getTopicPartitionPath(topic_partition); + auto & keeper_ref = getZooKeeper(); + keeper_ref.createOrUpdate(partition_prefix + commit_file_name, toString(committed_offset), zkutil::CreateMode::Persistent); + // This is best effort, if it fails we will try to remove in the next round + keeper_ref.tryRemove(partition_prefix + intent_file_name, -1); +} + +void StorageKafka2::saveIntent(const TopicPartition & topic_partition, int64_t intent) +{ + getZooKeeper().createOrUpdate( + getTopicPartitionPath(topic_partition) + intent_file_name, toString(intent), zkutil::CreateMode::Persistent); +} + + +StorageKafka2::PolledBatchInfo +StorageKafka2::pollConsumer(KafkaConsumer2 & consumer, const TopicPartition & topic_partition, const ContextPtr & modified_context) +{ + PolledBatchInfo batch_info; + auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); + Block non_virtual_header(storage_snapshot->metadata->getSampleBlockNonMaterialized()); + Block virtual_header(storage_snapshot->getSampleBlockForColumns(getVirtualColumnNames())); + + // now it's one-time usage InputStream + // one block of the needed size (or with desired flush timeout) is formed in one internal iteration + // otherwise external iteration will reuse that and logic will became even more fuzzy + MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); + + auto put_error_to_stream = kafka_settings->kafka_handle_error_mode == HandleKafkaErrorMode::STREAM; + + EmptyReadBuffer empty_buf; + auto input_format = FormatFactory::instance().getInput( + getFormatName(), empty_buf, non_virtual_header, modified_context, getMaxBlockSize(), std::nullopt, 1); + + std::optional exception_message; + size_t total_rows = 0; + size_t failed_poll_attempts = 0; + + auto on_error = [&](const MutableColumns & result_columns, Exception & e) + { + ProfileEvents::increment(ProfileEvents::KafkaMessagesFailed); + + if (put_error_to_stream) + { + exception_message = e.message(); + for (const auto & column : result_columns) + { + // read_kafka_message could already push some rows to result_columns + // before exception, we need to fix it. + auto cur_rows = column->size(); + if (cur_rows > total_rows) + column->popBack(cur_rows - total_rows); + + // all data columns will get default value in case of error + column->insertDefault(); + } + + return 1; + } + else + { + e.addMessage( + "while parsing Kafka message (topic: {}, partition: {}, offset: {})'", + consumer.currentTopic(), + consumer.currentPartition(), + consumer.currentOffset()); + throw std::move(e); + } + }; + + StreamingFormatExecutor executor(non_virtual_header, input_format, std::move(on_error)); + + while (true) + { + size_t new_rows = 0; + exception_message.reset(); + if (auto buf = consumer.consume(topic_partition)) + { + ProfileEvents::increment(ProfileEvents::KafkaMessagesRead); + new_rows = executor.execute(*buf); + } + + if (new_rows) + { + // In read_kafka_message(), KafkaConsumer::nextImpl() + // will be called, that may make something unusable, i.e. clean + // KafkaConsumer::messages, which is accessed from + // KafkaConsumer::currentTopic() (and other helpers). + if (consumer.isStalled()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Polled messages became unusable"); + + ProfileEvents::increment(ProfileEvents::KafkaRowsRead, new_rows); + + const auto & header_list = consumer.currentHeaderList(); + + Array headers_names; + Array headers_values; + + if (!header_list.empty()) + { + headers_names.reserve(header_list.size()); + headers_values.reserve(header_list.size()); + for (const auto & header : header_list) + { + headers_names.emplace_back(header.get_name()); + headers_values.emplace_back(static_cast(header.get_value())); + } + } + + for (size_t i = 0; i < new_rows; ++i) + { + virtual_columns[0]->insert(consumer.currentTopic()); + virtual_columns[1]->insert(consumer.currentKey()); + virtual_columns[2]->insert(consumer.currentOffset()); + virtual_columns[3]->insert(consumer.currentPartition()); + + + auto timestamp_raw = consumer.currentTimestamp(); + if (timestamp_raw) + { + auto ts = timestamp_raw->get_timestamp(); + virtual_columns[4]->insert(std::chrono::duration_cast(ts).count()); + virtual_columns[5]->insert( + DecimalField(std::chrono::duration_cast(ts).count(), 3)); + } + else + { + virtual_columns[4]->insertDefault(); + virtual_columns[5]->insertDefault(); + } + virtual_columns[6]->insert(headers_names); + virtual_columns[7]->insert(headers_values); + if (put_error_to_stream) + { + if (exception_message) + { + virtual_columns[8]->insert(consumer.currentPayload()); + virtual_columns[9]->insert(*exception_message); + } + else + { + virtual_columns[8]->insertDefault(); + virtual_columns[9]->insertDefault(); + } + } + } + + total_rows = total_rows + new_rows; + batch_info.last_offset = consumer.currentOffset(); + } + else if (consumer.polledDataUnusable(topic_partition)) + { + break; + } + else if (consumer.isStalled()) + { + ++failed_poll_attempts; + } + else + { + // We came here in case of tombstone (or sometimes zero-length) messages, and it is not something abnormal + // TODO: it seems like in case of put_error_to_stream=true we may need to process those differently + // currently we just skip them with note in logs. + LOG_DEBUG( + log, + "Parsing of message (topic: {}, partition: {}, offset: {}) return no rows.", + consumer.currentTopic(), + consumer.currentPartition(), + consumer.currentOffset()); + } + + if (!consumer.hasMorePolledMessages() + && (total_rows >= kafka_settings->kafka_max_block_size || /*!checkTimeLimit() + ||*/ + failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS)) + { + break; + } + } + + if (total_rows == 0) + { + return {}; + } + else if (consumer.polledDataUnusable(topic_partition)) + { + // the rows were counted already before by KafkaRowsRead, + // so let's count the rows we ignore separately + // (they will be retried after the rebalance) + ProfileEvents::increment(ProfileEvents::KafkaRowsRejected, total_rows); + return {}; + } + + /// MATERIALIZED columns can be added here, but I think + // they are not needed here: + // and it's misleading to use them here, + // as columns 'materialized' that way stays 'ephemeral' + // i.e. will not be stored anythere + // If needed any extra columns can be added using DEFAULT they can be added at MV level if needed. + + auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns()); + auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); + + for (const auto & column : virtual_block.getColumnsWithTypeAndName()) + result_block.insert(column); + + batch_info.blocks.emplace_back(std::move(result_block)); + return batch_info; +} + void StorageKafka2::threadFunc(size_t idx) { assert(idx < tasks.size()); @@ -566,7 +863,7 @@ void StorageKafka2::threadFunc(size_t idx) LOG_DEBUG(log, "Started streaming to {} attached views", num_views); // Exit the loop & reschedule if some stream stalled - auto some_stream_is_stalled = streamToViews(); + auto some_stream_is_stalled = streamToViews(idx); if (some_stream_is_stalled) { LOG_TRACE(log, "Stream(s) stalled. Reschedule."); @@ -595,9 +892,18 @@ void StorageKafka2::threadFunc(size_t idx) task->holder->scheduleAfter(RESCHEDULE_MS); } - -bool StorageKafka2::streamToViews() +bool StorageKafka2::streamToViews(size_t idx) { + // What to do? + // 1. Select a topic partition to consume from + // 2. Do a casual poll for every other consumer to keep them alive + // 3. Get the necessary data from Keeper + // 4. Get the corresponding consumer + // 5. Pull messages + // 6. Create a BlockList from it + // 7. Execute the pipeline + // 8. Write the offset to Keeper + Stopwatch watch; auto table_id = getStorageID(); @@ -608,14 +914,56 @@ bool StorageKafka2::streamToViews() CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaBackgroundReads}; ProfileEvents::increment(ProfileEvents::KafkaBackgroundReads); - auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); + auto & consumer_info = consumers[idx]; + auto & consumer = consumer_info.consumer; + // To keep the consumer alive + + LOG_TRACE(log, "Polling consumer #{} for events", idx); + consumer->pollEvents(); + if (nullptr == consumer->getAssignment()) + return true; + + + LOG_TRACE(log, "Consumer #{} has assignment", idx); + + if (consumer->needsOffsetUpdate()) + { + LOG_TRACE(log, "Consumer #{} needs update offset", idx); + consumer_info.consume_from_topic_partition_index = 0; + + consumer_info.locks.clear(); + consumer_info.topic_partitions.clear(); + + if (const auto * current_assignment = consumer->getAssignment(); nullptr != current_assignment) + { + auto maybe_locks = lockTopicPartitions(*current_assignment); + + if (!maybe_locks.has_value()) + { + // TODO(antaljanosbenjamin): signal this somehow to caller, maybe wait a bit longer. + return true; + } + + consumer_info.locks = std::move(*maybe_locks); + + consumer_info.topic_partitions.reserve(current_assignment->size()); + for (const auto& topic_partition : *current_assignment) { + TopicPartition topic_partition_copy{topic_partition}; + if( const auto & maybe_committed_offset = consumer_info.locks.at(topic_partition).committed_offset; maybe_committed_offset.has_value()) + topic_partition_copy.offset = *maybe_committed_offset + 1; + else + topic_partition_copy.offset = KafkaConsumer2::BEGINNING_OFFSET; + consumer_info.topic_partitions.push_back(std::move(topic_partition_copy)); + + } + } + consumer_info.consumer->updateOffsets(consumer_info.topic_partitions); + } // Create an INSERT query for streaming data auto insert = std::make_shared(); insert->table_id = table_id; - size_t block_size = getMaxBlockSize(); - auto kafka_context = Context::createCopy(getContext()); kafka_context->makeQueryContext(); kafka_context->applySettingsChanges(settings_adjustments); @@ -625,59 +973,80 @@ bool StorageKafka2::streamToViews() InterpreterInsertQuery interpreter(insert, kafka_context, false, true, true); auto block_io = interpreter.execute(); - // Create a stream for each consumer and join them in a union stream - std::vector> sources; - Pipes pipes; + auto & topic_partition = consumer_info.topic_partitions[consumer_info.consume_from_topic_partition_index]; + LOG_TRACE( + log, + "Consumer #{} will fetch {}:{} (consume_from_topic_partition_index is {})", + idx, + topic_partition.topic, + topic_partition.partition_id, + consumer_info.consume_from_topic_partition_index); + consumer_info.consume_from_topic_partition_index + = (consumer_info.consume_from_topic_partition_index + 1) % consumer_info.topic_partitions.size(); - auto stream_count = thread_per_consumer ? 1 : num_created_consumers; - sources.reserve(stream_count); - pipes.reserve(stream_count); - for (size_t i = 0; i < stream_count; ++i) + auto [blocks, last_offset] = pollConsumer(*consumer_info.consumer, topic_partition, kafka_context); + + if (blocks.empty()) { - auto source = std::make_shared( - *this, storage_snapshot, kafka_context, block_io.pipeline.getHeader().getNames(), log, block_size, false); - sources.emplace_back(source); - pipes.emplace_back(source); - - // Limit read batch to maximum block size to allow DDL - StreamLocalLimits limits; - - Poco::Timespan max_execution_time = kafka_settings->kafka_flush_interval_ms.changed - ? kafka_settings->kafka_flush_interval_ms - : getContext()->getSettingsRef().stream_flush_interval_ms; - - source->setTimeLimit(max_execution_time); + LOG_TRACE(log, "Consumer #{} didn't get any messages", idx); + return true; } - auto pipe = Pipe::unitePipes(std::move(pipes)); + + auto converting_dag = ActionsDAG::makeConvertingActions( + blocks.front().cloneEmpty().getColumnsWithTypeAndName(), + block_io.pipeline.getHeader().getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto converting_actions = std::make_shared(std::move(converting_dag)); + + for (auto & block : blocks) + { + converting_actions->execute(block); + } // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. // It will be cancelled on underlying layer (kafka buffer) + auto & lock_info = consumer_info.locks.at(topic_partition); + const auto intent = lock_info.committed_offset.value_or(0); + saveIntent(topic_partition, intent); std::atomic_size_t rows = 0; { - block_io.pipeline.complete(std::move(pipe)); - - // we need to read all consumers in parallel (sequential read may lead to situation - // when some of consumers are not used, and will break some Kafka consumer invariants) - block_io.pipeline.setNumThreads(stream_count); + block_io.pipeline.complete(Pipe{std::make_shared(std::move(blocks))}); block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); } - bool some_stream_is_stalled = false; - for (auto & source : sources) - { - some_stream_is_stalled = some_stream_is_stalled || source->isStalled(); - source->commit(); - } + saveCommittedOffset(topic_partition, last_offset); + lock_info.intent_size = intent; + lock_info.committed_offset = last_offset; + topic_partition.offset = last_offset; UInt64 milliseconds = watch.elapsedMilliseconds(); LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); - return some_stream_is_stalled; + return false; +} + + +zkutil::ZooKeeper & StorageKafka2::getZooKeeper() +{ + if (keeper->expired()) + { + keeper = keeper->startNewSession(); + //TODO(antaljanosbenjamin): handle ephemeral nodes + } + return *keeper; +} + + +std::string StorageKafka2::getTopicPartitionPath(const TopicPartition & topic_partition) +{ + return kafka_settings->kafka_keeper_path.value + "/topics/" + topic_partition.topic + "/partitions/" + + std::to_string(topic_partition.partition_id) + '/'; } NamesAndTypesList StorageKafka2::getVirtuals() const diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index d0bc5cc78b7..89d5019bd4b 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -1,17 +1,22 @@ #pragma once -#include #include +#include +#include #include #include #include +#include #include +#include +#include "Core/Block.h" #include -#include -#include #include +#include +#include +#include namespace cppkafka { @@ -51,6 +56,8 @@ public: void startup() override; void shutdown() override; + void drop() override; + Pipe read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, @@ -60,19 +67,12 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write( - const ASTPtr & query, - const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr context, - bool async_insert) override; + SinkToStoragePtr + write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; /// We want to control the number of rows in a chunk inserted into Kafka bool prefersLargeBlocks() const override { return false; } - void pushConsumer(KafkaConsumer2Ptr consumer); - KafkaConsumer2Ptr popConsumer(); - KafkaConsumer2Ptr popConsumer(std::chrono::milliseconds timeout); - const auto & getFormatName() const { return format_name; } NamesAndTypesList getVirtuals() const override; @@ -81,6 +81,7 @@ public: private: // Configuration and state + zkutil::ZooKeeperPtr keeper; std::unique_ptr kafka_settings; Macros::MacroExpansionInfo macros_info; const Names topics; @@ -102,18 +103,45 @@ private: /// In this case we still need to be able to shutdown() properly. size_t num_created_consumers = 0; /// number of actually created consumers. - std::vector consumers; /// available consumers + using TopicPartition = KafkaConsumer2::TopicPartition; + using TopicPartitions = KafkaConsumer2::TopicPartitions; - std::mutex mutex; + struct LockedTopicPartitionInfo + { + zkutil::EphemeralNodeHolderPtr lock; + std::optional committed_offset; + std::optional intent_size; + }; + + using TopicPartitionLocks = std::unordered_map< + TopicPartition, + LockedTopicPartitionInfo, + KafkaConsumer2::OnlyTopicNameAndPartitionIdHash, + KafkaConsumer2::OnlyTopicNameAndPartitionIdEquality>; + + struct ConsumerAndAssignmentInfo + { + KafkaConsumer2Ptr consumer; /// available consumers + size_t consume_from_topic_partition_index{0}; + TopicPartitions topic_partitions; + // TODO(antaljanosbenjamin): maybe recreate the ephemeral node + TopicPartitionLocks locks; + }; + + struct PolledBatchInfo + { + BlocksList blocks; + int64_t last_offset; + }; + + std::vector consumers; // Stream thread struct TaskContext { BackgroundSchedulePool::TaskHolder holder; - std::atomic stream_cancelled {false}; - explicit TaskContext(BackgroundSchedulePool::TaskHolder&& task_) : holder(std::move(task_)) - { - } + std::atomic stream_cancelled{false}; + explicit TaskContext(BackgroundSchedulePool::TaskHolder && task_) : holder(std::move(task_)) { } }; std::vector> tasks; bool thread_per_consumer = false; @@ -129,6 +157,7 @@ private: String collection_name; std::atomic shutdown_called = false; + UUID uuid{UUIDHelpers::generateV4()}; // Update Kafka configuration with values from CH user configuration. void updateConfiguration(cppkafka::Configuration & kafka_config); @@ -142,8 +171,21 @@ private: static Names parseTopics(String topic_list); static String getDefaultClientId(const StorageID & table_id_); - bool streamToViews(); + bool streamToViews(size_t idx); bool checkDependencies(const StorageID & table_id); + + // Takes lock over topic partitions and set's the committed offset in topic_partitions + void createKeeperNodes(const KafkaConsumer2Ptr & consumer); + + std::optional lockTopicPartitions(const TopicPartitions & topic_partitions); + void saveCommittedOffset(const TopicPartition & topic_partition, int64_t committed_offset); + void saveIntent(const TopicPartition & topic_partition, int64_t intent); + + PolledBatchInfo pollConsumer(KafkaConsumer2 & consumer, const TopicPartition & topic_partition, const ContextPtr & context); + + zkutil::ZooKeeper& getZooKeeper(); + + std::string getTopicPartitionPath(const TopicPartition& topic_partition ); }; } diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp index 3a35272ac74..0df4faa9030 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.cpp +++ b/src/Storages/Kafka/StorageKafkaCommon.cpp @@ -356,7 +356,7 @@ void registerStorageKafka(StorageFactory & factory) "See https://clickhouse.com/docs/en/engines/table-engines/integrations/kafka/#configuration"); } - if (kafka_settings->kafka_keeper_path.changed) + if (!kafka_settings->kafka_keeper_path.value.empty()) { if (!args.getLocalContext()->getSettingsRef().allow_experimental_kafka_store_offsets_in_keeper) From 5c635d5210b81d313acd9d68c6da55367167230a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 4 Sep 2023 12:19:03 +0000 Subject: [PATCH 005/644] Clarify how assignment is handled in consumer --- src/Storages/Kafka/KafkaConsumer2.cpp | 11 +++++------ src/Storages/Kafka/KafkaConsumer2.h | 22 ++++++++++------------ src/Storages/Kafka/StorageKafka2.cpp | 4 ++-- 3 files changed, 17 insertions(+), 20 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 61b6f801e9e..a7911f1c2d1 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -99,9 +99,9 @@ KafkaConsumer2::KafkaConsumer2( { assignment->push_back(TopicPartition{topic_partition.get_topic(), topic_partition.get_partition(), INVALID_OFFSET}); } - std::sort(assignment->begin(), assignment->end()); - updateOffsets(topic_partitions); + // We need to initialize the queues here in order to detach them from the consumer queue. Otherwise `pollEvents` might eventually poll actual messages also. + initializeQueues(topic_partitions); }); // called (synchronously, during poll) when we leave the consumer group @@ -272,7 +272,7 @@ bool KafkaConsumer2::polledDataUnusable(const TopicPartition & topic_partition) return consumer_in_wrong_state || different_topic_partition; } -KafkaConsumer2::TopicPartitions const * KafkaConsumer2::getAssignment() const +KafkaConsumer2::TopicPartitions const * KafkaConsumer2::getKafkaAssignment() const { if (assignment.has_value()) { @@ -284,7 +284,6 @@ KafkaConsumer2::TopicPartitions const * KafkaConsumer2::getAssignment() const void KafkaConsumer2::updateOffsets(const TopicPartitions & topic_partitions) { - // TODO(antaljanosbenjamin): Make sure topic_partitions and assignment is in sync. cppkafka::TopicPartitionList original_topic_partitions; original_topic_partitions.reserve(topic_partitions.size()); std::transform( @@ -294,12 +293,12 @@ void KafkaConsumer2::updateOffsets(const TopicPartitions & topic_partitions) [](const TopicPartition & tp) { return cppkafka::TopicPartition{tp.topic, tp.partition_id, tp.offset}; }); - updateOffsets(original_topic_partitions); + initializeQueues(original_topic_partitions); needs_offset_update = false; stalled_status = StalledStatus::NOT_STALLED; } -void KafkaConsumer2::updateOffsets(const cppkafka::TopicPartitionList & topic_partitions) +void KafkaConsumer2::initializeQueues(const cppkafka::TopicPartitionList & topic_partitions) { queues.clear(); // cppkafka itself calls assign(), but in order to detach the queues here we have to do the assignment manually. Later on we have to reassign the topic partitions with correct offsets. diff --git a/src/Storages/Kafka/KafkaConsumer2.h b/src/Storages/Kafka/KafkaConsumer2.h index 3341dc2c42f..5b0f7fca465 100644 --- a/src/Storages/Kafka/KafkaConsumer2.h +++ b/src/Storages/Kafka/KafkaConsumer2.h @@ -14,12 +14,12 @@ namespace CurrentMetrics { - extern const Metric KafkaConsumers; +extern const Metric KafkaConsumers; } namespace Poco { - class Logger; +class Logger; } namespace DB @@ -40,8 +40,8 @@ public: int32_t partition_id; int64_t offset{INVALID_OFFSET}; - bool operator==(const TopicPartition&) const = default; - bool operator<(const TopicPartition& other) const; + bool operator==(const TopicPartition &) const = default; + bool operator<(const TopicPartition & other) const; }; using TopicPartitions = std::vector; @@ -85,8 +85,7 @@ public: size_t poll_timeout_, bool intermediate_commit_, const std::atomic & stopped_, - const Names & _topics - ); + const Names & _topics); ~KafkaConsumer2(); @@ -96,16 +95,15 @@ public: auto pollTimeout() const { return poll_timeout; } - inline bool hasMorePolledMessages() const - { - return (stalled_status == StalledStatus::NOT_STALLED) && (current != messages.end()); - } + inline bool hasMorePolledMessages() const { return (stalled_status == StalledStatus::NOT_STALLED) && (current != messages.end()); } inline bool isStalled() const { return stalled_status != StalledStatus::NOT_STALLED; } bool polledDataUnusable(const TopicPartition & topic_partition) const; - TopicPartitions const * getAssignment() const; + // Returns the topic partitions that the consumer got from rebalancing the consumer group. If the consumer received + // no topic partitions or all of them were revoked, it returns a null pointer. + TopicPartitions const * getKafkaAssignment() const; // As the main source of offsets is not Kafka, the offsets needs to pushed to the consumer from outside bool needsOffsetUpdate() const { return needs_offset_update; } @@ -171,7 +169,7 @@ private: size_t filterMessageErrors(); ReadBufferPtr getNextMessage(); - void updateOffsets(const cppkafka::TopicPartitionList & topic_partitions); + void initializeQueues(const cppkafka::TopicPartitionList & topic_partitions); }; } diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index a1e7a81b792..c01337c6305 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -920,7 +920,7 @@ bool StorageKafka2::streamToViews(size_t idx) LOG_TRACE(log, "Polling consumer #{} for events", idx); consumer->pollEvents(); - if (nullptr == consumer->getAssignment()) + if (nullptr == consumer->getKafkaAssignment()) return true; @@ -934,7 +934,7 @@ bool StorageKafka2::streamToViews(size_t idx) consumer_info.locks.clear(); consumer_info.topic_partitions.clear(); - if (const auto * current_assignment = consumer->getAssignment(); nullptr != current_assignment) + if (const auto * current_assignment = consumer->getKafkaAssignment(); nullptr != current_assignment) { auto maybe_locks = lockTopicPartitions(*current_assignment); From 30d5b93d58025629545ccb1d7e5d9559862828b7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 4 Sep 2023 12:48:27 +0000 Subject: [PATCH 006/644] Remove outdated TODO --- src/Storages/Kafka/KafkaConsumer2.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index a7911f1c2d1..1032a5c8a5f 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -108,7 +108,6 @@ KafkaConsumer2::KafkaConsumer2( consumer->set_revocation_callback( [this](const cppkafka::TopicPartitionList & topic_partitions) { - // TODO(antaljanosbenjamin): deal with revocation CurrentMetrics::sub(CurrentMetrics::KafkaAssignedPartitions, topic_partitions.size()); ProfileEvents::increment(ProfileEvents::KafkaRebalanceRevocations); From 06160d2423de9ca59dbc5f8d26e3de26cfebc308 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 4 Sep 2023 13:19:32 +0000 Subject: [PATCH 007/644] Do not erase already fetched messages on rebalance --- src/Storages/Kafka/KafkaConsumer2.cpp | 48 ++++--------------------- src/Storages/Kafka/KafkaConsumer2.h | 8 ++--- src/Storages/Kafka/StorageKafka2.cpp | 52 ++++++++++++++------------- 3 files changed, 35 insertions(+), 73 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 1032a5c8a5f..55f65f3ce74 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -120,30 +120,10 @@ KafkaConsumer2::KafkaConsumer2( CurrentMetrics::sub(CurrentMetrics::KafkaConsumersWithAssignment, 1); } - // we can not flush data to target from that point (it is pulled, not pushed) - // so the best we can now it to - // 1) repeat last commit in sync mode (async could be still in queue, we need to be sure is is properly committed before rebalance) - // 2) stop / brake the current reading: - // * clean buffered non-commited messages - // * set flag / flush - - cleanUnprocessed(); - - stalled_status = StalledStatus::REBALANCE_HAPPENED; assignment.reset(); queues.clear(); needs_offset_update = true; waited_for_assignment = 0; - - // for now we use slower (but reliable) sync commit in main loop, so no need to repeat - // try - // { - // consumer->commit(); - // } - // catch (cppkafka::HandleException & e) - // { - // LOG_WARNING(log, "Commit error: {}", e.what()); - // } }); consumer->set_rebalance_error_callback( @@ -170,7 +150,7 @@ KafkaConsumer2::~KafkaConsumer2() { LOG_ERROR(log, "Error during unsubscribe: {}", e.what()); } - drain(); + drainConsumerQueue(); } } catch (const cppkafka::HandleException & e) @@ -179,18 +159,15 @@ KafkaConsumer2::~KafkaConsumer2() } } -// Needed to drain rest of the messages / queued callback calls from the consumer -// after unsubscribe, otherwise consumer will hang on destruction +// Needed to drain rest of the messages / queued callback calls from the consumer after unsubscribe, otherwise consumer +// will hang on destruction. Partition queues doesn't have to be attached as events are not handled by those queues. // see https://github.com/edenhill/librdkafka/issues/2077 // https://github.com/confluentinc/confluent-kafka-go/issues/189 etc. -void KafkaConsumer2::drain() +void KafkaConsumer2::drainConsumerQueue() { auto start_time = std::chrono::steady_clock::now(); cppkafka::Error last_error(RD_KAFKA_RESP_ERR_NO_ERROR); - for (auto & [tp, queue] : queues) - queue.forward_to_queue(consumer->get_consumer_queue()); - while (true) { auto msg = consumer->poll(100ms); @@ -224,13 +201,6 @@ void KafkaConsumer2::drain() } } -void KafkaConsumer2::cleanUnprocessed() -{ - messages.clear(); - current = messages.begin(); - offsets_stored = 0; -} - void KafkaConsumer2::pollEvents() { // All the partition queues are detached, so the consumer shouldn't be able to poll any messages @@ -263,12 +233,10 @@ KafkaConsumer2::TopicPartitionCounts KafkaConsumer2::getPartitionCounts() const bool KafkaConsumer2::polledDataUnusable(const TopicPartition & topic_partition) const { - const auto consumer_in_wrong_state - = (stalled_status != StalledStatus::NOT_STALLED) && (stalled_status != StalledStatus::NO_MESSAGES_RETURNED); const auto different_topic_partition = current == messages.end() ? false : (current->get_topic() != topic_partition.topic || current->get_partition() != topic_partition.partition_id); - return consumer_in_wrong_state || different_topic_partition; + return different_topic_partition; } KafkaConsumer2::TopicPartitions const * KafkaConsumer2::getKafkaAssignment() const @@ -321,6 +289,7 @@ ReadBufferPtr KafkaConsumer2::consume(const TopicPartition & topic_partition) return getNextMessage(); + // TODO(antaljanosbenjamin): check if we should poll new messages or not while (true) { stalled_status = StalledStatus::NO_MESSAGES_RETURNED; @@ -346,10 +315,6 @@ ReadBufferPtr KafkaConsumer2::consume(const TopicPartition & topic_partition) { return nullptr; } - else if (stalled_status == StalledStatus::REBALANCE_HAPPENED) - { - return nullptr; - } if (new_messages.empty()) { @@ -452,7 +417,6 @@ void KafkaConsumer2::resetIfStopped() if (stopped) { stalled_status = StalledStatus::CONSUMER_STOPPED; - cleanUnprocessed(); } } } diff --git a/src/Storages/Kafka/KafkaConsumer2.h b/src/Storages/Kafka/KafkaConsumer2.h index 5b0f7fca465..d5351af1bd2 100644 --- a/src/Storages/Kafka/KafkaConsumer2.h +++ b/src/Storages/Kafka/KafkaConsumer2.h @@ -99,8 +99,6 @@ public: inline bool isStalled() const { return stalled_status != StalledStatus::NOT_STALLED; } - bool polledDataUnusable(const TopicPartition & topic_partition) const; - // Returns the topic partitions that the consumer got from rebalancing the consumer group. If the consumer received // no topic partitions or all of them were revoked, it returns a null pointer. TopicPartitions const * getKafkaAssignment() const; @@ -133,7 +131,6 @@ private: { NOT_STALLED, NO_MESSAGES_RETURNED, - REBALANCE_HAPPENED, CONSUMER_STOPPED, NO_ASSIGNMENT, ERRORS_RETURNED @@ -143,7 +140,6 @@ private: Poco::Logger * log; const size_t batch_size = 1; const size_t poll_timeout = 0; - size_t offsets_stored = 0; StalledStatus stalled_status = StalledStatus::NO_MESSAGES_RETURNED; @@ -162,8 +158,8 @@ private: std::unordered_map queues; const Names topics; - void drain(); - void cleanUnprocessed(); + bool polledDataUnusable(const TopicPartition & topic_partition) const; + void drainConsumerQueue(); void resetIfStopped(); /// Return number of messages with an error. size_t filterMessageErrors(); diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index c01337c6305..a1091d1ac42 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -701,6 +701,26 @@ StorageKafka2::pollConsumer(KafkaConsumer2 & consumer, const TopicPartition & to StreamingFormatExecutor executor(non_virtual_header, input_format, std::move(on_error)); + + Poco::Timespan max_execution_time = kafka_settings->kafka_flush_interval_ms.changed + ? kafka_settings->kafka_flush_interval_ms + : getContext()->getSettingsRef().stream_flush_interval_ms; + + Stopwatch total_stopwatch{CLOCK_MONOTONIC_COARSE}; + + const auto check_time_limit = [&max_execution_time, &total_stopwatch]() + { + if (max_execution_time != 0) + { + auto elapsed_ns = total_stopwatch.elapsed(); + + if (elapsed_ns > static_cast(max_execution_time.totalMicroseconds()) * 1000) + return false; + } + + return true; + }; + while (true) { size_t new_rows = 0; @@ -713,13 +733,6 @@ StorageKafka2::pollConsumer(KafkaConsumer2 & consumer, const TopicPartition & to if (new_rows) { - // In read_kafka_message(), KafkaConsumer::nextImpl() - // will be called, that may make something unusable, i.e. clean - // KafkaConsumer::messages, which is accessed from - // KafkaConsumer::currentTopic() (and other helpers). - if (consumer.isStalled()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Polled messages became unusable"); - ProfileEvents::increment(ProfileEvents::KafkaRowsRead, new_rows); const auto & header_list = consumer.currentHeaderList(); @@ -779,10 +792,7 @@ StorageKafka2::pollConsumer(KafkaConsumer2 & consumer, const TopicPartition & to total_rows = total_rows + new_rows; batch_info.last_offset = consumer.currentOffset(); } - else if (consumer.polledDataUnusable(topic_partition)) - { - break; - } + // TODO(antaljanosbenjamin): think about this when rebalance is happening, because `isStalled()` will return true else if (consumer.isStalled()) { ++failed_poll_attempts; @@ -801,9 +811,8 @@ StorageKafka2::pollConsumer(KafkaConsumer2 & consumer, const TopicPartition & to } if (!consumer.hasMorePolledMessages() - && (total_rows >= kafka_settings->kafka_max_block_size || /*!checkTimeLimit() - ||*/ - failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS)) + && (total_rows >= kafka_settings->kafka_max_block_size || !check_time_limit() + || failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS || consumer.needsOffsetUpdate())) { break; } @@ -813,14 +822,6 @@ StorageKafka2::pollConsumer(KafkaConsumer2 & consumer, const TopicPartition & to { return {}; } - else if (consumer.polledDataUnusable(topic_partition)) - { - // the rows were counted already before by KafkaRowsRead, - // so let's count the rows we ignore separately - // (they will be retried after the rebalance) - ProfileEvents::increment(ProfileEvents::KafkaRowsRejected, total_rows); - return {}; - } /// MATERIALIZED columns can be added here, but I think // they are not needed here: @@ -947,14 +948,15 @@ bool StorageKafka2::streamToViews(size_t idx) consumer_info.locks = std::move(*maybe_locks); consumer_info.topic_partitions.reserve(current_assignment->size()); - for (const auto& topic_partition : *current_assignment) { + for (const auto & topic_partition : *current_assignment) + { TopicPartition topic_partition_copy{topic_partition}; - if( const auto & maybe_committed_offset = consumer_info.locks.at(topic_partition).committed_offset; maybe_committed_offset.has_value()) + if (const auto & maybe_committed_offset = consumer_info.locks.at(topic_partition).committed_offset; + maybe_committed_offset.has_value()) topic_partition_copy.offset = *maybe_committed_offset + 1; else topic_partition_copy.offset = KafkaConsumer2::BEGINNING_OFFSET; consumer_info.topic_partitions.push_back(std::move(topic_partition_copy)); - } } consumer_info.consumer->updateOffsets(consumer_info.topic_partitions); From 76e05d7c949eb23543be35ba4992409c8cead865 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 4 Sep 2023 14:48:15 +0000 Subject: [PATCH 008/644] Move TODO to the correct place --- src/Storages/Kafka/StorageKafka2.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index a1091d1ac42..2f2e390f7b9 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -583,7 +583,6 @@ std::optional StorageKafka2::lockTopicPartit for (auto try_count{0}; try_count < 10; ++try_count) { Coordination::Responses responses; - // TODO(antaljanosbenjamin): this can go wrong if we start a new session simultaneously from multiple threads. auto & keeper_ref = getZooKeeper(); if (const auto code = keeper_ref.tryMulti(ops, responses); code == Coordination::Error::ZOK) @@ -1038,6 +1037,7 @@ zkutil::ZooKeeper & StorageKafka2::getZooKeeper() { if (keeper->expired()) { + // TODO(antaljanosbenjamin): this can go wrong if we start a new session simultaneously from multiple threads. keeper = keeper->startNewSession(); //TODO(antaljanosbenjamin): handle ephemeral nodes } From fee6c0d0de2da883fd22b2713ba3a89836531377 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 4 Sep 2023 14:49:24 +0000 Subject: [PATCH 009/644] Handle errors properly --- src/Storages/Kafka/KafkaConsumer2.cpp | 2 ++ src/Storages/Kafka/StorageKafka2.cpp | 32 +++++++++++++++++++-------- 2 files changed, 25 insertions(+), 9 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 55f65f3ce74..bcad31e82df 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -268,6 +268,8 @@ void KafkaConsumer2::updateOffsets(const TopicPartitions & topic_partitions) void KafkaConsumer2::initializeQueues(const cppkafka::TopicPartitionList & topic_partitions) { queues.clear(); + messages.clear(); + current = messages.end(); // cppkafka itself calls assign(), but in order to detach the queues here we have to do the assignment manually. Later on we have to reassign the topic partitions with correct offsets. consumer->assign(topic_partitions); for (const auto & topic_partition : topic_partitions) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 2f2e390f7b9..86f6dec71ad 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -618,7 +619,13 @@ std::optional StorageKafka2::lockTopicPartit lock_info.committed_offset = getNumber(keeper_ref, *path_it + commit_file_name); lock_info.intent_size = getNumber(keeper_ref, *path_it + intent_file_name); - + LOG_TRACE( + log, + "Locked topic partition: {}:{} at offset {} with intent size {}", + tp_it->topic, + tp_it->partition_id, + lock_info.committed_offset.value_or(0), + lock_info.intent_size.value_or(0)); locks.emplace(TopicPartition(*tp_it), std::move(lock_info)); } } @@ -985,15 +992,21 @@ bool StorageKafka2::streamToViews(size_t idx) consumer_info.consume_from_topic_partition_index = (consumer_info.consume_from_topic_partition_index + 1) % consumer_info.topic_partitions.size(); - auto [blocks, last_offset] = pollConsumer(*consumer_info.consumer, topic_partition, kafka_context); + bool needs_offset_reset = false; + SCOPE_EXIT({ + if (!needs_offset_reset) + return; + consumer_info.consumer->updateOffsets(consumer_info.topic_partitions); + }); + auto [blocks, last_read_offset] = pollConsumer(*consumer_info.consumer, topic_partition, kafka_context); if (blocks.empty()) { LOG_TRACE(log, "Consumer #{} didn't get any messages", idx); + needs_offset_reset = false; return true; } - auto converting_dag = ActionsDAG::makeConvertingActions( blocks.front().cloneEmpty().getColumnsWithTypeAndName(), block_io.pipeline.getHeader().getColumnsWithTypeAndName(), @@ -1010,8 +1023,8 @@ bool StorageKafka2::streamToViews(size_t idx) // It will be cancelled on underlying layer (kafka buffer) auto & lock_info = consumer_info.locks.at(topic_partition); - const auto intent = lock_info.committed_offset.value_or(0); - saveIntent(topic_partition, intent); + lock_info.intent_size = last_read_offset - lock_info.committed_offset.value_or(0); + saveIntent(topic_partition, *lock_info.intent_size); std::atomic_size_t rows = 0; { block_io.pipeline.complete(Pipe{std::make_shared(std::move(blocks))}); @@ -1021,10 +1034,11 @@ bool StorageKafka2::streamToViews(size_t idx) executor.execute(); } - saveCommittedOffset(topic_partition, last_offset); - lock_info.intent_size = intent; - lock_info.committed_offset = last_offset; - topic_partition.offset = last_offset; + saveCommittedOffset(topic_partition, last_read_offset); + lock_info.intent_size.reset(); + lock_info.committed_offset = last_read_offset; + topic_partition.offset = last_read_offset + 1; + needs_offset_reset = false; UInt64 milliseconds = watch.elapsedMilliseconds(); LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); From cc3b48e63254a94d404b9ef7f98744c7b18caca7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 18 Sep 2023 12:41:08 +0000 Subject: [PATCH 010/644] fix typo --- src/Storages/Kafka/StorageKafkaCommon.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp index 0df4faa9030..6d9ea13b844 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.cpp +++ b/src/Storages/Kafka/StorageKafkaCommon.cpp @@ -175,9 +175,9 @@ void KafkaConfigLoader::loadTopicConfig( /// Read topic name between ... const String kafka_topic_path = config_prefix + "." + tag; - const String kafpa_topic_name_path = kafka_topic_path + "." + String{CONFIG_NAME_TAG}; + const String kafka_topic_name_path = kafka_topic_path + "." + String{CONFIG_NAME_TAG}; - const String topic_name = config.getString(kafpa_topic_name_path); + const String topic_name = config.getString(kafka_topic_name_path); if (topic_name == topic) { /// Found it! Now read the per-topic configuration into cppkafka. From 83a3c463de518be7bf6237a26aee571cd01bd62e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 9 Oct 2023 15:23:18 +0000 Subject: [PATCH 011/644] Try to handle keeper session expiry --- src/Storages/Kafka/StorageKafka2.cpp | 209 +++++++++++++++------------ src/Storages/Kafka/StorageKafka2.h | 13 +- 2 files changed, 123 insertions(+), 99 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 86f6dec71ad..5fbab603928 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include #include #include @@ -84,20 +85,22 @@ namespace DB namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; - extern const int LOGICAL_ERROR; - extern const int QUERY_NOT_ALLOWED; +extern const int NOT_IMPLEMENTED; +extern const int LOGICAL_ERROR; +extern const int QUERY_NOT_ALLOWED; } namespace { - constexpr auto RESCHEDULE_MS = 500; - // const auto CLEANUP_TIMEOUT_MS = 3000; - constexpr auto MAX_THREAD_WORK_DURATION_MS = 60000; // once per minute leave do reschedule (we can't lock threads in pool forever) +constexpr auto RESCHEDULE_MS = 500; +// const auto CLEANUP_TIMEOUT_MS = 3000; +constexpr auto MAX_THREAD_WORK_DURATION_MS = 60000; // once per minute leave do reschedule (we can't lock threads in pool forever) - constexpr auto MAX_FAILED_POLL_ATTEMPTS = 10; +constexpr auto MAX_FAILED_POLL_ATTEMPTS = 10; } +// TODO(antaljanosbenjamin): check performance + StorageKafka2::StorageKafka2( const StorageID & table_id_, ContextPtr context_, @@ -162,14 +165,10 @@ SettingsChanges StorageKafka2::createSettingsAdjustments() } if (!kafka_settings->input_format_allow_errors_ratio.changed) - { kafka_settings->input_format_allow_errors_ratio = 0.; - } if (!kafka_settings->input_format_allow_errors_num.changed) - { kafka_settings->input_format_allow_errors_num = kafka_settings->kafka_skip_broken_messages.value; - } if (!schema_name.empty()) result.emplace_back("format_schema", schema_name); @@ -188,9 +187,7 @@ Names StorageKafka2::parseTopics(String topic_list) Names result; boost::split(result, topic_list, [](char c) { return c == ','; }); for (String & topic : result) - { boost::trim(topic); - } return result; } @@ -254,7 +251,7 @@ void StorageKafka2::startup() { try { - consumers.push_back(ConsumerAndAssignmentInfo{.consumer = createConsumer(i)}); + consumers.push_back(ConsumerAndAssignmentInfo{.consumer = createConsumer(i), .keeper = keeper}); ++num_created_consumers; } catch (const cppkafka::Exception &) @@ -263,12 +260,21 @@ void StorageKafka2::startup() } } - createKeeperNodes(consumers.front().consumer); + try + { + createKeeperNodes(consumers.front().consumer); + } + catch (const Exception & ex) + { + if (ex.code() == ErrorCodes::LOGICAL_ERROR) + throw; + + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } + // Start the reader thread for (auto & task : tasks) - { task->holder->activateAndSchedule(); - } } @@ -290,7 +296,7 @@ void StorageKafka2::shutdown() void StorageKafka2::drop() { - getZooKeeper().removeRecursive(kafka_settings->kafka_keeper_path); + getZooKeeper()->removeRecursive(kafka_settings->kafka_keeper_path); } KafkaConsumer2Ptr StorageKafka2::createConsumer(size_t consumer_number) @@ -300,13 +306,9 @@ KafkaConsumer2Ptr StorageKafka2::createConsumer(size_t consumer_number) conf.set("metadata.broker.list", brokers); conf.set("group.id", group); if (num_consumers > 1) - { conf.set("client.id", fmt::format("{}-{}", client_id, consumer_number)); - } else - { conf.set("client.id", client_id); - } conf.set("client.software.name", VERSION_NAME); conf.set("client.software.version", VERSION_DESCRIBE); conf.set("auto.offset.reset", "earliest"); // If no offset stored for this group, read all messages from the start @@ -505,33 +507,30 @@ bool StorageKafka2::checkDependencies(const StorageID & table_id) namespace { - const std::string lock_file_name{"lock"}; - const std::string commit_file_name{"committed"}; - const std::string intent_file_name{"intention"}; +const std::string lock_file_name{"lock"}; +const std::string commit_file_name{"committed"}; +const std::string intent_file_name{"intention"}; - std::optional getNumber(zkutil::ZooKeeper & keeper, const std::string & path) - { - std::string result; - if (!keeper.tryGet(path, result)) - return std::nullopt; +std::optional getNumber(zkutil::ZooKeeper & keeper, const std::string & path) +{ + std::string result; + if (!keeper.tryGet(path, result)) + return std::nullopt; - return DB::parse(result); - } + return DB::parse(result); +} } void StorageKafka2::createKeeperNodes(const KafkaConsumer2Ptr & consumer) { // TODO(antaljanosbenjamin): check config with other StorageKafkas + // TODO(antaljanosbenjamin): maybe also create a node in `keeper_path/replicas/` to note that this replica has the table? const auto & keeper_path = kafka_settings->kafka_keeper_path.value; - auto & keeper_ref = getZooKeeper(); - - if (keeper_ref.exists(keeper_path)) - { + if (keeper->exists(keeper_path)) return; - } - keeper_ref.createAncestors(keeper_path); + keeper->createAncestors(keeper_path); Coordination::Requests ops; ops.emplace_back(zkutil::makeCreateRequest(keeper_path, "", zkutil::CreateMode::Persistent)); @@ -553,14 +552,13 @@ void StorageKafka2::createKeeperNodes(const KafkaConsumer2Ptr & consumer) Coordination::Responses responses; - const auto code = keeper_ref.tryMulti(ops, responses); + const auto code = keeper->tryMulti(ops, responses); if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNODEEXISTS) - { zkutil::KeeperMultiException::check(code, ops, responses); - } } -std::optional StorageKafka2::lockTopicPartitions(const TopicPartitions & topic_partitions) +std::optional +StorageKafka2::lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const TopicPartitions & topic_partitions) { // TODO(antaljanosbenjamin): Review this function with somebody who know keeper better than me const auto uuid_as_string = toString(uuid); @@ -568,9 +566,7 @@ std::optional StorageKafka2::lockTopicPartit std::vector topic_partition_paths; topic_partition_paths.reserve(topic_partitions.size()); for (const auto & topic_partition : topic_partitions) - { topic_partition_paths.emplace_back(getTopicPartitionPath(topic_partition)); - } Coordination::Requests ops; @@ -584,21 +580,18 @@ std::optional StorageKafka2::lockTopicPartit for (auto try_count{0}; try_count < 10; ++try_count) { Coordination::Responses responses; - auto & keeper_ref = getZooKeeper(); - if (const auto code = keeper_ref.tryMulti(ops, responses); code == Coordination::Error::ZOK) + if (const auto code = keeper_to_use.tryMulti(ops, responses); code == Coordination::Error::ZOK) { success = true; break; } - else - { + else if (code != Coordination::Error::ZNODEEXISTS) zkutil::KeeperMultiException::check(code, ops, responses); - } - // TODO(antaljanosbenjamin): Probably handle the most common errors + // TODO(antaljanosbenjamin): We shouldn't wait here, but let's give the other consumers to release the locks for (const auto & topic_partition_path : topic_partition_paths) - keeper_ref.waitForDisappear(topic_partition_path + lock_file_name); + keeper_to_use.waitForDisappear(topic_partition_path + lock_file_name); } if (!success) @@ -608,16 +601,15 @@ std::optional StorageKafka2::lockTopicPartit // We have the locks TopicPartitionLocks locks; { - auto & keeper_ref = getZooKeeper(); auto tp_it = topic_partitions.begin(); auto path_it = topic_partition_paths.begin(); for (; tp_it != topic_partitions.end(); ++tp_it, ++path_it) { using zkutil::EphemeralNodeHolder; - LockedTopicPartitionInfo lock_info{.lock = EphemeralNodeHolder::existing(*path_it + lock_file_name, keeper_ref)}; + LockedTopicPartitionInfo lock_info{.lock = EphemeralNodeHolder::existing(*path_it + lock_file_name, keeper_to_use)}; - lock_info.committed_offset = getNumber(keeper_ref, *path_it + commit_file_name); - lock_info.intent_size = getNumber(keeper_ref, *path_it + intent_file_name); + lock_info.committed_offset = getNumber(keeper_to_use, *path_it + commit_file_name); + lock_info.intent_size = getNumber(keeper_to_use, *path_it + intent_file_name); LOG_TRACE( log, @@ -634,18 +626,17 @@ std::optional StorageKafka2::lockTopicPartit } -void StorageKafka2::saveCommittedOffset(const TopicPartition & topic_partition, int64_t committed_offset) +void StorageKafka2::saveCommittedOffset(zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition, int64_t committed_offset) { const auto partition_prefix = getTopicPartitionPath(topic_partition); - auto & keeper_ref = getZooKeeper(); - keeper_ref.createOrUpdate(partition_prefix + commit_file_name, toString(committed_offset), zkutil::CreateMode::Persistent); + keeper_to_use.createOrUpdate(partition_prefix + commit_file_name, toString(committed_offset), zkutil::CreateMode::Persistent); // This is best effort, if it fails we will try to remove in the next round - keeper_ref.tryRemove(partition_prefix + intent_file_name, -1); + keeper_to_use.tryRemove(partition_prefix + intent_file_name, -1); } -void StorageKafka2::saveIntent(const TopicPartition & topic_partition, int64_t intent) +void StorageKafka2::saveIntent(zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition, int64_t intent) { - getZooKeeper().createOrUpdate( + keeper_to_use.createOrUpdate( getTopicPartitionPath(topic_partition) + intent_file_name, toString(intent), zkutil::CreateMode::Persistent); } @@ -825,9 +816,7 @@ StorageKafka2::pollConsumer(KafkaConsumer2 & consumer, const TopicPartition & to } if (total_rows == 0) - { return {}; - } /// MATERIALIZED columns can be added here, but I think // they are not needed here: @@ -922,20 +911,52 @@ bool StorageKafka2::streamToViews(size_t idx) ProfileEvents::increment(ProfileEvents::KafkaBackgroundReads); auto & consumer_info = consumers[idx]; + try + { + LOG_TRACE(log, "Trying to consume from consumer {}", idx); + const auto rows = streamFromConsumer(consumer_info); + if (rows.has_value()) + { + const auto milliseconds = watch.elapsedMilliseconds(); + LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(*rows), table_id.getNameForLogs(), milliseconds); + } + else + { + LOG_DEBUG(log, "Couldn't push any rows"); + return true; + } + } + catch (const zkutil::KeeperException & e) + { + if (Coordination::isHardwareError(e.code)) + { + consumer_info.locks.clear(); + consumer_info.keeper = getZooKeeper(); + } + else + throw; + + // TODO(antaljanosbenjamin): Should we reschedule in case of keeper error? + } + return false; +} + + +std::optional StorageKafka2::streamFromConsumer(ConsumerAndAssignmentInfo & consumer_info) +{ auto & consumer = consumer_info.consumer; + auto & keeper_to_use = *consumer_info.keeper; // To keep the consumer alive - LOG_TRACE(log, "Polling consumer #{} for events", idx); + LOG_TRACE(log, "Polling consumer for events"); consumer->pollEvents(); if (nullptr == consumer->getKafkaAssignment()) - return true; + return std::nullopt; + LOG_TRACE(log, "Consumer has assignment"); - - LOG_TRACE(log, "Consumer #{} has assignment", idx); - - if (consumer->needsOffsetUpdate()) + if (consumer->needsOffsetUpdate() || consumer_info.locks.empty()) { - LOG_TRACE(log, "Consumer #{} needs update offset", idx); + LOG_TRACE(log, "Consumer needs update offset"); consumer_info.consume_from_topic_partition_index = 0; consumer_info.locks.clear(); @@ -943,7 +964,7 @@ bool StorageKafka2::streamToViews(size_t idx) if (const auto * current_assignment = consumer->getKafkaAssignment(); nullptr != current_assignment) { - auto maybe_locks = lockTopicPartitions(*current_assignment); + auto maybe_locks = lockTopicPartitions(keeper_to_use, *current_assignment); if (!maybe_locks.has_value()) { @@ -962,15 +983,20 @@ bool StorageKafka2::streamToViews(size_t idx) topic_partition_copy.offset = *maybe_committed_offset + 1; else topic_partition_copy.offset = KafkaConsumer2::BEGINNING_OFFSET; + consumer_info.topic_partitions.push_back(std::move(topic_partition_copy)); } + consumer_info.consumer->updateOffsets(consumer_info.topic_partitions); + } + else + { + LOG_TRACE(log, "Consumer lost assignment while trying to lock partitions"); } - consumer_info.consumer->updateOffsets(consumer_info.topic_partitions); } // Create an INSERT query for streaming data auto insert = std::make_shared(); - insert->table_id = table_id; + insert->table_id = getStorageID(); auto kafka_context = Context::createCopy(getContext()); kafka_context->makeQueryContext(); @@ -984,15 +1010,14 @@ bool StorageKafka2::streamToViews(size_t idx) auto & topic_partition = consumer_info.topic_partitions[consumer_info.consume_from_topic_partition_index]; LOG_TRACE( log, - "Consumer #{} will fetch {}:{} (consume_from_topic_partition_index is {})", - idx, + "Will fetch {}:{} (consume_from_topic_partition_index is {})", topic_partition.topic, topic_partition.partition_id, consumer_info.consume_from_topic_partition_index); consumer_info.consume_from_topic_partition_index = (consumer_info.consume_from_topic_partition_index + 1) % consumer_info.topic_partitions.size(); - bool needs_offset_reset = false; + bool needs_offset_reset = true; SCOPE_EXIT({ if (!needs_offset_reset) return; @@ -1002,9 +1027,9 @@ bool StorageKafka2::streamToViews(size_t idx) if (blocks.empty()) { - LOG_TRACE(log, "Consumer #{} didn't get any messages", idx); + LOG_TRACE(log, "Didn't get any messages"); needs_offset_reset = false; - return true; + return std::nullopt; } auto converting_dag = ActionsDAG::makeConvertingActions( @@ -1015,16 +1040,14 @@ bool StorageKafka2::streamToViews(size_t idx) auto converting_actions = std::make_shared(std::move(converting_dag)); for (auto & block : blocks) - { converting_actions->execute(block); - } // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. // It will be cancelled on underlying layer (kafka buffer) auto & lock_info = consumer_info.locks.at(topic_partition); lock_info.intent_size = last_read_offset - lock_info.committed_offset.value_or(0); - saveIntent(topic_partition, *lock_info.intent_size); + saveIntent(keeper_to_use, topic_partition, *lock_info.intent_size); std::atomic_size_t rows = 0; { block_io.pipeline.complete(Pipe{std::make_shared(std::move(blocks))}); @@ -1034,28 +1057,24 @@ bool StorageKafka2::streamToViews(size_t idx) executor.execute(); } - saveCommittedOffset(topic_partition, last_read_offset); + saveCommittedOffset(keeper_to_use, topic_partition, last_read_offset); lock_info.intent_size.reset(); lock_info.committed_offset = last_read_offset; topic_partition.offset = last_read_offset + 1; needs_offset_reset = false; - - UInt64 milliseconds = watch.elapsedMilliseconds(); - LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); - - return false; + return rows; } -zkutil::ZooKeeper & StorageKafka2::getZooKeeper() -{ - if (keeper->expired()) + zkutil::ZooKeeperPtr StorageKafka2::getZooKeeper() { - // TODO(antaljanosbenjamin): this can go wrong if we start a new session simultaneously from multiple threads. - keeper = keeper->startNewSession(); - //TODO(antaljanosbenjamin): handle ephemeral nodes - } - return *keeper; + if (keeper->expired()) + { + // TODO(antaljanosbenjamin): this can go wrong if we start a new session simultaneously from multiple threads. + keeper = keeper->startNewSession(); + //TODO(antaljanosbenjamin): handle ephemeral nodes + } + return keeper; } diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index 89d5019bd4b..4bc4240fb48 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -81,6 +81,7 @@ public: private: // Configuration and state + std::mutex keeper_mutex; zkutil::ZooKeeperPtr keeper; std::unique_ptr kafka_settings; Macros::MacroExpansionInfo macros_info; @@ -125,6 +126,7 @@ private: size_t consume_from_topic_partition_index{0}; TopicPartitions topic_partitions; // TODO(antaljanosbenjamin): maybe recreate the ephemeral node + zkutil::ZooKeeperPtr keeper; TopicPartitionLocks locks; }; @@ -172,18 +174,21 @@ private: static String getDefaultClientId(const StorageID & table_id_); bool streamToViews(size_t idx); + + std::optional streamFromConsumer(ConsumerAndAssignmentInfo& consumer_info); + bool checkDependencies(const StorageID & table_id); // Takes lock over topic partitions and set's the committed offset in topic_partitions void createKeeperNodes(const KafkaConsumer2Ptr & consumer); - std::optional lockTopicPartitions(const TopicPartitions & topic_partitions); - void saveCommittedOffset(const TopicPartition & topic_partition, int64_t committed_offset); - void saveIntent(const TopicPartition & topic_partition, int64_t intent); + std::optional lockTopicPartitions(zkutil::ZooKeeper& keeper_to_use, const TopicPartitions & topic_partitions); + void saveCommittedOffset(zkutil::ZooKeeper& keeper_to_use,const TopicPartition & topic_partition, int64_t committed_offset); + void saveIntent(zkutil::ZooKeeper& keeper_to_use,const TopicPartition & topic_partition, int64_t intent); PolledBatchInfo pollConsumer(KafkaConsumer2 & consumer, const TopicPartition & topic_partition, const ContextPtr & context); - zkutil::ZooKeeper& getZooKeeper(); + zkutil::ZooKeeperPtr getZooKeeper(); std::string getTopicPartitionPath(const TopicPartition& topic_partition ); }; From fa5150130d1db39f98b8c5ed191cc7bf4564a27d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Feb 2024 16:15:43 +0000 Subject: [PATCH 012/644] Fix compilation issues --- src/Storages/Kafka/StorageKafka2.cpp | 12 ++++++------ src/Storages/Kafka/StorageKafka2.h | 4 ++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 5fbab603928..4ee8523a4a9 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -48,7 +48,7 @@ #include #include #include "Storages/Kafka/KafkaConsumer2.h" -#include "config_version.h" +#include "Common/config_version.h" #if USE_KRB5 # include @@ -136,7 +136,7 @@ StorageKafka2::StorageKafka2( if (thread_per_consumer) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "The new Kafka storage cannot use multiple threads yet!"); - if (kafka_settings->kafka_handle_error_mode == HandleKafkaErrorMode::STREAM) + if (kafka_settings->kafka_handle_error_mode == StreamingHandleErrorMode::STREAM) { kafka_settings->input_format_allow_errors_num = 0; kafka_settings->input_format_allow_errors_ratio = 0; @@ -278,7 +278,7 @@ void StorageKafka2::startup() } -void StorageKafka2::shutdown() +void StorageKafka2::shutdown(bool) { for (auto & task : tasks) { @@ -654,7 +654,7 @@ StorageKafka2::pollConsumer(KafkaConsumer2 & consumer, const TopicPartition & to // otherwise external iteration will reuse that and logic will became even more fuzzy MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); - auto put_error_to_stream = kafka_settings->kafka_handle_error_mode == HandleKafkaErrorMode::STREAM; + auto put_error_to_stream = kafka_settings->kafka_handle_error_mode == StreamingHandleErrorMode::STREAM; EmptyReadBuffer empty_buf; auto input_format = FormatFactory::instance().getInput( @@ -1095,7 +1095,7 @@ NamesAndTypesList StorageKafka2::getVirtuals() const {"_timestamp_ms", std::make_shared(std::make_shared(3))}, {"_headers.name", std::make_shared(std::make_shared())}, {"_headers.value", std::make_shared(std::make_shared())}}; - if (kafka_settings->kafka_handle_error_mode == HandleKafkaErrorMode::STREAM) + if (kafka_settings->kafka_handle_error_mode == StreamingHandleErrorMode::STREAM) { result.push_back({"_raw_message", std::make_shared()}); result.push_back({"_error", std::make_shared()}); @@ -1115,7 +1115,7 @@ Names StorageKafka2::getVirtualColumnNames() const "_headers.name", "_headers.value", }; - if (kafka_settings->kafka_handle_error_mode == HandleKafkaErrorMode::STREAM) + if (kafka_settings->kafka_handle_error_mode == StreamingHandleErrorMode::STREAM) { result.push_back({"_raw_message"}); result.push_back({"_error"}); diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index 4bc4240fb48..17606ea60df 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -54,7 +54,7 @@ public: bool noPushingToViews() const override { return true; } void startup() override; - void shutdown() override; + void shutdown(bool is_drop) override; void drop() override; @@ -77,7 +77,7 @@ public: NamesAndTypesList getVirtuals() const override; Names getVirtualColumnNames() const; - HandleKafkaErrorMode getHandleKafkaErrorMode() const { return kafka_settings->kafka_handle_error_mode; } + StreamingHandleErrorMode getHandleKafkaErrorMode() const { return kafka_settings->kafka_handle_error_mode; } private: // Configuration and state From a4c176f0d508d98456a73a444b3d3892814df1e8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 14 Feb 2024 17:57:27 +0000 Subject: [PATCH 013/644] Propagate intent size to consumer --- src/Storages/Kafka/KafkaConsumer2.cpp | 32 ++--- src/Storages/Kafka/KafkaConsumer2.h | 4 +- src/Storages/Kafka/StorageKafka2.cpp | 142 ++++++++++++---------- src/Storages/Kafka/StorageKafka2.h | 72 ++++++----- src/Storages/Kafka/StorageKafkaCommon.cpp | 6 +- 5 files changed, 137 insertions(+), 119 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index bcad31e82df..01169596e1a 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -280,7 +280,7 @@ void KafkaConsumer2::initializeQueues(const cppkafka::TopicPartitionList & topic } // it do the poll when needed -ReadBufferPtr KafkaConsumer2::consume(const TopicPartition & topic_partition) +ReadBufferPtr KafkaConsumer2::consume(const TopicPartition & topic_partition, const std::optional & message_count) { resetIfStopped(); @@ -288,10 +288,12 @@ ReadBufferPtr KafkaConsumer2::consume(const TopicPartition & topic_partition) return nullptr; if (hasMorePolledMessages()) - return getNextMessage(); + { + if (auto next_message = getNextMessage(); next_message) + return next_message; + } - - // TODO(antaljanosbenjamin): check if we should poll new messages or not + // TODO(antaljanosbenjamin): check if we should poll new messages or not while (true) { stalled_status = StalledStatus::NO_MESSAGES_RETURNED; @@ -309,8 +311,9 @@ ReadBufferPtr KafkaConsumer2::consume(const TopicPartition & topic_partition) queue_to_poll_from.forward_to_queue(consumer->get_consumer_queue()); SCOPE_EXIT({ queue_to_poll_from.disable_queue_forwarding(); }); + const auto messages_to_pull = message_count.value_or(batch_size); /// Don't drop old messages immediately, since we may need them for virtual columns. - auto new_messages = consumer->poll_batch(batch_size, std::chrono::milliseconds(actual_poll_timeout_ms)); + auto new_messages = consumer->poll_batch(messages_to_pull, std::chrono::milliseconds(actual_poll_timeout_ms)); resetIfStopped(); if (stalled_status == StalledStatus::CONSUMER_STOPPED) @@ -376,17 +379,18 @@ ReadBufferPtr KafkaConsumer2::consume(const TopicPartition & topic_partition) ReadBufferPtr KafkaConsumer2::getNextMessage() { - if (current == messages.end()) - return nullptr; + while (current != messages.end()) + { + const auto * data = current->get_payload().get_data(); + size_t size = current->get_payload().get_size(); + ++current; - const auto * data = current->get_payload().get_data(); - size_t size = current->get_payload().get_size(); - ++current; + // TODO(antaljanosbenjamin): When this can be nullptr? + if (data) + return std::make_shared(data, size); + } - if (data) - return std::make_shared(data, size); - - return getNextMessage(); + return nullptr; } size_t KafkaConsumer2::filterMessageErrors() diff --git a/src/Storages/Kafka/KafkaConsumer2.h b/src/Storages/Kafka/KafkaConsumer2.h index d5351af1bd2..b6967b0c5fb 100644 --- a/src/Storages/Kafka/KafkaConsumer2.h +++ b/src/Storages/Kafka/KafkaConsumer2.h @@ -103,7 +103,7 @@ public: // no topic partitions or all of them were revoked, it returns a null pointer. TopicPartitions const * getKafkaAssignment() const; - // As the main source of offsets is not Kafka, the offsets needs to pushed to the consumer from outside + // As the main source of offsets is not Kafka, the offsets needs to be pushed to the consumer from outside bool needsOffsetUpdate() const { return needs_offset_update; } // Returns true if it received new assignment and could update the internal state accordingly, false otherwise @@ -112,7 +112,7 @@ public: /// Polls batch of messages from Kafka and returns read buffer containing the next message or /// nullptr when there are no messages to process. /// TODO(antaljanosbenjamin): add batch size param - ReadBufferPtr consume(const TopicPartition & topic_partition); + ReadBufferPtr consume(const TopicPartition & topic_partition, const std::optional & message_count); // Return values for the message that's being read. String currentTopic() const { return current[-1].get_topic(); } diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 740398b2640..c658174b9d5 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -636,8 +636,11 @@ void StorageKafka2::saveIntent(zkutil::ZooKeeper & keeper_to_use, const TopicPar } -StorageKafka2::PolledBatchInfo -StorageKafka2::pollConsumer(KafkaConsumer2 & consumer, const TopicPartition & topic_partition, const ContextPtr & modified_context) +StorageKafka2::PolledBatchInfo StorageKafka2::pollConsumer( + KafkaConsumer2 & consumer, + const TopicPartition & topic_partition, + std::optional message_count, + const ContextPtr & modified_context) { PolledBatchInfo batch_info; auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); @@ -717,7 +720,7 @@ StorageKafka2::pollConsumer(KafkaConsumer2 & consumer, const TopicPartition & to { size_t new_rows = 0; exception_message.reset(); - if (auto buf = consumer.consume(topic_partition)) + if (auto buf = consumer.consume(topic_partition, message_count)) { ProfileEvents::increment(ProfileEvents::KafkaMessagesRead); new_rows = executor.execute(*buf); @@ -906,18 +909,72 @@ bool StorageKafka2::streamToViews(size_t idx) ProfileEvents::increment(ProfileEvents::KafkaBackgroundReads); auto & consumer_info = consumers[idx]; + auto & consumer = consumer_info.consumer; + + // To keep the consumer alive + LOG_TRACE(log, "Polling consumer for events"); + consumer->pollEvents(); + + if (consumer->needsOffsetUpdate() || consumer_info.locks.empty()) + { + // First release the locks so let other consumers acquire them ASAP + consumer_info.locks.clear(); + + const auto * current_assignment = consumer->getKafkaAssignment(); + if (current_assignment == nullptr) + { + // The consumer lost its assignment and haven't received a new one. + // TODO(antaljanosbenjamin): returning a proper value representing the state + // By returning true this function reports the current consumer as a "stalled" stream, which + return true; + } + LOG_TRACE(log, "Consumer needs update offset"); + consumer_info.consume_from_topic_partition_index = 0; + + consumer_info.locks.clear(); + consumer_info.topic_partitions.clear(); + + auto maybe_locks = lockTopicPartitions(*consumer_info.keeper, *current_assignment); + + if (!maybe_locks.has_value()) + { + // We couldn't acquire locks, probably some other consumers are still holding them. + return true; + } + + consumer_info.locks = std::move(*maybe_locks); + + consumer_info.topic_partitions.reserve(current_assignment->size()); + for (const auto & topic_partition : *current_assignment) + { + TopicPartition topic_partition_copy{topic_partition}; + if (const auto & maybe_committed_offset = consumer_info.locks.at(topic_partition).committed_offset; + maybe_committed_offset.has_value()) + topic_partition_copy.offset = *maybe_committed_offset + 1; + else + topic_partition_copy.offset = KafkaConsumer2::BEGINNING_OFFSET; + + consumer_info.topic_partitions.push_back(std::move(topic_partition_copy)); + } + consumer_info.consumer->updateOffsets(consumer_info.topic_partitions); + + } + + LOG_TRACE(log, "Consumer has assignment"); + + // Here we will try to pull messages regardless if we loose our assignment try { LOG_TRACE(log, "Trying to consume from consumer {}", idx); - const auto rows = streamFromConsumer(consumer_info); - if (rows.has_value()) + const auto maybe_rows = streamFromConsumer(consumer_info); + if (maybe_rows.has_value()) { const auto milliseconds = watch.elapsedMilliseconds(); - LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(*rows), table_id.getNameForLogs(), milliseconds); + LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(*maybe_rows), table_id.getNameForLogs(), milliseconds); } else { - LOG_DEBUG(log, "Couldn't push any rows"); + LOG_DEBUG(log, "Couldn't stream any messages"); return true; } } @@ -939,56 +996,6 @@ bool StorageKafka2::streamToViews(size_t idx) std::optional StorageKafka2::streamFromConsumer(ConsumerAndAssignmentInfo & consumer_info) { - auto & consumer = consumer_info.consumer; - auto & keeper_to_use = *consumer_info.keeper; - // To keep the consumer alive - - LOG_TRACE(log, "Polling consumer for events"); - consumer->pollEvents(); - if (nullptr == consumer->getKafkaAssignment()) - return std::nullopt; - LOG_TRACE(log, "Consumer has assignment"); - - if (consumer->needsOffsetUpdate() || consumer_info.locks.empty()) - { - LOG_TRACE(log, "Consumer needs update offset"); - consumer_info.consume_from_topic_partition_index = 0; - - consumer_info.locks.clear(); - consumer_info.topic_partitions.clear(); - - if (const auto * current_assignment = consumer->getKafkaAssignment(); nullptr != current_assignment) - { - auto maybe_locks = lockTopicPartitions(keeper_to_use, *current_assignment); - - if (!maybe_locks.has_value()) - { - // TODO(antaljanosbenjamin): signal this somehow to caller, maybe wait a bit longer. - return true; - } - - consumer_info.locks = std::move(*maybe_locks); - - consumer_info.topic_partitions.reserve(current_assignment->size()); - for (const auto & topic_partition : *current_assignment) - { - TopicPartition topic_partition_copy{topic_partition}; - if (const auto & maybe_committed_offset = consumer_info.locks.at(topic_partition).committed_offset; - maybe_committed_offset.has_value()) - topic_partition_copy.offset = *maybe_committed_offset + 1; - else - topic_partition_copy.offset = KafkaConsumer2::BEGINNING_OFFSET; - - consumer_info.topic_partitions.push_back(std::move(topic_partition_copy)); - } - consumer_info.consumer->updateOffsets(consumer_info.topic_partitions); - } - else - { - LOG_TRACE(log, "Consumer lost assignment while trying to lock partitions"); - } - } - // Create an INSERT query for streaming data auto insert = std::make_shared(); insert->table_id = getStorageID(); @@ -1018,7 +1025,8 @@ std::optional StorageKafka2::streamFromConsumer(ConsumerAndAssignmentInf return; consumer_info.consumer->updateOffsets(consumer_info.topic_partitions); }); - auto [blocks, last_read_offset] = pollConsumer(*consumer_info.consumer, topic_partition, kafka_context); + auto [blocks, last_read_offset] + = pollConsumer(*consumer_info.consumer, topic_partition, consumer_info.locks[topic_partition].intent_size, kafka_context); if (blocks.empty()) { @@ -1040,6 +1048,7 @@ std::optional StorageKafka2::streamFromConsumer(ConsumerAndAssignmentInf // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. // It will be cancelled on underlying layer (kafka buffer) + auto & keeper_to_use = *consumer_info.keeper; auto & lock_info = consumer_info.locks.at(topic_partition); lock_info.intent_size = last_read_offset - lock_info.committed_offset.value_or(0); saveIntent(keeper_to_use, topic_partition, *lock_info.intent_size); @@ -1057,19 +1066,20 @@ std::optional StorageKafka2::streamFromConsumer(ConsumerAndAssignmentInf lock_info.committed_offset = last_read_offset; topic_partition.offset = last_read_offset + 1; needs_offset_reset = false; + return rows; } - zkutil::ZooKeeperPtr StorageKafka2::getZooKeeper() +zkutil::ZooKeeperPtr StorageKafka2::getZooKeeper() +{ + if (keeper->expired()) { - if (keeper->expired()) - { - // TODO(antaljanosbenjamin): this can go wrong if we start a new session simultaneously from multiple threads. - keeper = keeper->startNewSession(); - //TODO(antaljanosbenjamin): handle ephemeral nodes - } - return keeper; + // TODO(antaljanosbenjamin): this can go wrong if we start a new session simultaneously from multiple threads. + keeper = keeper->startNewSession(); + //TODO(antaljanosbenjamin): handle ephemeral nodes + } + return keeper; } diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index 17606ea60df..d3735e87afc 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -80,30 +80,6 @@ public: StreamingHandleErrorMode getHandleKafkaErrorMode() const { return kafka_settings->kafka_handle_error_mode; } private: - // Configuration and state - std::mutex keeper_mutex; - zkutil::ZooKeeperPtr keeper; - std::unique_ptr kafka_settings; - Macros::MacroExpansionInfo macros_info; - const Names topics; - const String brokers; - const String group; - const String client_id; - const String format_name; - const size_t max_rows_per_message; - const String schema_name; - const size_t num_consumers; /// total number of consumers - Poco::Logger * log; - Poco::Semaphore semaphore; - const bool intermediate_commit; - const SettingsChanges settings_adjustments; - - std::atomic mv_attached = false; - - /// Can differ from num_consumers in case of exception in startup() (or if startup() hasn't been called). - /// In this case we still need to be able to shutdown() properly. - size_t num_created_consumers = 0; /// number of actually created consumers. - using TopicPartition = KafkaConsumer2::TopicPartition; using TopicPartitions = KafkaConsumer2::TopicPartitions; @@ -136,8 +112,6 @@ private: int64_t last_offset; }; - std::vector consumers; - // Stream thread struct TaskContext { @@ -145,20 +119,48 @@ private: std::atomic stream_cancelled{false}; explicit TaskContext(BackgroundSchedulePool::TaskHolder && task_) : holder(std::move(task_)) { } }; + + enum class AssignmentChange + { + NotChanged, + Updated, + Lost + }; + + // Configuration and state + std::mutex keeper_mutex; + zkutil::ZooKeeperPtr keeper; + std::unique_ptr kafka_settings; + Macros::MacroExpansionInfo macros_info; + const Names topics; + const String brokers; + const String group; + const String client_id; + const String format_name; + const size_t max_rows_per_message; + const String schema_name; + const size_t num_consumers; /// total number of consumers + Poco::Logger * log; + Poco::Semaphore semaphore; + const bool intermediate_commit; + const SettingsChanges settings_adjustments; + std::atomic mv_attached = false; + /// Can differ from num_consumers in case of exception in startup() (or if startup() hasn't been called). + /// In this case we still need to be able to shutdown() properly. + size_t num_created_consumers = 0; /// number of actually created consumers. + std::vector consumers; std::vector> tasks; bool thread_per_consumer = false; - /// For memory accounting in the librdkafka threads. std::mutex thread_statuses_mutex; std::list> thread_statuses; + /// If named_collection is specified. + String collection_name; + std::atomic shutdown_called = false; SettingsChanges createSettingsAdjustments(); KafkaConsumer2Ptr createConsumer(size_t consumer_number); - /// If named_collection is specified. - String collection_name; - - std::atomic shutdown_called = false; UUID uuid{UUIDHelpers::generateV4()}; // Update Kafka configuration with values from CH user configuration. @@ -175,7 +177,7 @@ private: bool streamToViews(size_t idx); - std::optional streamFromConsumer(ConsumerAndAssignmentInfo& consumer_info); + std::optional streamFromConsumer(ConsumerAndAssignmentInfo & consumer_info); bool checkDependencies(const StorageID & table_id); @@ -186,7 +188,11 @@ private: void saveCommittedOffset(zkutil::ZooKeeper& keeper_to_use,const TopicPartition & topic_partition, int64_t committed_offset); void saveIntent(zkutil::ZooKeeper& keeper_to_use,const TopicPartition & topic_partition, int64_t intent); - PolledBatchInfo pollConsumer(KafkaConsumer2 & consumer, const TopicPartition & topic_partition, const ContextPtr & context); + PolledBatchInfo pollConsumer( + KafkaConsumer2 & consumer, + const TopicPartition & topic_partition, + std::optional message_count, + const ContextPtr & context); zkutil::ZooKeeperPtr getZooKeeper(); diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp index ff5bdb3e5b6..b0f23c38163 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.cpp +++ b/src/Storages/Kafka/StorageKafkaCommon.cpp @@ -363,15 +363,13 @@ void registerStorageKafka(StorageFactory & factory) throw Exception( ErrorCodes::SUPPORT_IS_DISABLED, - "Storing the Kafka offsets in Keeper is experimental. " - "Set `allow_experimental_kafka_store_offsets_in_keeper` setting to enable it"); + "Storing the Kafka offsets in Keeper is experimental. Set `allow_experimental_kafka_store_offsets_in_keeper` setting " + "to enable it"); return std::make_shared( args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); } - //return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); - // [[maybe_unused]] auto * a = new StorageKafka(args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); }; From 4701f39b284cdd533b58a614b138325018228ed8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 22 Feb 2024 15:18:52 +0000 Subject: [PATCH 014/644] Poll directly from partition queues --- src/Storages/Kafka/KafkaConsumer2.cpp | 29 ++++++++++++--------------- 1 file changed, 13 insertions(+), 16 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 01169596e1a..37829c1a181 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -46,7 +46,7 @@ namespace ErrorCodes using namespace std::chrono_literals; const auto MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS = 15000; -const std::size_t POLL_TIMEOUT_WO_ASSIGNMENT_MS = 50; +const auto POLL_TIMEOUT_WO_ASSIGNMENT = 50ms; const auto DRAIN_TIMEOUT_MS = 5000ms; @@ -203,9 +203,17 @@ void KafkaConsumer2::drainConsumerQueue() void KafkaConsumer2::pollEvents() { + // POLL_TIMEOUT_WO_ASSIGNMENT_MS (50ms) is 100% enough just to check if we got assignment + // (see https://github.com/ClickHouse/ClickHouse/issues/11218) + auto msg = consumer->poll(POLL_TIMEOUT_WO_ASSIGNMENT); + // All the partition queues are detached, so the consumer shouldn't be able to poll any messages - auto msg = consumer->poll(10ms); chassert(!msg && "Consumer returned a message when it was not expected"); + + auto consumer_queue = consumer->get_consumer_queue(); + // There should be events in the queue, so let's consume them all + while (consumer_queue.get_length() > 0) + consumer->poll(); }; KafkaConsumer2::TopicPartitionCounts KafkaConsumer2::getPartitionCounts() const @@ -298,22 +306,11 @@ ReadBufferPtr KafkaConsumer2::consume(const TopicPartition & topic_partition, co { stalled_status = StalledStatus::NO_MESSAGES_RETURNED; - // we already wait enough for assignment in the past, - // let's make polls shorter and not block other consumer - // which can work successfully in parallel - // POLL_TIMEOUT_WO_ASSIGNMENT_MS (50ms) is 100% enough just to check if we got assignment - // (see https://github.com/ClickHouse/ClickHouse/issues/11218) - auto actual_poll_timeout_ms = (waited_for_assignment >= MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS) - ? std::min(POLL_TIMEOUT_WO_ASSIGNMENT_MS, poll_timeout) - : poll_timeout; - - auto & queue_to_poll_from = queues[topic_partition]; - queue_to_poll_from.forward_to_queue(consumer->get_consumer_queue()); - SCOPE_EXIT({ queue_to_poll_from.disable_queue_forwarding(); }); - + auto & queue_to_poll_from = queues.at(topic_partition); + LOG_TRACE(log, "Batch size {}, offset {}", batch_size, topic_partition.offset); const auto messages_to_pull = message_count.value_or(batch_size); /// Don't drop old messages immediately, since we may need them for virtual columns. - auto new_messages = consumer->poll_batch(messages_to_pull, std::chrono::milliseconds(actual_poll_timeout_ms)); + auto new_messages = queue_to_poll_from.consume_batch(messages_to_pull, std::chrono::milliseconds(poll_timeout)); resetIfStopped(); if (stalled_status == StalledStatus::CONSUMER_STOPPED) From 266ef9081ddbeb200db251a477938cd0cfe5fc96 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 27 Feb 2024 15:29:30 +0000 Subject: [PATCH 015/644] Make new StorageKafka able to run multiple threads --- src/Storages/Kafka/StorageKafka2.cpp | 143 +++++++++++---------------- 1 file changed, 60 insertions(+), 83 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index c658174b9d5..e465a3835b5 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -125,11 +125,8 @@ StorageKafka2::StorageKafka2( , thread_per_consumer(kafka_settings->kafka_thread_per_consumer.value) , collection_name(collection_name_) { - if (kafka_settings->kafka_num_consumers != 1) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Multiple consumers not yet implemented!"); - - if (thread_per_consumer) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "The new Kafka storage cannot use multiple threads yet!"); + if (kafka_settings->kafka_num_consumers > 1 && !thread_per_consumer) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "With multiple consumer you have to use thread per consumer!"); if (kafka_settings->kafka_handle_error_mode == StreamingHandleErrorMode::STREAM) { @@ -239,7 +236,6 @@ StorageKafka2::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapsho return std::make_shared(header, getFormatName(), max_rows, std::move(producer), getName(), modified_context); } - void StorageKafka2::startup() { for (size_t i = 0; i < num_consumers; ++i) @@ -445,9 +441,9 @@ void StorageKafka2::updateConfiguration(cppkafka::Configuration & kafka_config) // No need to add any prefix, messages can be distinguished kafka_config.set_log_callback( - [this](cppkafka::KafkaHandleBase &, int /*level*/, const std::string & facility, const std::string & message) + [this](cppkafka::KafkaHandleBase &, int level, const std::string & facility, const std::string & message) { - auto [poco_level, client_logs_level] = parseSyslogLevel(1); + auto [poco_level, client_logs_level] = parseSyslogLevel(level); LOG_IMPL(log, client_logs_level, poco_level, "[rdk:{}] {}", facility, message); }); @@ -555,7 +551,7 @@ void StorageKafka2::createKeeperNodes(const KafkaConsumer2Ptr & consumer) std::optional StorageKafka2::lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const TopicPartitions & topic_partitions) { - // TODO(antaljanosbenjamin): Review this function with somebody who know keeper better than me + // TODO(antaljanosbenjamin): Review this function with somebody who knows keeper better than me const auto uuid_as_string = toString(uuid); std::vector topic_partition_paths; @@ -565,35 +561,21 @@ StorageKafka2::lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const Topi Coordination::Requests ops; - // for (const auto & topic_partition_path : topic_partition_paths) - // ops.push_back(zkutil::makeCheckRequest(topic_partition_path + lock_file_name, -1)); - for (const auto & topic_partition_path : topic_partition_paths) ops.push_back(zkutil::makeCreateRequest(topic_partition_path + lock_file_name, uuid_as_string, zkutil::CreateMode::Ephemeral)); - bool success = false; - for (auto try_count{0}; try_count < 10; ++try_count) - { Coordination::Responses responses; - if (const auto code = keeper_to_use.tryMulti(ops, responses); code == Coordination::Error::ZOK) + if (const auto code = keeper_to_use.tryMulti(ops, responses); code != Coordination::Error::ZOK) { - success = true; - break; + if (code != Coordination::Error::ZNODEEXISTS) + zkutil::KeeperMultiException::check(code, ops, responses); + + // TODO(antaljanosbenjamin): maybe check the content, if we have the locks, we can continue with them + return std::nullopt; } - else if (code != Coordination::Error::ZNODEEXISTS) - zkutil::KeeperMultiException::check(code, ops, responses); - // TODO(antaljanosbenjamin): We shouldn't wait here, but let's give the other consumers to release the locks - for (const auto & topic_partition_path : topic_partition_paths) - keeper_to_use.waitForDisappear(topic_partition_path + lock_file_name); - } - - if (!success) - return std::nullopt; - - - // We have the locks + // We have the locks, let's gather the information we needed TopicPartitionLocks locks; { auto tp_it = topic_partitions.begin(); @@ -915,56 +897,52 @@ bool StorageKafka2::streamToViews(size_t idx) LOG_TRACE(log, "Polling consumer for events"); consumer->pollEvents(); - if (consumer->needsOffsetUpdate() || consumer_info.locks.empty()) - { - // First release the locks so let other consumers acquire them ASAP - consumer_info.locks.clear(); - - const auto * current_assignment = consumer->getKafkaAssignment(); - if (current_assignment == nullptr) - { - // The consumer lost its assignment and haven't received a new one. - // TODO(antaljanosbenjamin): returning a proper value representing the state - // By returning true this function reports the current consumer as a "stalled" stream, which - return true; - } - LOG_TRACE(log, "Consumer needs update offset"); - consumer_info.consume_from_topic_partition_index = 0; - - consumer_info.locks.clear(); - consumer_info.topic_partitions.clear(); - - auto maybe_locks = lockTopicPartitions(*consumer_info.keeper, *current_assignment); - - if (!maybe_locks.has_value()) - { - // We couldn't acquire locks, probably some other consumers are still holding them. - return true; - } - - consumer_info.locks = std::move(*maybe_locks); - - consumer_info.topic_partitions.reserve(current_assignment->size()); - for (const auto & topic_partition : *current_assignment) - { - TopicPartition topic_partition_copy{topic_partition}; - if (const auto & maybe_committed_offset = consumer_info.locks.at(topic_partition).committed_offset; - maybe_committed_offset.has_value()) - topic_partition_copy.offset = *maybe_committed_offset + 1; - else - topic_partition_copy.offset = KafkaConsumer2::BEGINNING_OFFSET; - - consumer_info.topic_partitions.push_back(std::move(topic_partition_copy)); - } - consumer_info.consumer->updateOffsets(consumer_info.topic_partitions); - - } - - LOG_TRACE(log, "Consumer has assignment"); - - // Here we will try to pull messages regardless if we loose our assignment try { + if (consumer->needsOffsetUpdate() || consumer_info.locks.empty()) + { + // First release the locks so let other consumers acquire them ASAP + consumer_info.locks.clear(); + + const auto * current_assignment = consumer->getKafkaAssignment(); + if (current_assignment == nullptr) + { + // The consumer lost its assignment and haven't received a new one. + // TODO(antaljanosbenjamin): returning a proper value representing the state + // By returning true this function reports the current consumer as a "stalled" stream, which + return true; + } + LOG_TRACE(log, "Consumer needs update offset"); + consumer_info.consume_from_topic_partition_index = 0; + + consumer_info.locks.clear(); + consumer_info.topic_partitions.clear(); + + auto maybe_locks = lockTopicPartitions(*consumer_info.keeper, *current_assignment); + + if (!maybe_locks.has_value()) + { + // We couldn't acquire locks, probably some other consumers are still holding them. + return true; + } + + consumer_info.locks = std::move(*maybe_locks); + + consumer_info.topic_partitions.reserve(current_assignment->size()); + for (const auto & topic_partition : *current_assignment) + { + TopicPartition topic_partition_copy{topic_partition}; + if (const auto & maybe_committed_offset = consumer_info.locks.at(topic_partition).committed_offset; + maybe_committed_offset.has_value()) + topic_partition_copy.offset = *maybe_committed_offset + 1; + else + topic_partition_copy.offset = KafkaConsumer2::BEGINNING_OFFSET; + + consumer_info.topic_partitions.push_back(std::move(topic_partition_copy)); + } + consumer_info.consumer->updateOffsets(consumer_info.topic_partitions); + } + LOG_TRACE(log, "Trying to consume from consumer {}", idx); const auto maybe_rows = streamFromConsumer(consumer_info); if (maybe_rows.has_value()) @@ -982,13 +960,13 @@ bool StorageKafka2::streamToViews(size_t idx) { if (Coordination::isHardwareError(e.code)) { + // Clear ephemeral nodes here as we got a new keeper here consumer_info.locks.clear(); consumer_info.keeper = getZooKeeper(); + return true; } - else - throw; - // TODO(antaljanosbenjamin): Should we reschedule in case of keeper error? + throw; } return false; } @@ -1073,11 +1051,10 @@ std::optional StorageKafka2::streamFromConsumer(ConsumerAndAssignmentInf zkutil::ZooKeeperPtr StorageKafka2::getZooKeeper() { + std::unique_lock lock{keeper_mutex}; if (keeper->expired()) { - // TODO(antaljanosbenjamin): this can go wrong if we start a new session simultaneously from multiple threads. keeper = keeper->startNewSession(); - //TODO(antaljanosbenjamin): handle ephemeral nodes } return keeper; } From 71ea7be9d82aa2d65bdc242e1a3ca37f78ea57e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 28 Feb 2024 10:27:19 +0000 Subject: [PATCH 016/644] Fix build --- src/Storages/Kafka/StorageKafka2.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index e465a3835b5..6e11a8f9264 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -564,16 +564,16 @@ StorageKafka2::lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const Topi for (const auto & topic_partition_path : topic_partition_paths) ops.push_back(zkutil::makeCreateRequest(topic_partition_path + lock_file_name, uuid_as_string, zkutil::CreateMode::Ephemeral)); - Coordination::Responses responses; + Coordination::Responses responses; - if (const auto code = keeper_to_use.tryMulti(ops, responses); code != Coordination::Error::ZOK) - { - if (code != Coordination::Error::ZNODEEXISTS) - zkutil::KeeperMultiException::check(code, ops, responses); + if (const auto code = keeper_to_use.tryMulti(ops, responses); code != Coordination::Error::ZOK) + { + if (code != Coordination::Error::ZNODEEXISTS) + zkutil::KeeperMultiException::check(code, ops, responses); - // TODO(antaljanosbenjamin): maybe check the content, if we have the locks, we can continue with them - return std::nullopt; - } + // TODO(antaljanosbenjamin): maybe check the content, if we have the locks, we can continue with them + return std::nullopt; + } // We have the locks, let's gather the information we needed TopicPartitionLocks locks; From d7b34a80bbbf98ea11e0d679eeede076421748f1 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Mar 2024 16:09:14 +0000 Subject: [PATCH 017/644] stash --- .../Net/include/Poco/Net/HTTPServerSession.h | 5 +++++ base/poco/Net/src/HTTPServerSession.cpp | 22 ++++++++++--------- .../library-bridge/LibraryBridgeHandlers.cpp | 16 +++++--------- .../library-bridge/LibraryBridgeHandlers.h | 6 ++--- programs/odbc-bridge/PingHandler.cpp | 2 +- programs/server/Server.cpp | 1 + src/Core/ServerSettings.h | 1 + src/IO/HTTPCommon.cpp | 15 +++++++++---- src/IO/HTTPCommon.h | 2 +- src/Server/HTTP/HTTPServer.cpp | 3 ++- src/Server/HTTP/HTTPServerResponse.h | 2 ++ .../WriteBufferFromHTTPServerResponse.cpp | 4 +--- .../HTTP/WriteBufferFromHTTPServerResponse.h | 2 -- src/Server/HTTPHandler.cpp | 3 +-- src/Server/InterserverIOHTTPHandler.cpp | 3 +-- src/Server/PrometheusRequestHandler.cpp | 10 ++------- src/Server/PrometheusRequestHandler.h | 7 +----- src/Server/ReplicasStatusHandler.cpp | 3 +-- src/Server/StaticRequestHandler.cpp | 10 ++++----- src/Server/WebUIRequestHandler.cpp | 9 +++----- .../00408_http_keep_alive.reference | 6 ++--- .../0_stateless/00501_http_head.reference | 4 ++-- 22 files changed, 63 insertions(+), 73 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPServerSession.h b/base/poco/Net/include/Poco/Net/HTTPServerSession.h index ec928af304f..192d71962bc 100644 --- a/base/poco/Net/include/Poco/Net/HTTPServerSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPServerSession.h @@ -56,10 +56,15 @@ namespace Net SocketAddress serverAddress(); /// Returns the server's address. + size_t getKeepAliveTimeout() const { return _params->getKeepAliveTimeout().totalSeconds(); } + + size_t getMaxKeepAliveRequests() const { return _params->getMaxKeepAliveRequests(); } + private: bool _firstRequest; Poco::Timespan _keepAliveTimeout; int _maxKeepAliveRequests; + HTTPServerParams::Ptr _params; }; diff --git a/base/poco/Net/src/HTTPServerSession.cpp b/base/poco/Net/src/HTTPServerSession.cpp index d4f2b24879e..3ea689cb0cf 100644 --- a/base/poco/Net/src/HTTPServerSession.cpp +++ b/base/poco/Net/src/HTTPServerSession.cpp @@ -19,11 +19,12 @@ namespace Poco { namespace Net { -HTTPServerSession::HTTPServerSession(const StreamSocket& socket, HTTPServerParams::Ptr pParams): - HTTPSession(socket, pParams->getKeepAlive()), - _firstRequest(true), - _keepAliveTimeout(pParams->getKeepAliveTimeout()), - _maxKeepAliveRequests(pParams->getMaxKeepAliveRequests()) +HTTPServerSession::HTTPServerSession(const StreamSocket & socket, HTTPServerParams::Ptr pParams) + : HTTPSession(socket, pParams->getKeepAlive()) + , _firstRequest(true) + , _keepAliveTimeout(pParams->getKeepAliveTimeout()) + , _maxKeepAliveRequests(pParams->getMaxKeepAliveRequests()) + , _params(pParams) { setTimeout(pParams->getTimeout()); } @@ -46,11 +47,12 @@ bool HTTPServerSession::hasMoreRequests() } else if (_maxKeepAliveRequests != 0 && getKeepAlive()) { - if (_maxKeepAliveRequests > 0) - --_maxKeepAliveRequests; - return buffered() > 0 || socket().poll(_keepAliveTimeout, Socket::SELECT_READ); - } - else return false; + if (_maxKeepAliveRequests > 0) + --_maxKeepAliveRequests; + return buffered() > 0 || socket().poll(_keepAliveTimeout, Socket::SELECT_READ); + } + else + return false; } diff --git a/programs/library-bridge/LibraryBridgeHandlers.cpp b/programs/library-bridge/LibraryBridgeHandlers.cpp index 26d887cfc98..094cef6716d 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.cpp +++ b/programs/library-bridge/LibraryBridgeHandlers.cpp @@ -374,10 +374,8 @@ void ExternalDictionaryLibraryBridgeRequestHandler::handleRequest(HTTPServerRequ } -ExternalDictionaryLibraryBridgeExistsHandler::ExternalDictionaryLibraryBridgeExistsHandler(size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , keep_alive_timeout(keep_alive_timeout_) - , log(getLogger("ExternalDictionaryLibraryBridgeExistsHandler")) +ExternalDictionaryLibraryBridgeExistsHandler::ExternalDictionaryLibraryBridgeExistsHandler(ContextPtr context_) + : WithContext(context_), log(getLogger("ExternalDictionaryLibraryBridgeExistsHandler")) { } @@ -401,7 +399,7 @@ void ExternalDictionaryLibraryBridgeExistsHandler::handleRequest(HTTPServerReque String res = library_handler ? "1" : "0"; - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response); LOG_TRACE(log, "Sending ping response: {} (dictionary id: {})", res, dictionary_id); response.sendBuffer(res.data(), res.size()); } @@ -617,10 +615,8 @@ void CatBoostLibraryBridgeRequestHandler::handleRequest(HTTPServerRequest & requ } -CatBoostLibraryBridgeExistsHandler::CatBoostLibraryBridgeExistsHandler(size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , keep_alive_timeout(keep_alive_timeout_) - , log(getLogger("CatBoostLibraryBridgeExistsHandler")) +CatBoostLibraryBridgeExistsHandler::CatBoostLibraryBridgeExistsHandler(ContextPtr context_) + : WithContext(context_), log(getLogger("CatBoostLibraryBridgeExistsHandler")) { } @@ -634,7 +630,7 @@ void CatBoostLibraryBridgeExistsHandler::handleRequest(HTTPServerRequest & reque String res = "1"; - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response); LOG_TRACE(log, "Sending ping response: {}", res); response.sendBuffer(res.data(), res.size()); } diff --git a/programs/library-bridge/LibraryBridgeHandlers.h b/programs/library-bridge/LibraryBridgeHandlers.h index 1db71eb24cb..83bca24ce1f 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.h +++ b/programs/library-bridge/LibraryBridgeHandlers.h @@ -34,12 +34,11 @@ private: class ExternalDictionaryLibraryBridgeExistsHandler : public HTTPRequestHandler, WithContext { public: - ExternalDictionaryLibraryBridgeExistsHandler(size_t keep_alive_timeout_, ContextPtr context_); + ExternalDictionaryLibraryBridgeExistsHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: - const size_t keep_alive_timeout; LoggerPtr log; }; @@ -77,12 +76,11 @@ private: class CatBoostLibraryBridgeExistsHandler : public HTTPRequestHandler, WithContext { public: - CatBoostLibraryBridgeExistsHandler(size_t keep_alive_timeout_, ContextPtr context_); + CatBoostLibraryBridgeExistsHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: - const size_t keep_alive_timeout; LoggerPtr log; }; diff --git a/programs/odbc-bridge/PingHandler.cpp b/programs/odbc-bridge/PingHandler.cpp index 80d0e2bf4a9..e5d094fb7eb 100644 --- a/programs/odbc-bridge/PingHandler.cpp +++ b/programs/odbc-bridge/PingHandler.cpp @@ -10,7 +10,7 @@ void PingHandler::handleRequest(HTTPServerRequest & /* request */, HTTPServerRes { try { - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response); const char * data = "Ok.\n"; response.sendBuffer(data, strlen(data)); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b67a4eccd15..b741cd7f644 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2251,6 +2251,7 @@ void Server::createServers( Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; http_params->setTimeout(settings.http_receive_timeout); http_params->setKeepAliveTimeout(global_context->getServerSettings().keep_alive_timeout); + http_params->setMaxKeepAliveRequests(static_cast(global_context->getServerSettings().max_keep_alive_requests)); Poco::Util::AbstractConfiguration::Keys protocols; config.keys("protocols", protocols); diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index da82cdea5a4..7480d94e81d 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -113,6 +113,7 @@ namespace DB M(Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0) \ M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \ M(Seconds, keep_alive_timeout, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT, "The number of seconds that ClickHouse waits for incoming requests before closing the connection.", 0) \ + M(UInt64, max_keep_alive_requests, 10000, "The number of seconds that ClickHouse waits for incoming requests before closing the connection.", 0) \ M(Seconds, replicated_fetches_http_connection_timeout, 0, "HTTP connection timeout for part fetch requests. Inherited from default profile `http_connection_timeout` if not set explicitly.", 0) \ M(Seconds, replicated_fetches_http_send_timeout, 0, "HTTP send timeout for part fetch requests. Inherited from default profile `http_send_timeout` if not set explicitly.", 0) \ M(Seconds, replicated_fetches_http_receive_timeout, 0, "HTTP receive timeout for fetch part requests. Inherited from default profile `http_receive_timeout` if not set explicitly.", 0) \ diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 09f7724d613..2b3f7f062bc 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -33,14 +34,20 @@ namespace ErrorCodes extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS; } -void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout) +void setResponseDefaultHeaders(HTTPServerResponse & response) { if (!response.getKeepAlive()) return; - Poco::Timespan timeout(keep_alive_timeout, 0); - if (timeout.totalSeconds()) - response.set("Keep-Alive", "timeout=" + std::to_string(timeout.totalSeconds())); + const size_t keep_alive_timeout = response.getSession().getKeepAliveTimeout(); + const size_t keep_alive_max_requests = response.getSession().getMaxKeepAliveRequests(); + if (keep_alive_timeout) + { + if (keep_alive_max_requests) + response.set("Keep-Alive", fmt::format("timeout={}, max={}", keep_alive_timeout, keep_alive_max_requests)); + else + response.set("Keep-Alive", fmt::format("timeout={}", keep_alive_timeout)); + } } HTTPSessionPtr makeHTTPSession( diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index 63dffcf6878..fa6086224f5 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -54,7 +54,7 @@ private: using HTTPSessionPtr = std::shared_ptr; -void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout); +void setResponseDefaultHeaders(HTTPServerResponse & response); /// Create session object to perform requests and set required parameters. HTTPSessionPtr makeHTTPSession( diff --git a/src/Server/HTTP/HTTPServer.cpp b/src/Server/HTTP/HTTPServer.cpp index 90bdebf6451..9b8feae3e26 100644 --- a/src/Server/HTTP/HTTPServer.cpp +++ b/src/Server/HTTP/HTTPServer.cpp @@ -13,7 +13,8 @@ HTTPServer::HTTPServer( Poco::Net::HTTPServerParams::Ptr params, const ProfileEvents::Event & read_event, const ProfileEvents::Event & write_event) - : TCPServer(new HTTPServerConnectionFactory(context, params, factory_, read_event, write_event), thread_pool, socket_, params), factory(factory_) + : TCPServer(new HTTPServerConnectionFactory(context, params, factory_, read_event, write_event), thread_pool, socket_, params) + , factory(factory_) { } diff --git a/src/Server/HTTP/HTTPServerResponse.h b/src/Server/HTTP/HTTPServerResponse.h index 8edb785e7c5..9793fc8b24b 100644 --- a/src/Server/HTTP/HTTPServerResponse.h +++ b/src/Server/HTTP/HTTPServerResponse.h @@ -245,6 +245,8 @@ public: void attachRequest(HTTPServerRequest * request_) { request = request_; } + const Poco::Net::HTTPServerSession & getSession() const { return session; } + private: Poco::Net::HTTPServerSession & session; HTTPServerRequest * request = nullptr; diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index 8098671a903..a39f6de51d0 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -30,7 +30,7 @@ void WriteBufferFromHTTPServerResponse::startSendHeaders() if (add_cors_header) response.set("Access-Control-Allow-Origin", "*"); - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response); std::stringstream header; //STYLE_CHECK_ALLOW_STD_STRING_STREAM response.beginWrite(header); @@ -119,12 +119,10 @@ void WriteBufferFromHTTPServerResponse::nextImpl() WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse( HTTPServerResponse & response_, bool is_http_method_head_, - UInt64 keep_alive_timeout_, const ProfileEvents::Event & write_event_) : HTTPWriteBuffer(response_.getSocket(), write_event_) , response(response_) , is_http_method_head(is_http_method_head_) - , keep_alive_timeout(keep_alive_timeout_) { } diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h index a3952b7c553..f0c80f24582 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h @@ -29,7 +29,6 @@ public: WriteBufferFromHTTPServerResponse( HTTPServerResponse & response_, bool is_http_method_head_, - UInt64 keep_alive_timeout_, const ProfileEvents::Event & write_event_ = ProfileEvents::end()); ~WriteBufferFromHTTPServerResponse() override; @@ -91,7 +90,6 @@ private: bool is_http_method_head; bool add_cors_header = false; - size_t keep_alive_timeout = 0; bool initialized = false; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index c112eefec6c..ac6c9d6a0a5 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -621,7 +621,6 @@ void HTTPHandler::processQuery( std::make_shared( response, request.getMethod() == HTTPRequest::HTTP_HEAD, - context->getServerSettings().keep_alive_timeout.totalSeconds(), write_event); used_output.out = used_output.out_holder; used_output.out_maybe_compressed = used_output.out_holder; @@ -926,7 +925,7 @@ try if (!used_output.out_holder && !used_output.exception_is_written) { /// If nothing was sent yet and we don't even know if we must compress the response. - WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT).writeln(s); + WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD).writeln(s); } else if (used_output.out_maybe_compressed) { diff --git a/src/Server/InterserverIOHTTPHandler.cpp b/src/Server/InterserverIOHTTPHandler.cpp index 28045380cd7..9a87992731c 100644 --- a/src/Server/InterserverIOHTTPHandler.cpp +++ b/src/Server/InterserverIOHTTPHandler.cpp @@ -87,9 +87,8 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe response.setChunkedTransferEncoding(true); Output used_output; - const auto keep_alive_timeout = server.context()->getServerSettings().keep_alive_timeout.totalSeconds(); used_output.out = std::make_shared( - response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout, write_event); + response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, write_event); auto finalize_output = [&] { diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index dff960f7031..0ad5f907467 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -18,21 +18,15 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe { try { - /// Raw config reference is used here to avoid dependency on Context and ServerSettings. - /// This is painful, because this class is also used in a build with CLICKHOUSE_KEEPER_STANDALONE_BUILD=1 - /// And there ordinary Context is replaced with a tiny clone. - const auto & config = server.config(); - unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT); - /// In order to make keep-alive works. if (request.getVersion() == HTTPServerRequest::HTTP_1_1) response.setChunkedTransferEncoding(true); - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response); response.setContentType("text/plain; version=0.0.4; charset=UTF-8"); - WriteBufferFromHTTPServerResponse wb(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout, write_event); + WriteBufferFromHTTPServerResponse wb(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, write_event); try { metrics_writer->write(wb); diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index d120752c8c5..cc7848d1dd0 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -12,15 +12,10 @@ class IServer; class PrometheusRequestHandler : public HTTPRequestHandler { private: - IServer & server; PrometheusMetricsWriterPtr metrics_writer; public: - PrometheusRequestHandler(IServer & server_, PrometheusMetricsWriterPtr metrics_writer_) - : server(server_) - , metrics_writer(std::move(metrics_writer_)) - { - } + PrometheusRequestHandler(IServer &, PrometheusMetricsWriterPtr metrics_writer_) : metrics_writer(std::move(metrics_writer_)) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; }; diff --git a/src/Server/ReplicasStatusHandler.cpp b/src/Server/ReplicasStatusHandler.cpp index 91c6bd722d3..964e3834037 100644 --- a/src/Server/ReplicasStatusHandler.cpp +++ b/src/Server/ReplicasStatusHandler.cpp @@ -84,8 +84,7 @@ void ReplicasStatusHandler::handleRequest(HTTPServerRequest & request, HTTPServe } } - const auto & server_settings = getContext()->getServerSettings(); - setResponseDefaultHeaders(response, server_settings.keep_alive_timeout.totalSeconds()); + setResponseDefaultHeaders(response); if (!ok) { diff --git a/src/Server/StaticRequestHandler.cpp b/src/Server/StaticRequestHandler.cpp index 67bf3875de4..3d618031875 100644 --- a/src/Server/StaticRequestHandler.cpp +++ b/src/Server/StaticRequestHandler.cpp @@ -33,10 +33,9 @@ namespace ErrorCodes extern const int INVALID_CONFIG_PARAMETER; } -static inline std::unique_ptr -responseWriteBuffer(HTTPServerRequest & request, HTTPServerResponse & response, UInt64 keep_alive_timeout) +static inline std::unique_ptr responseWriteBuffer(HTTPServerRequest & request, HTTPServerResponse & response) { - auto buf = std::unique_ptr(new WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, keep_alive_timeout)); + auto buf = std::unique_ptr(new WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD)); /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). String http_response_compression_methods = request.get("Accept-Encoding", ""); @@ -89,8 +88,7 @@ static inline void trySendExceptionToClient( void StaticRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & /*write_event*/) { - auto keep_alive_timeout = server.context()->getServerSettings().keep_alive_timeout.totalSeconds(); - auto out = responseWriteBuffer(request, response, keep_alive_timeout); + auto out = responseWriteBuffer(request, response); try { @@ -105,7 +103,7 @@ void StaticRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServer "The Transfer-Encoding is not chunked and there " "is no Content-Length header for POST request"); - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response); response.setStatusAndReason(Poco::Net::HTTPResponse::HTTPStatus(status)); writeResponse(*out); } diff --git a/src/Server/WebUIRequestHandler.cpp b/src/Server/WebUIRequestHandler.cpp index 68d3ff0b325..faad9d57519 100644 --- a/src/Server/WebUIRequestHandler.cpp +++ b/src/Server/WebUIRequestHandler.cpp @@ -29,18 +29,15 @@ DashboardWebUIRequestHandler::DashboardWebUIRequestHandler(IServer & server_) : BinaryWebUIRequestHandler::BinaryWebUIRequestHandler(IServer & server_) : server(server_) {} JavaScriptWebUIRequestHandler::JavaScriptWebUIRequestHandler(IServer & server_) : server(server_) {} -static void handle(const IServer & server, HTTPServerRequest & request, HTTPServerResponse & response, std::string_view html) +static void handle(const IServer &, HTTPServerRequest & request, HTTPServerResponse & response, std::string_view html) { - auto keep_alive_timeout = server.context()->getServerSettings().keep_alive_timeout.totalSeconds(); - response.setContentType("text/html; charset=UTF-8"); if (request.getVersion() == HTTPServerRequest::HTTP_1_1) response.setChunkedTransferEncoding(true); - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response); response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_OK); - WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, keep_alive_timeout).write(html.data(), html.size()); - + WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD).write(html.data(), html.size()); } void PlayWebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event &) diff --git a/tests/queries/0_stateless/00408_http_keep_alive.reference b/tests/queries/0_stateless/00408_http_keep_alive.reference index 17a7fd690a8..d5d7dacce9e 100644 --- a/tests/queries/0_stateless/00408_http_keep_alive.reference +++ b/tests/queries/0_stateless/00408_http_keep_alive.reference @@ -1,6 +1,6 @@ < Connection: Keep-Alive -< Keep-Alive: timeout=10 +< Keep-Alive: timeout=10, max=10000 < Connection: Keep-Alive -< Keep-Alive: timeout=10 +< Keep-Alive: timeout=10, max=10000 < Connection: Keep-Alive -< Keep-Alive: timeout=10 +< Keep-Alive: timeout=10, max=10000 diff --git a/tests/queries/0_stateless/00501_http_head.reference b/tests/queries/0_stateless/00501_http_head.reference index 8351327b356..db82132b145 100644 --- a/tests/queries/0_stateless/00501_http_head.reference +++ b/tests/queries/0_stateless/00501_http_head.reference @@ -2,11 +2,11 @@ HTTP/1.1 200 OK Connection: Keep-Alive Content-Type: text/tab-separated-values; charset=UTF-8 Transfer-Encoding: chunked -Keep-Alive: timeout=10 +Keep-Alive: timeout=10, max=10000 HTTP/1.1 200 OK Connection: Keep-Alive Content-Type: text/tab-separated-values; charset=UTF-8 Transfer-Encoding: chunked -Keep-Alive: timeout=10 +Keep-Alive: timeout=10, max=10000 From 7aace4d876173ce18ade57ec1bdc332efff7ce80 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Mar 2024 17:24:25 +0000 Subject: [PATCH 018/644] add test --- .../test_server_keep_alive/__init__.py | 0 .../configs/keep_alive_settings.xml | 4 ++ .../test_server_keep_alive/test.py | 46 +++++++++++++++++++ 3 files changed, 50 insertions(+) create mode 100644 tests/integration/test_server_keep_alive/__init__.py create mode 100644 tests/integration/test_server_keep_alive/configs/keep_alive_settings.xml create mode 100644 tests/integration/test_server_keep_alive/test.py diff --git a/tests/integration/test_server_keep_alive/__init__.py b/tests/integration/test_server_keep_alive/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_server_keep_alive/configs/keep_alive_settings.xml b/tests/integration/test_server_keep_alive/configs/keep_alive_settings.xml new file mode 100644 index 00000000000..06e68044817 --- /dev/null +++ b/tests/integration/test_server_keep_alive/configs/keep_alive_settings.xml @@ -0,0 +1,4 @@ + + 3600 + 5 + diff --git a/tests/integration/test_server_keep_alive/test.py b/tests/integration/test_server_keep_alive/test.py new file mode 100644 index 00000000000..0f88fe47673 --- /dev/null +++ b/tests/integration/test_server_keep_alive/test.py @@ -0,0 +1,46 @@ +import logging +import pytest +import requests + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node", main_configs=["configs/keep_alive_settings.xml"]) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def test_requests_with_keep_alive(start_cluster): + # In this test we have `keep_alive_timeout` set to one hour to never trigger connection reset by timeout, `max_keep_alive_requests` is set to 5. + # We expect server to close connection after each 5 requests. We detect connection reset by change in src port. + # So the first 5 requests should come from the same port, the following 5 requests should come from another port. + session = requests.Session() + for i in range(10): + session.get( + f"http://{node.ip_address}:8123/?query=select%201&log_comment=test_requests_with_keep_alive_{i}" + ) + + ports = node.query( + """ + SYSTEM FLUSH LOGS; + + SELECT port + FROM system.query_log + WHERE log_comment like 'test_requests_with_keep_alive_%' AND type = 'QueryFinish' + ORDER BY log_comment + """ + ).split("\n")[:-1] + + expected = 5 * [ports[0]] + [ports[5]] * 5 + + assert ports == expected From 146d7603388ca161ee3340ab1f582971a4e45a03 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Mar 2024 17:38:06 +0000 Subject: [PATCH 019/644] rm more --- programs/keeper/Keeper.cpp | 2 +- src/IO/HTTPCommon.cpp | 1 - src/Server/HTTP/HTTPServer.cpp | 3 +-- src/Server/HTTPHandlerFactory.cpp | 10 ++++------ src/Server/HTTPHandlerFactory.h | 8 ++------ src/Server/PrometheusRequestHandler.cpp | 13 +++---------- src/Server/PrometheusRequestHandler.h | 2 +- src/Server/WebUIRequestHandler.cpp | 14 +++++++------- 8 files changed, 19 insertions(+), 34 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index a558ed64bf9..238964fb25e 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -507,7 +507,7 @@ try "Prometheus: http://" + address.toString(), std::make_unique( std::move(my_http_context), - createPrometheusMainHandlerFactory(*this, config_getter(), metrics_writer, "PrometheusHandler-factory"), + createPrometheusMainHandlerFactory(config_getter(), metrics_writer, "PrometheusHandler-factory"), server_pool, socket, http_params)); diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 2b3f7f062bc..56226941228 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -1,4 +1,3 @@ -#include #include #include diff --git a/src/Server/HTTP/HTTPServer.cpp b/src/Server/HTTP/HTTPServer.cpp index 9b8feae3e26..90bdebf6451 100644 --- a/src/Server/HTTP/HTTPServer.cpp +++ b/src/Server/HTTP/HTTPServer.cpp @@ -13,8 +13,7 @@ HTTPServer::HTTPServer( Poco::Net::HTTPServerParams::Ptr params, const ProfileEvents::Event & read_event, const ProfileEvents::Event & write_event) - : TCPServer(new HTTPServerConnectionFactory(context, params, factory_, read_event, write_event), thread_pool, socket_, params) - , factory(factory_) + : TCPServer(new HTTPServerConnectionFactory(context, params, factory_, read_event, write_event), thread_pool, socket_, params), factory(factory_) { } diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 9a67e576345..23d4c081d2d 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -123,7 +123,7 @@ static inline auto createHandlersFactoryFromConfig( } else if (handler_type == "prometheus") { - main_handler_factory->addHandler(createPrometheusHandlerFactory(server, config, async_metrics, prefix + "." + key)); + main_handler_factory->addHandler(createPrometheusHandlerFactory(config, async_metrics, prefix + "." + key)); } else if (handler_type == "replicas_status") { @@ -202,7 +202,7 @@ HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco:: else if (name == "PrometheusHandler-factory") { auto metrics_writer = std::make_shared(config, "prometheus", async_metrics); - return createPrometheusMainHandlerFactory(server, config, metrics_writer, name); + return createPrometheusMainHandlerFactory(config, metrics_writer, name); } throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown HTTP handler factory name."); @@ -294,10 +294,8 @@ void addDefaultHandlersFactory( if (config.has("prometheus") && config.getInt("prometheus.port", 0) == 0) { auto writer = std::make_shared(config, "prometheus", async_metrics); - auto creator = [&server, writer] () -> std::unique_ptr - { - return std::make_unique(server, writer); - }; + auto creator + = [writer]() -> std::unique_ptr { return std::make_unique(writer); }; auto prometheus_handler = std::make_shared>(std::move(creator)); prometheus_handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); prometheus_handler->allowGetAndHeadRequest(); diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index ac18c36e6c9..5c1a12d9e06 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -126,14 +126,10 @@ HTTPRequestHandlerFactoryPtr createReplicasStatusHandlerFactory(IServer & server const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); -HTTPRequestHandlerFactoryPtr -createPrometheusHandlerFactory(IServer & server, - const Poco::Util::AbstractConfiguration & config, - AsynchronousMetrics & async_metrics, - const std::string & config_prefix); +HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactory( + const Poco::Util::AbstractConfiguration & config, AsynchronousMetrics & async_metrics, const std::string & config_prefix); HTTPRequestHandlerFactoryPtr createPrometheusMainHandlerFactory( - IServer & server, const Poco::Util::AbstractConfiguration & config, PrometheusMetricsWriterPtr metrics_writer, const std::string & name); diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index 0ad5f907467..1a04311116f 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -44,16 +44,12 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe } HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactory( - IServer & server, const Poco::Util::AbstractConfiguration & config, AsynchronousMetrics & async_metrics, const std::string & config_prefix) { auto writer = std::make_shared(config, config_prefix + ".handler", async_metrics); - auto creator = [&server, writer]() -> std::unique_ptr - { - return std::make_unique(server, writer); - }; + auto creator = [writer]() -> std::unique_ptr { return std::make_unique(writer); }; auto factory = std::make_shared>(std::move(creator)); factory->addFiltersFromConfig(config, config_prefix); @@ -61,13 +57,10 @@ HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactory( } HTTPRequestHandlerFactoryPtr createPrometheusMainHandlerFactory( - IServer & server, const Poco::Util::AbstractConfiguration & config, PrometheusMetricsWriterPtr metrics_writer, const std::string & name) + const Poco::Util::AbstractConfiguration & config, PrometheusMetricsWriterPtr metrics_writer, const std::string & name) { auto factory = std::make_shared(name); - auto creator = [&server, metrics_writer] - { - return std::make_unique(server, metrics_writer); - }; + auto creator = [metrics_writer] { return std::make_unique(metrics_writer); }; auto handler = std::make_shared>(std::move(creator)); handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index cc7848d1dd0..7f4d3c14f62 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -15,7 +15,7 @@ private: PrometheusMetricsWriterPtr metrics_writer; public: - PrometheusRequestHandler(IServer &, PrometheusMetricsWriterPtr metrics_writer_) : metrics_writer(std::move(metrics_writer_)) { } + PrometheusRequestHandler(PrometheusMetricsWriterPtr metrics_writer_) : metrics_writer(std::move(metrics_writer_)) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; }; diff --git a/src/Server/WebUIRequestHandler.cpp b/src/Server/WebUIRequestHandler.cpp index faad9d57519..e43412550f9 100644 --- a/src/Server/WebUIRequestHandler.cpp +++ b/src/Server/WebUIRequestHandler.cpp @@ -29,7 +29,7 @@ DashboardWebUIRequestHandler::DashboardWebUIRequestHandler(IServer & server_) : BinaryWebUIRequestHandler::BinaryWebUIRequestHandler(IServer & server_) : server(server_) {} JavaScriptWebUIRequestHandler::JavaScriptWebUIRequestHandler(IServer & server_) : server(server_) {} -static void handle(const IServer &, HTTPServerRequest & request, HTTPServerResponse & response, std::string_view html) +static void handle(HTTPServerRequest & request, HTTPServerResponse & response, std::string_view html) { response.setContentType("text/html; charset=UTF-8"); if (request.getVersion() == HTTPServerRequest::HTTP_1_1) @@ -42,7 +42,7 @@ static void handle(const IServer &, HTTPServerRequest & request, HTTPServerRespo void PlayWebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event &) { - handle(server, request, response, {reinterpret_cast(gresource_play_htmlData), gresource_play_htmlSize}); + handle(request, response, {reinterpret_cast(gresource_play_htmlData), gresource_play_htmlSize}); } void DashboardWebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event &) @@ -60,23 +60,23 @@ void DashboardWebUIRequestHandler::handleRequest(HTTPServerRequest & request, HT static re2::RE2 lz_string_url = R"(https://[^\s"'`]+lz-string[^\s"'`]*\.js)"; RE2::Replace(&html, lz_string_url, "/js/lz-string.js"); - handle(server, request, response, html); + handle(request, response, html); } void BinaryWebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event &) { - handle(server, request, response, {reinterpret_cast(gresource_binary_htmlData), gresource_binary_htmlSize}); + handle(request, response, {reinterpret_cast(gresource_binary_htmlData), gresource_binary_htmlSize}); } void JavaScriptWebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event &) { if (request.getURI() == "/js/uplot.js") { - handle(server, request, response, {reinterpret_cast(gresource_uplot_jsData), gresource_uplot_jsSize}); + handle(request, response, {reinterpret_cast(gresource_uplot_jsData), gresource_uplot_jsSize}); } else if (request.getURI() == "/js/lz-string.js") { - handle(server, request, response, {reinterpret_cast(gresource_lz_string_jsData), gresource_lz_string_jsSize}); + handle(request, response, {reinterpret_cast(gresource_lz_string_jsData), gresource_lz_string_jsSize}); } else { @@ -84,7 +84,7 @@ void JavaScriptWebUIRequestHandler::handleRequest(HTTPServerRequest & request, H *response.send() << "Not found.\n"; } - handle(server, request, response, {reinterpret_cast(gresource_binary_htmlData), gresource_binary_htmlSize}); + handle(request, response, {reinterpret_cast(gresource_binary_htmlData), gresource_binary_htmlSize}); } } From bd04fc5346d83e8450fa98578e325923a609abda Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Mar 2024 17:41:53 +0000 Subject: [PATCH 020/644] rename test --- tests/integration/test_server_keep_alive/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_server_keep_alive/test.py b/tests/integration/test_server_keep_alive/test.py index 0f88fe47673..96f08a37adb 100644 --- a/tests/integration/test_server_keep_alive/test.py +++ b/tests/integration/test_server_keep_alive/test.py @@ -20,7 +20,7 @@ def start_cluster(): cluster.shutdown() -def test_requests_with_keep_alive(start_cluster): +def test_max_keep_alive_requests_on_user_side(start_cluster): # In this test we have `keep_alive_timeout` set to one hour to never trigger connection reset by timeout, `max_keep_alive_requests` is set to 5. # We expect server to close connection after each 5 requests. We detect connection reset by change in src port. # So the first 5 requests should come from the same port, the following 5 requests should come from another port. From c153fae0b8377aeca5e636a7ad0370e6ada42688 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Mar 2024 19:10:06 +0000 Subject: [PATCH 021/644] add docs --- .../server-configuration-parameters/settings.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 07c9a2b88ab..0efb5a9e6e4 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1298,6 +1298,16 @@ The number of seconds that ClickHouse waits for incoming requests before closing 10 ``` +## max_keep_alive_requests {#max-keep-alive-requests} + +Maximal number of requests through a single keep-alive connection until it will be closed by ClickHouse server. Default to 10000. + +**Example** + +``` xml +10 +``` + ## listen_host {#listen_host} Restriction on hosts that requests can come from. If you want the server to answer all of them, specify `::`. From b93f483a0e2f312d50685fd499d2f52717b83925 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Mar 2024 20:07:12 +0000 Subject: [PATCH 022/644] fix build --- programs/library-bridge/LibraryBridge.cpp | 2 +- .../LibraryBridgeHandlerFactory.cpp | 10 ++++------ .../LibraryBridgeHandlerFactory.h | 2 -- .../library-bridge/LibraryBridgeHandlers.cpp | 17 ++++++----------- programs/library-bridge/LibraryBridgeHandlers.h | 6 ++---- programs/odbc-bridge/ColumnInfoHandler.cpp | 5 +---- programs/odbc-bridge/ColumnInfoHandler.h | 8 +------- programs/odbc-bridge/IdentifierQuoteHandler.cpp | 2 +- programs/odbc-bridge/IdentifierQuoteHandler.h | 8 +------- programs/odbc-bridge/MainHandler.cpp | 2 +- programs/odbc-bridge/MainHandler.h | 3 --- programs/odbc-bridge/ODBCHandlerFactory.cpp | 10 +++++----- programs/odbc-bridge/SchemaAllowedHandler.cpp | 2 +- programs/odbc-bridge/SchemaAllowedHandler.h | 8 +------- 14 files changed, 25 insertions(+), 60 deletions(-) diff --git a/programs/library-bridge/LibraryBridge.cpp b/programs/library-bridge/LibraryBridge.cpp index 8a07ca57104..f86e469a307 100644 --- a/programs/library-bridge/LibraryBridge.cpp +++ b/programs/library-bridge/LibraryBridge.cpp @@ -25,7 +25,7 @@ std::string LibraryBridge::bridgeName() const LibraryBridge::HandlerFactoryPtr LibraryBridge::getHandlerFactoryPtr(ContextPtr context) const { - return std::make_shared("LibraryRequestHandlerFactory", keep_alive_timeout, context); + return std::make_shared("LibraryRequestHandlerFactory", context); } } diff --git a/programs/library-bridge/LibraryBridgeHandlerFactory.cpp b/programs/library-bridge/LibraryBridgeHandlerFactory.cpp index e5ab22f2d40..234904c6265 100644 --- a/programs/library-bridge/LibraryBridgeHandlerFactory.cpp +++ b/programs/library-bridge/LibraryBridgeHandlerFactory.cpp @@ -9,12 +9,10 @@ namespace DB { LibraryBridgeHandlerFactory::LibraryBridgeHandlerFactory( const std::string & name_, - size_t keep_alive_timeout_, ContextPtr context_) : WithContext(context_) , log(getLogger(name_)) , name(name_) - , keep_alive_timeout(keep_alive_timeout_) { } @@ -26,17 +24,17 @@ std::unique_ptr LibraryBridgeHandlerFactory::createRequestHa if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) { if (uri.getPath() == "/extdict_ping") - return std::make_unique(keep_alive_timeout, getContext()); + return std::make_unique(getContext()); else if (uri.getPath() == "/catboost_ping") - return std::make_unique(keep_alive_timeout, getContext()); + return std::make_unique(getContext()); } if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) { if (uri.getPath() == "/extdict_request") - return std::make_unique(keep_alive_timeout, getContext()); + return std::make_unique(getContext()); else if (uri.getPath() == "/catboost_request") - return std::make_unique(keep_alive_timeout, getContext()); + return std::make_unique(getContext()); } return nullptr; diff --git a/programs/library-bridge/LibraryBridgeHandlerFactory.h b/programs/library-bridge/LibraryBridgeHandlerFactory.h index 5b0f088bc29..c65394efa3b 100644 --- a/programs/library-bridge/LibraryBridgeHandlerFactory.h +++ b/programs/library-bridge/LibraryBridgeHandlerFactory.h @@ -13,7 +13,6 @@ class LibraryBridgeHandlerFactory : public HTTPRequestHandlerFactory, WithContex public: LibraryBridgeHandlerFactory( const std::string & name_, - size_t keep_alive_timeout_, ContextPtr context_); std::unique_ptr createRequestHandler(const HTTPServerRequest & request) override; @@ -21,7 +20,6 @@ public: private: LoggerPtr log; const std::string name; - const size_t keep_alive_timeout; }; } diff --git a/programs/library-bridge/LibraryBridgeHandlers.cpp b/programs/library-bridge/LibraryBridgeHandlers.cpp index 094cef6716d..bd8faf76188 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.cpp +++ b/programs/library-bridge/LibraryBridgeHandlers.cpp @@ -86,10 +86,8 @@ static void writeData(Block data, OutputFormatPtr format) } -ExternalDictionaryLibraryBridgeRequestHandler::ExternalDictionaryLibraryBridgeRequestHandler(size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , keep_alive_timeout(keep_alive_timeout_) - , log(getLogger("ExternalDictionaryLibraryBridgeRequestHandler")) +ExternalDictionaryLibraryBridgeRequestHandler::ExternalDictionaryLibraryBridgeRequestHandler(ContextPtr context_) + : WithContext(context_), log(getLogger("ExternalDictionaryLibraryBridgeRequestHandler")) { } @@ -136,7 +134,7 @@ void ExternalDictionaryLibraryBridgeRequestHandler::handleRequest(HTTPServerRequ const String & dictionary_id = params.get("dictionary_id"); LOG_TRACE(log, "Library method: '{}', dictionary id: {}", method, dictionary_id); - WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); + WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD); try { @@ -410,11 +408,8 @@ void ExternalDictionaryLibraryBridgeExistsHandler::handleRequest(HTTPServerReque } -CatBoostLibraryBridgeRequestHandler::CatBoostLibraryBridgeRequestHandler( - size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , keep_alive_timeout(keep_alive_timeout_) - , log(getLogger("CatBoostLibraryBridgeRequestHandler")) +CatBoostLibraryBridgeRequestHandler::CatBoostLibraryBridgeRequestHandler(ContextPtr context_) + : WithContext(context_), log(getLogger("CatBoostLibraryBridgeRequestHandler")) { } @@ -453,7 +448,7 @@ void CatBoostLibraryBridgeRequestHandler::handleRequest(HTTPServerRequest & requ const String & method = params.get("method"); LOG_TRACE(log, "Library method: '{}'", method); - WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); + WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD); try { diff --git a/programs/library-bridge/LibraryBridgeHandlers.h b/programs/library-bridge/LibraryBridgeHandlers.h index 83bca24ce1f..70e3c9c78da 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.h +++ b/programs/library-bridge/LibraryBridgeHandlers.h @@ -18,14 +18,13 @@ namespace DB class ExternalDictionaryLibraryBridgeRequestHandler : public HTTPRequestHandler, WithContext { public: - ExternalDictionaryLibraryBridgeRequestHandler(size_t keep_alive_timeout_, ContextPtr context_); + ExternalDictionaryLibraryBridgeRequestHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: static constexpr inline auto FORMAT = "RowBinary"; - const size_t keep_alive_timeout; LoggerPtr log; }; @@ -62,12 +61,11 @@ private: class CatBoostLibraryBridgeRequestHandler : public HTTPRequestHandler, WithContext { public: - CatBoostLibraryBridgeRequestHandler(size_t keep_alive_timeout_, ContextPtr context_); + CatBoostLibraryBridgeRequestHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: - const size_t keep_alive_timeout; LoggerPtr log; }; diff --git a/programs/odbc-bridge/ColumnInfoHandler.cpp b/programs/odbc-bridge/ColumnInfoHandler.cpp index 4cb15de3b2c..438062e8169 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -200,10 +200,7 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ if (columns.empty()) throw Exception(ErrorCodes::UNKNOWN_TABLE, "Columns definition was not returned"); - WriteBufferFromHTTPServerResponse out( - response, - request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, - keep_alive_timeout); + WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD); try { writeStringBinary(columns.toString(), out); diff --git a/programs/odbc-bridge/ColumnInfoHandler.h b/programs/odbc-bridge/ColumnInfoHandler.h index ca7044fdf32..f16e09ec3f9 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.h +++ b/programs/odbc-bridge/ColumnInfoHandler.h @@ -16,18 +16,12 @@ namespace DB class ODBCColumnsInfoHandler : public HTTPRequestHandler, WithContext { public: - ODBCColumnsInfoHandler(size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , log(getLogger("ODBCColumnsInfoHandler")) - , keep_alive_timeout(keep_alive_timeout_) - { - } + ODBCColumnsInfoHandler(ContextPtr context_) : WithContext(context_), log(getLogger("ODBCColumnsInfoHandler")) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: LoggerPtr log; - size_t keep_alive_timeout; }; } diff --git a/programs/odbc-bridge/IdentifierQuoteHandler.cpp b/programs/odbc-bridge/IdentifierQuoteHandler.cpp index cf5acdc4534..0bd1e8758cd 100644 --- a/programs/odbc-bridge/IdentifierQuoteHandler.cpp +++ b/programs/odbc-bridge/IdentifierQuoteHandler.cpp @@ -73,7 +73,7 @@ void IdentifierQuoteHandler::handleRequest(HTTPServerRequest & request, HTTPServ auto identifier = getIdentifierQuote(std::move(connection)); - WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); + WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD); try { writeStringBinary(identifier, out); diff --git a/programs/odbc-bridge/IdentifierQuoteHandler.h b/programs/odbc-bridge/IdentifierQuoteHandler.h index 7b78c5b4b93..c0e07795ea5 100644 --- a/programs/odbc-bridge/IdentifierQuoteHandler.h +++ b/programs/odbc-bridge/IdentifierQuoteHandler.h @@ -14,18 +14,12 @@ namespace DB class IdentifierQuoteHandler : public HTTPRequestHandler, WithContext { public: - IdentifierQuoteHandler(size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , log(getLogger("IdentifierQuoteHandler")) - , keep_alive_timeout(keep_alive_timeout_) - { - } + IdentifierQuoteHandler(ContextPtr context_) : WithContext(context_), log(getLogger("IdentifierQuoteHandler")) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: LoggerPtr log; - size_t keep_alive_timeout; }; } diff --git a/programs/odbc-bridge/MainHandler.cpp b/programs/odbc-bridge/MainHandler.cpp index e350afa2b10..b086397446e 100644 --- a/programs/odbc-bridge/MainHandler.cpp +++ b/programs/odbc-bridge/MainHandler.cpp @@ -131,7 +131,7 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse return; } - WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); + WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD); try { diff --git a/programs/odbc-bridge/MainHandler.h b/programs/odbc-bridge/MainHandler.h index ed0c6b2e28c..0fcad61d274 100644 --- a/programs/odbc-bridge/MainHandler.h +++ b/programs/odbc-bridge/MainHandler.h @@ -20,12 +20,10 @@ class ODBCHandler : public HTTPRequestHandler, WithContext { public: ODBCHandler( - size_t keep_alive_timeout_, ContextPtr context_, const String & mode_) : WithContext(context_) , log(getLogger("ODBCHandler")) - , keep_alive_timeout(keep_alive_timeout_) , mode(mode_) { } @@ -35,7 +33,6 @@ public: private: LoggerPtr log; - size_t keep_alive_timeout; String mode; static inline std::mutex mutex; diff --git a/programs/odbc-bridge/ODBCHandlerFactory.cpp b/programs/odbc-bridge/ODBCHandlerFactory.cpp index eebb0c24c7a..7f095666447 100644 --- a/programs/odbc-bridge/ODBCHandlerFactory.cpp +++ b/programs/odbc-bridge/ODBCHandlerFactory.cpp @@ -30,26 +30,26 @@ std::unique_ptr ODBCBridgeHandlerFactory::createRequestHandl if (uri.getPath() == "/columns_info") #if USE_ODBC - return std::make_unique(keep_alive_timeout, getContext()); + return std::make_unique(getContext()); #else return nullptr; #endif else if (uri.getPath() == "/identifier_quote") #if USE_ODBC - return std::make_unique(keep_alive_timeout, getContext()); + return std::make_unique(getContext()); #else return nullptr; #endif else if (uri.getPath() == "/schema_allowed") #if USE_ODBC - return std::make_unique(keep_alive_timeout, getContext()); + return std::make_unique(getContext()); #else return nullptr; #endif else if (uri.getPath() == "/write") - return std::make_unique(keep_alive_timeout, getContext(), "write"); + return std::make_unique(getContext(), "write"); else - return std::make_unique(keep_alive_timeout, getContext(), "read"); + return std::make_unique(getContext(), "read"); } return nullptr; } diff --git a/programs/odbc-bridge/SchemaAllowedHandler.cpp b/programs/odbc-bridge/SchemaAllowedHandler.cpp index c7025ca4311..5dc0cb3aa2b 100644 --- a/programs/odbc-bridge/SchemaAllowedHandler.cpp +++ b/programs/odbc-bridge/SchemaAllowedHandler.cpp @@ -86,7 +86,7 @@ void SchemaAllowedHandler::handleRequest(HTTPServerRequest & request, HTTPServer bool result = isSchemaAllowed(std::move(connection)); - WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); + WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD); try { writeBoolText(result, out); diff --git a/programs/odbc-bridge/SchemaAllowedHandler.h b/programs/odbc-bridge/SchemaAllowedHandler.h index 8dc725dbb33..e73c0a2cb26 100644 --- a/programs/odbc-bridge/SchemaAllowedHandler.h +++ b/programs/odbc-bridge/SchemaAllowedHandler.h @@ -17,18 +17,12 @@ class Context; class SchemaAllowedHandler : public HTTPRequestHandler, WithContext { public: - SchemaAllowedHandler(size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , log(getLogger("SchemaAllowedHandler")) - , keep_alive_timeout(keep_alive_timeout_) - { - } + SchemaAllowedHandler(ContextPtr context_) : WithContext(context_), log(getLogger("SchemaAllowedHandler")) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: LoggerPtr log; - size_t keep_alive_timeout; }; } From c33511dcb9314da6b64b11cf21d231c9d3896dad Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Mar 2024 20:24:53 +0000 Subject: [PATCH 023/644] remove more --- programs/odbc-bridge/ODBCBridge.cpp | 2 +- programs/odbc-bridge/ODBCHandlerFactory.cpp | 9 +++------ programs/odbc-bridge/ODBCHandlerFactory.h | 3 +-- programs/odbc-bridge/PingHandler.h | 4 ---- 4 files changed, 5 insertions(+), 13 deletions(-) diff --git a/programs/odbc-bridge/ODBCBridge.cpp b/programs/odbc-bridge/ODBCBridge.cpp index e91cc3158df..2cde5bbf9f5 100644 --- a/programs/odbc-bridge/ODBCBridge.cpp +++ b/programs/odbc-bridge/ODBCBridge.cpp @@ -25,7 +25,7 @@ std::string ODBCBridge::bridgeName() const ODBCBridge::HandlerFactoryPtr ODBCBridge::getHandlerFactoryPtr(ContextPtr context) const { - return std::make_shared("ODBCRequestHandlerFactory-factory", keep_alive_timeout, context); + return std::make_shared("ODBCRequestHandlerFactory-factory", context); } } diff --git a/programs/odbc-bridge/ODBCHandlerFactory.cpp b/programs/odbc-bridge/ODBCHandlerFactory.cpp index 7f095666447..b5d0be908f4 100644 --- a/programs/odbc-bridge/ODBCHandlerFactory.cpp +++ b/programs/odbc-bridge/ODBCHandlerFactory.cpp @@ -9,11 +9,8 @@ namespace DB { -ODBCBridgeHandlerFactory::ODBCBridgeHandlerFactory(const std::string & name_, size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , log(getLogger(name_)) - , name(name_) - , keep_alive_timeout(keep_alive_timeout_) +ODBCBridgeHandlerFactory::ODBCBridgeHandlerFactory(const std::string & name_, ContextPtr context_) + : WithContext(context_), log(getLogger(name_)), name(name_) { } @@ -23,7 +20,7 @@ std::unique_ptr ODBCBridgeHandlerFactory::createRequestHandl LOG_TRACE(log, "Request URI: {}", uri.toString()); if (uri.getPath() == "/ping" && request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) - return std::make_unique(keep_alive_timeout); + return std::make_unique(); if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) { diff --git a/programs/odbc-bridge/ODBCHandlerFactory.h b/programs/odbc-bridge/ODBCHandlerFactory.h index 4aaf1b55453..f4a2717dc9f 100644 --- a/programs/odbc-bridge/ODBCHandlerFactory.h +++ b/programs/odbc-bridge/ODBCHandlerFactory.h @@ -17,14 +17,13 @@ namespace DB class ODBCBridgeHandlerFactory : public HTTPRequestHandlerFactory, WithContext { public: - ODBCBridgeHandlerFactory(const std::string & name_, size_t keep_alive_timeout_, ContextPtr context_); + ODBCBridgeHandlerFactory(const std::string & name_, ContextPtr context_); std::unique_ptr createRequestHandler(const HTTPServerRequest & request) override; private: LoggerPtr log; std::string name; - size_t keep_alive_timeout; }; } diff --git a/programs/odbc-bridge/PingHandler.h b/programs/odbc-bridge/PingHandler.h index c5447107e0c..4c557bd3cf6 100644 --- a/programs/odbc-bridge/PingHandler.h +++ b/programs/odbc-bridge/PingHandler.h @@ -9,11 +9,7 @@ namespace DB class PingHandler : public HTTPRequestHandler { public: - explicit PingHandler(size_t keep_alive_timeout_) : keep_alive_timeout(keep_alive_timeout_) {} void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; - -private: - size_t keep_alive_timeout; }; } From 1115fa4bc74d840e8fc3230908310cb7311dd0d0 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 25 Mar 2024 15:13:13 +0000 Subject: [PATCH 024/644] fix tidy --- src/Server/PrometheusRequestHandler.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index 7f4d3c14f62..a1bd18b394a 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -15,7 +15,7 @@ private: PrometheusMetricsWriterPtr metrics_writer; public: - PrometheusRequestHandler(PrometheusMetricsWriterPtr metrics_writer_) : metrics_writer(std::move(metrics_writer_)) { } + explicit PrometheusRequestHandler(PrometheusMetricsWriterPtr metrics_writer_) : metrics_writer(std::move(metrics_writer_)) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; }; From 1cfbc548bb415e89e294a22da0eea59302269c37 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 25 Mar 2024 17:28:51 +0100 Subject: [PATCH 025/644] Fix copy-paste Co-authored-by: Michael Lex --- src/Core/ServerSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 7480d94e81d..4a22082cdda 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -113,7 +113,7 @@ namespace DB M(Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0) \ M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \ M(Seconds, keep_alive_timeout, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT, "The number of seconds that ClickHouse waits for incoming requests before closing the connection.", 0) \ - M(UInt64, max_keep_alive_requests, 10000, "The number of seconds that ClickHouse waits for incoming requests before closing the connection.", 0) \ + M(UInt64, max_keep_alive_requests, 10000, "The maximum number of requests handled via a single http keepalive connection before the server closes this connection.", 0) \ M(Seconds, replicated_fetches_http_connection_timeout, 0, "HTTP connection timeout for part fetch requests. Inherited from default profile `http_connection_timeout` if not set explicitly.", 0) \ M(Seconds, replicated_fetches_http_send_timeout, 0, "HTTP send timeout for part fetch requests. Inherited from default profile `http_send_timeout` if not set explicitly.", 0) \ M(Seconds, replicated_fetches_http_receive_timeout, 0, "HTTP receive timeout for fetch part requests. Inherited from default profile `http_receive_timeout` if not set explicitly.", 0) \ From 3c2915934f31d82176b65e1e998eca030671d872 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 26 Mar 2024 04:29:34 +0000 Subject: [PATCH 026/644] update fuzzers --- ..._function_state_deserialization_fuzzer.cpp | 23 +++++++++++++++++++ src/Client/ClientBase.cpp | 2 +- src/Core/fuzzers/names_and_types_fuzzer.cpp | 22 ++++++++++++++++++ .../data_type_deserialization_fuzzer.cpp | 22 ++++++++++++++++++ src/Formats/fuzzers/format_fuzzer.cpp | 20 ++++++++++++++++ .../fuzzers/codegen_fuzzer/CMakeLists.txt | 2 +- .../codegen_fuzzer/codegen_select_fuzzer.cpp | 2 +- src/Parsers/fuzzers/create_parser_fuzzer.cpp | 2 +- .../fuzzers/columns_description_fuzzer.cpp | 22 ++++++++++++++++++ 9 files changed, 113 insertions(+), 4 deletions(-) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index 425364efb9c..9d490432c60 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -12,10 +12,33 @@ #include +#include + #include #include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class IFunctionBase; +using FunctionBasePtr = std::shared_ptr; + +FunctionBasePtr createFunctionBaseCast( + ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); +} + +} + + extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 767a9b2b9f9..bdee2233b27 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2729,7 +2729,7 @@ void ClientBase::runLibFuzzer() for (auto & arg : fuzzer_args_holder) fuzzer_args.emplace_back(arg.data()); - int fuzzer_argc = fuzzer_args.size(); + int fuzzer_argc = static_cast(fuzzer_args.size()); char ** fuzzer_argv = fuzzer_args.data(); LLVMFuzzerRunDriver(&fuzzer_argc, &fuzzer_argv, [](const uint8_t * data, size_t size) diff --git a/src/Core/fuzzers/names_and_types_fuzzer.cpp b/src/Core/fuzzers/names_and_types_fuzzer.cpp index 6fdd8703014..bc8cb7af61f 100644 --- a/src/Core/fuzzers/names_and_types_fuzzer.cpp +++ b/src/Core/fuzzers/names_and_types_fuzzer.cpp @@ -1,7 +1,29 @@ +#include +#include #include #include +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class IFunctionBase; +using FunctionBasePtr = std::shared_ptr; + +FunctionBasePtr createFunctionBaseCast( + ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); +} + +} + + extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp index 0ae325871fb..f1b03147929 100644 --- a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp +++ b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp @@ -8,11 +8,33 @@ #include #include +#include + #include #include +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class IFunctionBase; +using FunctionBasePtr = std::shared_ptr; + +FunctionBasePtr createFunctionBaseCast( + ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); +} + +} + + extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp index 46661e4828c..4426301b6e7 100644 --- a/src/Formats/fuzzers/format_fuzzer.cpp +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -21,6 +21,26 @@ #include +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class IFunctionBase; +using FunctionBasePtr = std::shared_ptr; + +FunctionBasePtr createFunctionBaseCast( + ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); +} + +} + + extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt index 20fd951d390..74fdcff79f7 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -39,7 +39,7 @@ set(CMAKE_INCLUDE_CURRENT_DIR TRUE) clickhouse_add_executable(codegen_select_fuzzer ${FUZZER_SRCS}) -set_source_files_properties("${PROTO_SRCS}" "out.cpp" PROPERTIES COMPILE_FLAGS "-Wno-reserved-identifier") +set_source_files_properties("${PROTO_SRCS}" "out.cpp" PROPERTIES COMPILE_FLAGS "-Wno-reserved-identifier -Wno-extra-semi-stmt -Wno-used-but-marked-unused") # contrib/libprotobuf-mutator/src/libfuzzer/libfuzzer_macro.h:143:44: error: no newline at end of file [-Werror,-Wnewline-eof] target_compile_options (codegen_select_fuzzer PRIVATE -Wno-newline-eof) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp index 9310d7d59f7..55daa370651 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp +++ b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp @@ -27,7 +27,7 @@ DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) DB::ParserQueryWithOutput parser(input.data() + input.size()); try { - DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); + DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0, 0); DB::WriteBufferFromOStream out(std::cerr, 4096); DB::formatAST(*ast, out); diff --git a/src/Parsers/fuzzers/create_parser_fuzzer.cpp b/src/Parsers/fuzzers/create_parser_fuzzer.cpp index 854885ad33b..1d5c3e27232 100644 --- a/src/Parsers/fuzzers/create_parser_fuzzer.cpp +++ b/src/Parsers/fuzzers/create_parser_fuzzer.cpp @@ -14,7 +14,7 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) std::string input = std::string(reinterpret_cast(data), size); DB::ParserCreateQuery parser; - DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000); + DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000, 0); const UInt64 max_ast_depth = 1000; ast->checkDepth(max_ast_depth); diff --git a/src/Storages/fuzzers/columns_description_fuzzer.cpp b/src/Storages/fuzzers/columns_description_fuzzer.cpp index b703a1e7051..cb0c6168225 100644 --- a/src/Storages/fuzzers/columns_description_fuzzer.cpp +++ b/src/Storages/fuzzers/columns_description_fuzzer.cpp @@ -1,4 +1,26 @@ +#include #include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class IFunctionBase; +using FunctionBasePtr = std::shared_ptr; + +FunctionBasePtr createFunctionBaseCast( + ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); +} + +} extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) From cb40fd7d0c9096a8df8d5e7c2e9924f66d51e061 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 26 Mar 2024 15:27:01 +0000 Subject: [PATCH 027/644] minor fixes --- .../fuzzers/aggregate_function_state_deserialization_fuzzer.cpp | 2 +- src/Core/fuzzers/names_and_types_fuzzer.cpp | 2 +- src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp | 2 +- src/Formats/fuzzers/format_fuzzer.cpp | 2 +- src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp | 2 +- src/Parsers/fuzzers/create_parser_fuzzer.cpp | 2 +- src/Storages/fuzzers/columns_description_fuzzer.cpp | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index 9d490432c60..a956d9906bc 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -33,7 +33,7 @@ using FunctionBasePtr = std::shared_ptr; FunctionBasePtr createFunctionBaseCast( ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for aggregate_function_state_deserialization_fuzzer"); } } diff --git a/src/Core/fuzzers/names_and_types_fuzzer.cpp b/src/Core/fuzzers/names_and_types_fuzzer.cpp index bc8cb7af61f..74debedf2a3 100644 --- a/src/Core/fuzzers/names_and_types_fuzzer.cpp +++ b/src/Core/fuzzers/names_and_types_fuzzer.cpp @@ -18,7 +18,7 @@ using FunctionBasePtr = std::shared_ptr; FunctionBasePtr createFunctionBaseCast( ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for names_and_types_fuzzer"); } } diff --git a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp index f1b03147929..7d9a0513d18 100644 --- a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp +++ b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp @@ -29,7 +29,7 @@ using FunctionBasePtr = std::shared_ptr; FunctionBasePtr createFunctionBaseCast( ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for data_type_deserialization_fuzzer"); } } diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp index 4426301b6e7..2c1ec65e54d 100644 --- a/src/Formats/fuzzers/format_fuzzer.cpp +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -35,7 +35,7 @@ using FunctionBasePtr = std::shared_ptr; FunctionBasePtr createFunctionBaseCast( ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for format_fuzzer"); } } diff --git a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp index 55daa370651..6b25b581532 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp +++ b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp @@ -27,7 +27,7 @@ DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) DB::ParserQueryWithOutput parser(input.data() + input.size()); try { - DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0, 0); + DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0, DB::DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); DB::WriteBufferFromOStream out(std::cerr, 4096); DB::formatAST(*ast, out); diff --git a/src/Parsers/fuzzers/create_parser_fuzzer.cpp b/src/Parsers/fuzzers/create_parser_fuzzer.cpp index 1d5c3e27232..bab8db5671d 100644 --- a/src/Parsers/fuzzers/create_parser_fuzzer.cpp +++ b/src/Parsers/fuzzers/create_parser_fuzzer.cpp @@ -14,7 +14,7 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) std::string input = std::string(reinterpret_cast(data), size); DB::ParserCreateQuery parser; - DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000, 0); + DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000, DB::DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); const UInt64 max_ast_depth = 1000; ast->checkDepth(max_ast_depth); diff --git a/src/Storages/fuzzers/columns_description_fuzzer.cpp b/src/Storages/fuzzers/columns_description_fuzzer.cpp index cb0c6168225..ac285ea50f7 100644 --- a/src/Storages/fuzzers/columns_description_fuzzer.cpp +++ b/src/Storages/fuzzers/columns_description_fuzzer.cpp @@ -17,7 +17,7 @@ using FunctionBasePtr = std::shared_ptr; FunctionBasePtr createFunctionBaseCast( ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for columns_description_fuzzer"); } } From 64e6c6a2fcf2f7017ec3749ad05eed2daeeb4b42 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 26 Mar 2024 22:31:40 +0000 Subject: [PATCH 028/644] fix tidy --- programs/library-bridge/LibraryBridgeHandlers.h | 8 ++++---- programs/odbc-bridge/ColumnInfoHandler.h | 2 +- programs/odbc-bridge/IdentifierQuoteHandler.h | 2 +- programs/odbc-bridge/SchemaAllowedHandler.h | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/programs/library-bridge/LibraryBridgeHandlers.h b/programs/library-bridge/LibraryBridgeHandlers.h index 70e3c9c78da..582619e174e 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.h +++ b/programs/library-bridge/LibraryBridgeHandlers.h @@ -18,7 +18,7 @@ namespace DB class ExternalDictionaryLibraryBridgeRequestHandler : public HTTPRequestHandler, WithContext { public: - ExternalDictionaryLibraryBridgeRequestHandler(ContextPtr context_); + explicit ExternalDictionaryLibraryBridgeRequestHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; @@ -33,7 +33,7 @@ private: class ExternalDictionaryLibraryBridgeExistsHandler : public HTTPRequestHandler, WithContext { public: - ExternalDictionaryLibraryBridgeExistsHandler(ContextPtr context_); + explicit ExternalDictionaryLibraryBridgeExistsHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; @@ -61,7 +61,7 @@ private: class CatBoostLibraryBridgeRequestHandler : public HTTPRequestHandler, WithContext { public: - CatBoostLibraryBridgeRequestHandler(ContextPtr context_); + explicit CatBoostLibraryBridgeRequestHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; @@ -74,7 +74,7 @@ private: class CatBoostLibraryBridgeExistsHandler : public HTTPRequestHandler, WithContext { public: - CatBoostLibraryBridgeExistsHandler(ContextPtr context_); + explicit CatBoostLibraryBridgeExistsHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; diff --git a/programs/odbc-bridge/ColumnInfoHandler.h b/programs/odbc-bridge/ColumnInfoHandler.h index f16e09ec3f9..bbbf0da218b 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.h +++ b/programs/odbc-bridge/ColumnInfoHandler.h @@ -16,7 +16,7 @@ namespace DB class ODBCColumnsInfoHandler : public HTTPRequestHandler, WithContext { public: - ODBCColumnsInfoHandler(ContextPtr context_) : WithContext(context_), log(getLogger("ODBCColumnsInfoHandler")) { } + explicit ODBCColumnsInfoHandler(ContextPtr context_) : WithContext(context_), log(getLogger("ODBCColumnsInfoHandler")) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; diff --git a/programs/odbc-bridge/IdentifierQuoteHandler.h b/programs/odbc-bridge/IdentifierQuoteHandler.h index c0e07795ea5..a85b56a9f6a 100644 --- a/programs/odbc-bridge/IdentifierQuoteHandler.h +++ b/programs/odbc-bridge/IdentifierQuoteHandler.h @@ -14,7 +14,7 @@ namespace DB class IdentifierQuoteHandler : public HTTPRequestHandler, WithContext { public: - IdentifierQuoteHandler(ContextPtr context_) : WithContext(context_), log(getLogger("IdentifierQuoteHandler")) { } + explicit IdentifierQuoteHandler(ContextPtr context_) : WithContext(context_), log(getLogger("IdentifierQuoteHandler")) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; diff --git a/programs/odbc-bridge/SchemaAllowedHandler.h b/programs/odbc-bridge/SchemaAllowedHandler.h index e73c0a2cb26..59022151b53 100644 --- a/programs/odbc-bridge/SchemaAllowedHandler.h +++ b/programs/odbc-bridge/SchemaAllowedHandler.h @@ -17,7 +17,7 @@ class Context; class SchemaAllowedHandler : public HTTPRequestHandler, WithContext { public: - SchemaAllowedHandler(ContextPtr context_) : WithContext(context_), log(getLogger("SchemaAllowedHandler")) { } + explicit SchemaAllowedHandler(ContextPtr context_) : WithContext(context_), log(getLogger("SchemaAllowedHandler")) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; From 8357bc7b1b2d48e808b63cc0aa6fb7c7aa36e98b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 31 Mar 2024 23:33:35 +0000 Subject: [PATCH 029/644] fix build --- base/base/CMakeLists.txt | 2 +- cmake/sanitize.cmake | 2 +- programs/CMakeLists.txt | 2 +- src/CMakeLists.txt | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/base/base/CMakeLists.txt b/base/base/CMakeLists.txt index 27aa0bd6baf..7b1da9ab4ad 100644 --- a/base/base/CMakeLists.txt +++ b/base/base/CMakeLists.txt @@ -1,4 +1,4 @@ -add_compile_options($<$,$>:${COVERAGE_FLAGS}>) +add_compile_options("$<$,$>:${COVERAGE_FLAGS}>") if (USE_CLANG_TIDY) set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}") diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 9d53b2004b4..227d96357b5 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -64,7 +64,7 @@ option(WITH_COVERAGE "Instrumentation for code coverage with default implementat if (WITH_COVERAGE) message (STATUS "Enabled instrumentation for code coverage") - set(COVERAGE_FLAGS "-fprofile-instr-generate -fcoverage-mapping") + set(COVERAGE_FLAGS -fprofile-instr-generate -fcoverage-mapping) endif() option (SANITIZE_COVERAGE "Instrumentation for code coverage with custom callbacks" OFF) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 0d91de2dad8..aa7781498c8 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -1,4 +1,4 @@ -add_compile_options($<$,$>:${COVERAGE_FLAGS}>) +add_compile_options("$<$,$>:${COVERAGE_FLAGS}>") if (USE_CLANG_TIDY) set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}") diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 73aa409e995..bd603c9f15e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -1,4 +1,4 @@ -add_compile_options($<$,$>:${COVERAGE_FLAGS}>) +add_compile_options("$<$,$>:${COVERAGE_FLAGS}>") if (USE_INCLUDE_WHAT_YOU_USE) set (CMAKE_CXX_INCLUDE_WHAT_YOU_USE ${IWYU_PATH}) From 99e25d762c2db3c544dd5590726fc039b1828d16 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 1 Apr 2024 12:28:51 +0000 Subject: [PATCH 030/644] remove WITH_COVERAGE for fuzzers build --- docker/packager/packager | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/packager/packager b/docker/packager/packager index 23fc26bc1a4..355149df38c 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -276,7 +276,6 @@ def parse_env_variables( elif package_type == "fuzzers": cmake_flags.append("-DENABLE_FUZZING=1") cmake_flags.append("-DENABLE_PROTOBUF=1") - cmake_flags.append("-DWITH_COVERAGE=1") # Reduce linking and building time by avoid *install/all dependencies cmake_flags.append("-DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON") From 8d667ad5a34d1ba3d9008a5a6308598483281b35 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 1 Apr 2024 22:55:51 +0000 Subject: [PATCH 031/644] fix build.sh --- docker/packager/binary-builder/build.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/packager/binary-builder/build.sh b/docker/packager/binary-builder/build.sh index 032aceb0af3..cbd14b1eac2 100755 --- a/docker/packager/binary-builder/build.sh +++ b/docker/packager/binary-builder/build.sh @@ -108,7 +108,8 @@ if [ -n "$MAKE_DEB" ]; then bash -x /build/packages/build fi -mv ./programs/clickhouse* /output || mv ./programs/*_fuzzer /output +mv ./programs/clickhouse* /output ||: +mv ./programs/*_fuzzer /output ||: [ -x ./programs/self-extracting/clickhouse ] && mv ./programs/self-extracting/clickhouse /output [ -x ./programs/self-extracting/clickhouse-stripped ] && mv ./programs/self-extracting/clickhouse-stripped /output mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds From db3d923d4cae57254cadcef7f6997f3912d46515 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 3 Apr 2024 20:25:29 +0000 Subject: [PATCH 032/644] return WITH_COVERAGE, fix build --- cmake/sanitize.cmake | 3 ++- docker/packager/packager | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 227d96357b5..9f4fa7081c6 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -64,7 +64,8 @@ option(WITH_COVERAGE "Instrumentation for code coverage with default implementat if (WITH_COVERAGE) message (STATUS "Enabled instrumentation for code coverage") - set(COVERAGE_FLAGS -fprofile-instr-generate -fcoverage-mapping) + set (COVERAGE_FLAGS -fprofile-instr-generate -fcoverage-mapping) + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fprofile-instr-generate -fcoverage-mapping") endif() option (SANITIZE_COVERAGE "Instrumentation for code coverage with custom callbacks" OFF) diff --git a/docker/packager/packager b/docker/packager/packager index 355149df38c..23fc26bc1a4 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -276,6 +276,7 @@ def parse_env_variables( elif package_type == "fuzzers": cmake_flags.append("-DENABLE_FUZZING=1") cmake_flags.append("-DENABLE_PROTOBUF=1") + cmake_flags.append("-DWITH_COVERAGE=1") # Reduce linking and building time by avoid *install/all dependencies cmake_flags.append("-DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON") From 56b03ee22a380722e94b83f0d2d3736cb16c2847 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 8 Apr 2024 15:28:20 +0000 Subject: [PATCH 033/644] Handle macros properly in create query --- src/Storages/Kafka/KafkaSettings.h | 2 +- src/Storages/Kafka/StorageKafkaCommon.cpp | 75 +++++++++++++++++++---- 2 files changed, 65 insertions(+), 12 deletions(-) diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 705fc9f4826..c9ee42b54aa 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -38,8 +38,8 @@ const auto KAFKA_CONSUMERS_POOL_TTL_MS_MAX = 600'000; M(StreamingHandleErrorMode, kafka_handle_error_mode, StreamingHandleErrorMode::DEFAULT, "How to handle errors for Kafka engine. Possible values: default (throw an exception after rabbitmq_skip_broken_messages broken messages), stream (save broken messages and errors in virtual columns _raw_message, _error).", 0) \ M(Bool, kafka_commit_on_select, false, "Commit messages when select query is made", 0) \ M(UInt64, kafka_max_rows_per_message, 1, "The maximum number of rows produced in one kafka message for row-based formats.", 0) \ - /* TODO(antaljanosbenjamin): Probably this shouldn't be here, but only read as an argument */ \ M(String, kafka_keeper_path, "", "TODO(antaljanosbenjamin)", 0) \ + M(String, kafka_replica_name, "", "TODO(antaljanosbenjamin)", 0) \ #define OBSOLETE_KAFKA_SETTINGS(M, ALIAS) \ MAKE_OBSOLETE(M, Char, kafka_row_delimiter, '\0') \ diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp index b0f23c38163..1319c871f4d 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.cpp +++ b/src/Storages/Kafka/StorageKafkaCommon.cpp @@ -1,6 +1,7 @@ #include +#include #include #include #include @@ -303,7 +304,6 @@ void registerStorageKafka(StorageFactory & factory) CHECK_KAFKA_STORAGE_ARGUMENT(15, kafka_handle_error_mode, 0) CHECK_KAFKA_STORAGE_ARGUMENT(16, kafka_commit_on_select, 0) CHECK_KAFKA_STORAGE_ARGUMENT(17, kafka_max_rows_per_message, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(18, kafka_keeper_path, 0) } #undef CHECK_KAFKA_STORAGE_ARGUMENT @@ -357,20 +357,73 @@ void registerStorageKafka(StorageFactory & factory) "See https://clickhouse.com/docs/en/engines/table-engines/integrations/kafka/#configuration"); } - if (!kafka_settings->kafka_keeper_path.value.empty()) - { - if (!args.getLocalContext()->getSettingsRef().allow_experimental_kafka_store_offsets_in_keeper) + const auto has_keeper_path = kafka_settings->kafka_keeper_path.changed && !kafka_settings->kafka_keeper_path.value.empty(); + const auto has_replica_name = kafka_settings->kafka_replica_name.changed && !kafka_settings->kafka_replica_name.value.empty(); - throw Exception( - ErrorCodes::SUPPORT_IS_DISABLED, - "Storing the Kafka offsets in Keeper is experimental. Set `allow_experimental_kafka_store_offsets_in_keeper` setting " - "to enable it"); - - return std::make_shared( + if (!has_keeper_path && !has_replica_name) + return std::make_shared( args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); + + if (!args.getLocalContext()->getSettingsRef().allow_experimental_kafka_store_offsets_in_keeper) + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "Storing the Kafka offsets in Keeper is experimental. Set `allow_experimental_kafka_store_offsets_in_keeper` setting " + "to enable it"); + + if (!has_keeper_path || !has_replica_name) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Either specify both zookeeper path and replica name or none of them"); + + const auto is_on_cluster = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; + const auto is_replicated_database = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY + && DatabaseCatalog::instance().getDatabase(args.table_id.database_name)->getEngineName() == "Replicated"; + + // TODO(antaljanosbenjamin): attach query? + // TODO(antaljanosbenjamin): why not on single atomic database? + const auto allow_uuid_macro = is_on_cluster || is_replicated_database || args.query.attach; + + auto context = args.getContext(); + /// Unfold {database} and {table} macro on table creation, so table can be renamed. + if (!args.attach) + { + Macros::MacroExpansionInfo info; + /// NOTE: it's not recursive + info.expand_special_macros_only = true; + info.table_id = args.table_id; + // TODO(antaljanosbenjamin): why to skip UUID here? + info.table_id.uuid = UUIDHelpers::Nil; + kafka_settings->kafka_keeper_path.value = context->getMacros()->expand(kafka_settings->kafka_keeper_path.value, info); + + info.level = 0; + kafka_settings->kafka_replica_name.value = context->getMacros()->expand(kafka_settings->kafka_replica_name.value, info); } - return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); + + auto * settings_query = args.storage_def->settings; + chassert(settings_query != nullptr && "Unexpected settings query in StorageKafka"); + + settings_query->changes.setSetting("kafka_keeper_path", kafka_settings->kafka_keeper_path.value); + settings_query->changes.setSetting("kafka_replica_name", kafka_settings->kafka_replica_name.value); + + /// Expand other macros (such as {shard} and {replica}). We do not expand them on previous step + /// to make possible copying metadata files between replicas. + Macros::MacroExpansionInfo info; + info.table_id = args.table_id; + if (is_replicated_database) + { + auto database = DatabaseCatalog::instance().getDatabase(args.table_id.database_name); + info.shard = getReplicatedDatabaseShardName(database); + info.replica = getReplicatedDatabaseReplicaName(database); + } + if (!allow_uuid_macro) + info.table_id.uuid = UUIDHelpers::Nil; + kafka_settings->kafka_keeper_path.value = context->getMacros()->expand(kafka_settings->kafka_keeper_path.value, info); + + info.level = 0; + info.table_id.uuid = UUIDHelpers::Nil; + kafka_settings->kafka_replica_name.value = context->getMacros()->expand(kafka_settings->kafka_replica_name.value, info); + + return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); }; factory.registerStorage( From 8b2bd3cfd7654fee98df6f024bcf7e4b6b4f2b49 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 26 Mar 2024 20:48:49 +0000 Subject: [PATCH 034/644] impl --- src/Common/AsynchronousMetrics.cpp | 187 +++++++++++++++++++++-------- src/Common/AsynchronousMetrics.h | 6 + 2 files changed, 142 insertions(+), 51 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index ab54b180fbf..cf9e8d21bd8 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -1,18 +1,19 @@ -#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 "config.h" @@ -78,6 +79,7 @@ AsynchronousMetrics::AsynchronousMetrics( openFileIfExists("/sys/fs/cgroup/memory.current", cgroupmem_usage_in_bytes); } openFileIfExists("/sys/fs/cgroup/cpu.max", cgroupcpu_max); + openFileIfExists("/sys/fs/cgroup/cpu.stat", cgroupcpu_stat); /// CGroups v1 if (!cgroupmem_limit_in_bytes) @@ -90,6 +92,8 @@ AsynchronousMetrics::AsynchronousMetrics( openFileIfExists("/sys/fs/cgroup/cpu/cpu.cfs_period_us", cgroupcpu_cfs_period); openFileIfExists("/sys/fs/cgroup/cpu/cpu.cfs_quota_us", cgroupcpu_cfs_quota); } + if (!cgroupcpu_stat) + openFileIfExists("/sys/fs/cgroup/cpuacct/cpuacct.stat", cgroupcpuacct_stat); openFileIfExists("/proc/sys/vm/max_map_count", vm_max_map_count); openFileIfExists("/proc/self/maps", vm_maps); @@ -561,6 +565,82 @@ AsynchronousMetrics::NetworkInterfaceStatValues::operator-(const AsynchronousMet #endif +void AsynchronousMetrics::applyCPUMetricsUpdate( + AsynchronousMetricValues & new_values, const std::string & cpu_suffix, const ProcStatValuesCPU & delta_values, double multiplier) +{ + new_values["OSUserTime" + cpu_suffix] + = {delta_values.user * multiplier, + "The ratio of time the CPU core was running userspace code. This is a system-wide metric, it includes all the processes on the " + "host machine, not just clickhouse-server." + " This includes also the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline " + "stalls, branch mispredictions, running another SMT core)." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSNiceTime" + cpu_suffix] + = {delta_values.nice * multiplier, + "The ratio of time the CPU core was running userspace code with higher priority. This is a system-wide metric, it includes all " + "the processes on the host machine, not just clickhouse-server." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSSystemTime" + cpu_suffix] + = {delta_values.system * multiplier, + "The ratio of time the CPU core was running OS kernel (system) code. This is a system-wide metric, it includes all the " + "processes on the host machine, not just clickhouse-server." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSIdleTime" + cpu_suffix] + = {delta_values.idle * multiplier, + "The ratio of time the CPU core was idle (not even ready to run a process waiting for IO) from the OS kernel standpoint. This " + "is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " This does not include the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline " + "stalls, branch mispredictions, running another SMT core)." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSIOWaitTime" + cpu_suffix] + = {delta_values.iowait * multiplier, + "The ratio of time the CPU core was not running the code but when the OS kernel did not run any other process on this CPU as " + "the processes were waiting for IO. This is a system-wide metric, it includes all the processes on the host machine, not just " + "clickhouse-server." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSIrqTime" + cpu_suffix] + = {delta_values.irq * multiplier, + "The ratio of time spent for running hardware interrupt requests on the CPU. This is a system-wide metric, it includes all the " + "processes on the host machine, not just clickhouse-server." + " A high number of this metric may indicate hardware misconfiguration or a very high network load." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSSoftIrqTime" + cpu_suffix] + = {delta_values.softirq * multiplier, + "The ratio of time spent for running software interrupt requests on the CPU. This is a system-wide metric, it includes all the " + "processes on the host machine, not just clickhouse-server." + " A high number of this metric may indicate inefficient software running on the system." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSStealTime" + cpu_suffix] + = {delta_values.steal * multiplier, + "The ratio of time spent in other operating systems by the CPU when running in a virtualized environment. This is a system-wide " + "metric, it includes all the processes on the host machine, not just clickhouse-server." + " Not every virtualized environments present this metric, and most of them don't." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSGuestTime" + cpu_suffix] + = {delta_values.guest * multiplier, + "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel (See `man " + "procfs`). This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " This metric is irrelevant for ClickHouse, but still exists for completeness." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSGuestNiceTime" + cpu_suffix] + = {delta_values.guest_nice * multiplier, + "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel, when a guest " + "was set to a higher priority (See `man procfs`). This is a system-wide metric, it includes all the processes on the host " + "machine, not just clickhouse-server." + " This metric is irrelevant for ClickHouse, but still exists for completeness." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; +} + void AsynchronousMetrics::update(TimePoint update_time, bool force_update) { Stopwatch watch; @@ -821,16 +901,57 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) new_values["CGroupMaxCPU"] = { max_cpu_cgroups, "The maximum number of CPU cores according to CGroups."}; } - if (proc_stat) + int64_t hz = sysconf(_SC_CLK_TCK); + if (-1 == hz) + throw ErrnoException(ErrorCodes::CANNOT_SYSCONF, "Cannot call 'sysconf' to obtain system HZ"); + + if (cgroupcpu_stat || cgroupcpuacct_stat) + { + ReadBufferFromFilePRead & in = cgroupcpu_stat ? *cgroupcpu_stat : *cgroupcpuacct_stat; + ProcStatValuesCPU current_values{}; + + /// We re-read the file from the beginning each time + in.rewind(); + + while (!in.eof()) + { + String name; + readStringUntilWhitespace(name, in); + skipWhitespaceIfAny(in); + + /// `user_usec` for cgroup v2 and `user` for cgroup v1 + if (name.starts_with("user")) + { + readText(current_values.user, in); + skipToNextLineOrEOF(in); + } + /// `system_usec` for cgroup v2 and `system` for cgroup v1 + else if (name.starts_with("system")) + { + readText(current_values.system, in); + skipToNextLineOrEOF(in); + } + else + skipToNextLineOrEOF(in); + } + + if (!first_run) + { + const ProcStatValuesCPU delta_values = current_values - proc_stat_values_all_cpus; + const auto cgroup_specific_divisor = cgroupcpu_stat ? 1e6 : hz; + const double multiplier = 1.0 / cgroup_specific_divisor + / (std::chrono::duration_cast(time_since_previous_update).count() / 1e9); + applyCPUMetricsUpdate(new_values, /*cpu_suffix=*/"", delta_values, multiplier); + } + + proc_stat_values_all_cpus = current_values; + } + else if (proc_stat) { try { proc_stat->rewind(); - int64_t hz = sysconf(_SC_CLK_TCK); - if (-1 == hz) - throw ErrnoException(ErrorCodes::CANNOT_SYSCONF, "Cannot call 'sysconf' to obtain system HZ"); - double multiplier = 1.0 / hz / (std::chrono::duration_cast(time_since_previous_update).count() / 1e9); size_t num_cpus = 0; @@ -876,43 +997,7 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) else delta_values_all_cpus = delta_values; - new_values["OSUserTime" + cpu_suffix] = { delta_values.user * multiplier, - "The ratio of time the CPU core was running userspace code. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " This includes also the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline stalls, branch mispredictions, running another SMT core)." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSNiceTime" + cpu_suffix] = { delta_values.nice * multiplier, - "The ratio of time the CPU core was running userspace code with higher priority. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSSystemTime" + cpu_suffix] = { delta_values.system * multiplier, - "The ratio of time the CPU core was running OS kernel (system) code. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSIdleTime" + cpu_suffix] = { delta_values.idle * multiplier, - "The ratio of time the CPU core was idle (not even ready to run a process waiting for IO) from the OS kernel standpoint. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " This does not include the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline stalls, branch mispredictions, running another SMT core)." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSIOWaitTime" + cpu_suffix] = { delta_values.iowait * multiplier, - "The ratio of time the CPU core was not running the code but when the OS kernel did not run any other process on this CPU as the processes were waiting for IO. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSIrqTime" + cpu_suffix] = { delta_values.irq * multiplier, - "The ratio of time spent for running hardware interrupt requests on the CPU. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " A high number of this metric may indicate hardware misconfiguration or a very high network load." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSSoftIrqTime" + cpu_suffix] = { delta_values.softirq * multiplier, - "The ratio of time spent for running software interrupt requests on the CPU. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " A high number of this metric may indicate inefficient software running on the system." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSStealTime" + cpu_suffix] = { delta_values.steal * multiplier, - "The ratio of time spent in other operating systems by the CPU when running in a virtualized environment. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " Not every virtualized environments present this metric, and most of them don't." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSGuestTime" + cpu_suffix] = { delta_values.guest * multiplier, - "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel (See `man procfs`). This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " This metric is irrelevant for ClickHouse, but still exists for completeness." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSGuestNiceTime" + cpu_suffix] = { delta_values.guest_nice * multiplier, - "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel, when a guest was set to a higher priority (See `man procfs`). This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " This metric is irrelevant for ClickHouse, but still exists for completeness." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + applyCPUMetricsUpdate(new_values, cpu_suffix, delta_values, multiplier); } prev_values = current_values; diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index 4b3d28e80c5..caebcd4cdef 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -122,6 +122,8 @@ private: std::optional cgroupcpu_cfs_period TSA_GUARDED_BY(data_mutex); std::optional cgroupcpu_cfs_quota TSA_GUARDED_BY(data_mutex); std::optional cgroupcpu_max TSA_GUARDED_BY(data_mutex); + std::optional cgroupcpu_stat TSA_GUARDED_BY(data_mutex); + std::optional cgroupcpuacct_stat TSA_GUARDED_BY(data_mutex); std::optional vm_max_map_count TSA_GUARDED_BY(data_mutex); std::optional vm_maps TSA_GUARDED_BY(data_mutex); @@ -217,6 +219,10 @@ private: void openBlockDevices(); void openSensorsChips(); void openEDAC(); + + void applyCPUMetricsUpdate( + AsynchronousMetricValues & new_values, const std::string & cpu_suffix, const ProcStatValuesCPU & delta_values, double multiplier); + #endif void run(); From 85e8a5678783521442a6e61bcd00ba6167302b6a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 27 Mar 2024 17:02:52 +0000 Subject: [PATCH 035/644] normalized metrics --- src/Common/AsynchronousMetrics.cpp | 170 ++++++++++++++++++----------- src/Common/AsynchronousMetrics.h | 6 + 2 files changed, 115 insertions(+), 61 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index cf9e8d21bd8..59595e701c1 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -641,6 +641,73 @@ void AsynchronousMetrics::applyCPUMetricsUpdate( "them [0..num cores]."}; } +void AsynchronousMetrics::applyNormalizedCPUMetricsUpdate( + AsynchronousMetricValues & new_values, double num_cpus_to_normalize, const ProcStatValuesCPU & delta_values_all_cpus, double multiplier) +{ + chassert(num_cpus_to_normalize); + + new_values["OSUserTimeNormalized"] + = {delta_values_all_cpus.user * multiplier / num_cpus_to_normalize, + "The value is similar to `OSUserTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless " + "of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSNiceTimeNormalized"] + = {delta_values_all_cpus.nice * multiplier / num_cpus_to_normalize, + "The value is similar to `OSNiceTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless " + "of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSSystemTimeNormalized"] + = {delta_values_all_cpus.system * multiplier / num_cpus_to_normalize, + "The value is similar to `OSSystemTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless " + "of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSIdleTimeNormalized"] + = {delta_values_all_cpus.idle * multiplier / num_cpus_to_normalize, + "The value is similar to `OSIdleTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless " + "of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSIOWaitTimeNormalized"] + = {delta_values_all_cpus.iowait * multiplier / num_cpus_to_normalize, + "The value is similar to `OSIOWaitTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless " + "of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSIrqTimeNormalized"] + = {delta_values_all_cpus.irq * multiplier / num_cpus_to_normalize, + "The value is similar to `OSIrqTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of " + "the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSSoftIrqTimeNormalized"] + = {delta_values_all_cpus.softirq * multiplier / num_cpus_to_normalize, + "The value is similar to `OSSoftIrqTime` but divided to the number of CPU cores to be measured in the [0..1] interval " + "regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSStealTimeNormalized"] + = {delta_values_all_cpus.steal * multiplier / num_cpus_to_normalize, + "The value is similar to `OSStealTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless " + "of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSGuestTimeNormalized"] + = {delta_values_all_cpus.guest * multiplier / num_cpus_to_normalize, + "The value is similar to `OSGuestTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless " + "of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSGuestNiceTimeNormalized"] + = {delta_values_all_cpus.guest_nice * multiplier / num_cpus_to_normalize, + "The value is similar to `OSGuestNiceTime` but divided to the number of CPU cores to be measured in the [0..1] interval " + "regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; +} + void AsynchronousMetrics::update(TimePoint update_time, bool force_update) { Stopwatch watch; @@ -907,44 +974,56 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) if (cgroupcpu_stat || cgroupcpuacct_stat) { - ReadBufferFromFilePRead & in = cgroupcpu_stat ? *cgroupcpu_stat : *cgroupcpuacct_stat; - ProcStatValuesCPU current_values{}; - - /// We re-read the file from the beginning each time - in.rewind(); - - while (!in.eof()) + try { - String name; - readStringUntilWhitespace(name, in); - skipWhitespaceIfAny(in); + ReadBufferFromFilePRead & in = cgroupcpu_stat ? *cgroupcpu_stat : *cgroupcpuacct_stat; + ProcStatValuesCPU current_values{}; - /// `user_usec` for cgroup v2 and `user` for cgroup v1 - if (name.starts_with("user")) + /// We re-read the file from the beginning each time + in.rewind(); + + while (!in.eof()) { - readText(current_values.user, in); - skipToNextLineOrEOF(in); + String name; + readStringUntilWhitespace(name, in); + skipWhitespaceIfAny(in); + + /// `user_usec` for cgroup v2 and `user` for cgroup v1 + if (name.starts_with("user")) + { + readText(current_values.user, in); + skipToNextLineOrEOF(in); + } + /// `system_usec` for cgroup v2 and `system` for cgroup v1 + else if (name.starts_with("system")) + { + readText(current_values.system, in); + skipToNextLineOrEOF(in); + } + else + skipToNextLineOrEOF(in); } - /// `system_usec` for cgroup v2 and `system` for cgroup v1 - else if (name.starts_with("system")) + + if (!first_run) { - readText(current_values.system, in); - skipToNextLineOrEOF(in); + const ProcStatValuesCPU delta_values = current_values - proc_stat_values_all_cpus; + const auto cgroup_specific_divisor = cgroupcpu_stat ? 1e6 : hz; + const double multiplier = 1.0 / cgroup_specific_divisor + / (std::chrono::duration_cast(time_since_previous_update).count() / 1e9); + applyCPUMetricsUpdate(new_values, /*cpu_suffix=*/"", delta_values, multiplier); + if (max_cpu_cgroups > 0) + applyNormalizedCPUMetricsUpdate(new_values, max_cpu_cgroups, delta_values, multiplier); } - else - skipToNextLineOrEOF(in); + + proc_stat_values_all_cpus = current_values; } - - if (!first_run) + catch (...) { - const ProcStatValuesCPU delta_values = current_values - proc_stat_values_all_cpus; - const auto cgroup_specific_divisor = cgroupcpu_stat ? 1e6 : hz; - const double multiplier = 1.0 / cgroup_specific_divisor - / (std::chrono::duration_cast(time_since_previous_update).count() / 1e9); - applyCPUMetricsUpdate(new_values, /*cpu_suffix=*/"", delta_values, multiplier); + tryLogCurrentException(__PRETTY_FUNCTION__); + openFileIfExists("/sys/fs/cgroup/cpu.stat", cgroupcpu_stat); + if (!cgroupcpu_stat) + openFileIfExists("/sys/fs/cgroup/cpuacct/cpuacct.stat", cgroupcpuacct_stat); } - - proc_stat_values_all_cpus = current_values; } else if (proc_stat) { @@ -1053,38 +1132,7 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) Float64 num_cpus_to_normalize = max_cpu_cgroups > 0 ? max_cpu_cgroups : num_cpus; if (num_cpus_to_normalize > 0) - { - new_values["OSUserTimeNormalized"] = { delta_values_all_cpus.user * multiplier / num_cpus_to_normalize, - "The value is similar to `OSUserTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSNiceTimeNormalized"] = { delta_values_all_cpus.nice * multiplier / num_cpus_to_normalize, - "The value is similar to `OSNiceTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSSystemTimeNormalized"] = { delta_values_all_cpus.system * multiplier / num_cpus_to_normalize, - "The value is similar to `OSSystemTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSIdleTimeNormalized"] = { delta_values_all_cpus.idle * multiplier / num_cpus_to_normalize, - "The value is similar to `OSIdleTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSIOWaitTimeNormalized"] = { delta_values_all_cpus.iowait * multiplier / num_cpus_to_normalize, - "The value is similar to `OSIOWaitTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSIrqTimeNormalized"] = { delta_values_all_cpus.irq * multiplier / num_cpus_to_normalize, - "The value is similar to `OSIrqTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSSoftIrqTimeNormalized"] = { delta_values_all_cpus.softirq * multiplier / num_cpus_to_normalize, - "The value is similar to `OSSoftIrqTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSStealTimeNormalized"] = { delta_values_all_cpus.steal * multiplier / num_cpus_to_normalize, - "The value is similar to `OSStealTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSGuestTimeNormalized"] = { delta_values_all_cpus.guest * multiplier / num_cpus_to_normalize, - "The value is similar to `OSGuestTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSGuestNiceTimeNormalized"] = { delta_values_all_cpus.guest_nice * multiplier / num_cpus_to_normalize, - "The value is similar to `OSGuestNiceTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - } + applyNormalizedCPUMetricsUpdate(new_values, num_cpus_to_normalize, delta_values_all_cpus, multiplier); } proc_stat_values_other = current_other_values; diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index caebcd4cdef..2b58fd78044 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -223,6 +223,12 @@ private: void applyCPUMetricsUpdate( AsynchronousMetricValues & new_values, const std::string & cpu_suffix, const ProcStatValuesCPU & delta_values, double multiplier); + void applyNormalizedCPUMetricsUpdate( + AsynchronousMetricValues & new_values, + double num_cpus_to_normalize, + const ProcStatValuesCPU & delta_values_all_cpus, + double multiplier); + #endif void run(); From 4aaae7fd4d3340131515be83764e56b5f5c17c13 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 27 Mar 2024 19:49:00 +0000 Subject: [PATCH 036/644] add test --- .../test_async_metrics_in_cgroup/test.py | 77 +++++++++++++++++++ 1 file changed, 77 insertions(+) create mode 100644 tests/integration/test_async_metrics_in_cgroup/test.py diff --git a/tests/integration/test_async_metrics_in_cgroup/test.py b/tests/integration/test_async_metrics_in_cgroup/test.py new file mode 100644 index 00000000000..1bba42cb980 --- /dev/null +++ b/tests/integration/test_async_metrics_in_cgroup/test.py @@ -0,0 +1,77 @@ +import pytest +import subprocess +import time + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node") + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_check_client_logs_level(start_cluster): + # check that our metrics sources actually exist + assert ( + subprocess.Popen("test -f /sys/fs/cgroup/cpu.stat".split(" ")).wait() == 0 + or subprocess.Popen( + "test -f /sys/fs/cgroup/cpuacct/cpuacct.stat".split(" ") + ).wait() + == 0 + ) + + # first let's spawn some cpu-intensive process outside of the container and check that it doesn't accounted by ClickHouse server + proc = subprocess.Popen( + "openssl speed -multi 8".split(" "), + stdout=subprocess.DEVNULL, + stderr=subprocess.DEVNULL, + ) + + time.sleep(5) + + metric = node.query( + """ + SYSTEM FLUSH LOGS; + + SELECT max(value) + FROM ( + SELECT toStartOfInterval(event_time, toIntervalSecond(1)) AS t, avg(value) AS value + FROM system.asynchronous_metric_log + WHERE event_time >= now() - 60 AND metric = 'OSUserTime' + GROUP BY t + ) + """ + ).strip("\n") + + assert float(metric) <= 2 + + proc.kill() + + # then let's test that we will account cpu time spent by the server itself + node.query( + "SELECT cityHash64(*) FROM system.numbers_mt FORMAT Null SETTINGS max_execution_time=5, max_threads=8", + ignore_error=True, + ) + + metric = node.query( + """ + SYSTEM FLUSH LOGS; + + SELECT max(value) + FROM ( + SELECT toStartOfInterval(event_time, toIntervalSecond(1)) AS t, avg(value) AS value + FROM system.asynchronous_metric_log + WHERE event_time >= now() - 60 AND metric = 'OSUserTime' + GROUP BY t + ) + """ + ).strip("\n") + + assert 4 <= float(metric) <= 12 From 75011d6f21e4948bf86fd52e2330fe2f2d8fa922 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 27 Mar 2024 20:22:15 +0000 Subject: [PATCH 037/644] fix style --- tests/integration/test_async_metrics_in_cgroup/__init__.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/integration/test_async_metrics_in_cgroup/__init__.py diff --git a/tests/integration/test_async_metrics_in_cgroup/__init__.py b/tests/integration/test_async_metrics_in_cgroup/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From d84a01cabfbb97a8b875620292f843c1247e6382 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 28 Mar 2024 20:46:02 +0000 Subject: [PATCH 038/644] better --- src/Common/AsynchronousMetrics.cpp | 20 +++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 59595e701c1..0943232e776 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -565,6 +565,7 @@ AsynchronousMetrics::NetworkInterfaceStatValues::operator-(const AsynchronousMet #endif +#if defined(OS_LINUX) void AsynchronousMetrics::applyCPUMetricsUpdate( AsynchronousMetricValues & new_values, const std::string & cpu_suffix, const ProcStatValuesCPU & delta_values, double multiplier) { @@ -707,6 +708,7 @@ void AsynchronousMetrics::applyNormalizedCPUMetricsUpdate( " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " "non-uniform, and still get the average resource utilization metric."}; } +#endif void AsynchronousMetrics::update(TimePoint update_time, bool force_update) { @@ -968,10 +970,6 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) new_values["CGroupMaxCPU"] = { max_cpu_cgroups, "The maximum number of CPU cores according to CGroups."}; } - int64_t hz = sysconf(_SC_CLK_TCK); - if (-1 == hz) - throw ErrnoException(ErrorCodes::CANNOT_SYSCONF, "Cannot call 'sysconf' to obtain system HZ"); - if (cgroupcpu_stat || cgroupcpuacct_stat) { try @@ -1006,10 +1004,14 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) if (!first_run) { - const ProcStatValuesCPU delta_values = current_values - proc_stat_values_all_cpus; - const auto cgroup_specific_divisor = cgroupcpu_stat ? 1e6 : hz; - const double multiplier = 1.0 / cgroup_specific_divisor + int64_t hz = sysconf(_SC_CLK_TCK); + if (-1 == hz) + throw ErrnoException(ErrorCodes::CANNOT_SYSCONF, "Cannot call 'sysconf' to obtain system HZ"); + const auto cgroup_version_specific_divisor = cgroupcpu_stat ? 1e6 : hz; + const double multiplier = 1.0 / cgroup_version_specific_divisor / (std::chrono::duration_cast(time_since_previous_update).count() / 1e9); + + const ProcStatValuesCPU delta_values = current_values - proc_stat_values_all_cpus; applyCPUMetricsUpdate(new_values, /*cpu_suffix=*/"", delta_values, multiplier); if (max_cpu_cgroups > 0) applyNormalizedCPUMetricsUpdate(new_values, max_cpu_cgroups, delta_values, multiplier); @@ -1031,6 +1033,10 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) { proc_stat->rewind(); + int64_t hz = sysconf(_SC_CLK_TCK); + if (-1 == hz) + throw ErrnoException(ErrorCodes::CANNOT_SYSCONF, "Cannot call 'sysconf' to obtain system HZ"); + double multiplier = 1.0 / hz / (std::chrono::duration_cast(time_since_previous_update).count() / 1e9); size_t num_cpus = 0; From 77e3ff7ff50b0e78235ab9a8ee88b258bdcaf510 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 28 Mar 2024 21:18:29 +0000 Subject: [PATCH 039/644] fix test --- tests/integration/test_async_metrics_in_cgroup/test.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_async_metrics_in_cgroup/test.py b/tests/integration/test_async_metrics_in_cgroup/test.py index 1bba42cb980..e63d53e1485 100644 --- a/tests/integration/test_async_metrics_in_cgroup/test.py +++ b/tests/integration/test_async_metrics_in_cgroup/test.py @@ -17,7 +17,7 @@ def start_cluster(): cluster.shutdown() -def test_check_client_logs_level(start_cluster): +def test_user_cpu_accounting(start_cluster): # check that our metrics sources actually exist assert ( subprocess.Popen("test -f /sys/fs/cgroup/cpu.stat".split(" ")).wait() == 0 @@ -50,7 +50,7 @@ def test_check_client_logs_level(start_cluster): """ ).strip("\n") - assert float(metric) <= 2 + assert float(metric) < 2 proc.kill() @@ -74,4 +74,5 @@ def test_check_client_logs_level(start_cluster): """ ).strip("\n") - assert 4 <= float(metric) <= 12 + # this check is really weak, but CI is tough place and we cannot guarantee that test process will get many cpu time + assert float(metric) > 1 From bc6a82d9cd68a8a4af3ef92b9a91eaa3be0aa347 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 9 Apr 2024 18:35:11 +0000 Subject: [PATCH 040/644] fix test --- .../test_async_metrics_in_cgroup/test.py | 21 +++++++++---------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_async_metrics_in_cgroup/test.py b/tests/integration/test_async_metrics_in_cgroup/test.py index e63d53e1485..00951c95a0e 100644 --- a/tests/integration/test_async_metrics_in_cgroup/test.py +++ b/tests/integration/test_async_metrics_in_cgroup/test.py @@ -18,6 +18,9 @@ def start_cluster(): def test_user_cpu_accounting(start_cluster): + if node.is_built_with_sanitizer(): + pytest.skip("Disabled for sanitizers") + # check that our metrics sources actually exist assert ( subprocess.Popen("test -f /sys/fs/cgroup/cpu.stat".split(" ")).wait() == 0 @@ -38,14 +41,12 @@ def test_user_cpu_accounting(start_cluster): metric = node.query( """ - SYSTEM FLUSH LOGS; - SELECT max(value) FROM ( SELECT toStartOfInterval(event_time, toIntervalSecond(1)) AS t, avg(value) AS value - FROM system.asynchronous_metric_log - WHERE event_time >= now() - 60 AND metric = 'OSUserTime' - GROUP BY t + FROM system.asynchronous_metric_log + WHERE event_time >= now() - 60 AND metric = 'OSUserTime' + GROUP BY t ) """ ).strip("\n") @@ -56,20 +57,18 @@ def test_user_cpu_accounting(start_cluster): # then let's test that we will account cpu time spent by the server itself node.query( - "SELECT cityHash64(*) FROM system.numbers_mt FORMAT Null SETTINGS max_execution_time=5, max_threads=8", + "SELECT cityHash64(*) FROM system.numbers_mt FORMAT Null SETTINGS max_execution_time=10", ignore_error=True, ) metric = node.query( """ - SYSTEM FLUSH LOGS; - SELECT max(value) FROM ( SELECT toStartOfInterval(event_time, toIntervalSecond(1)) AS t, avg(value) AS value - FROM system.asynchronous_metric_log - WHERE event_time >= now() - 60 AND metric = 'OSUserTime' - GROUP BY t + FROM system.asynchronous_metric_log + WHERE event_time >= now() - 60 AND metric = 'OSUserTime' + GROUP BY t ) """ ).strip("\n") From 421ace6271af8dfceaf7e0ccd7475032c694b30a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 1 May 2024 13:03:31 +0000 Subject: [PATCH 041/644] Use proper max block size in case of setting is not specified --- src/Storages/Kafka/StorageKafka2.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 6e11a8f9264..35073638e8b 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -788,9 +788,16 @@ StorageKafka2::PolledBatchInfo StorageKafka2::pollConsumer( } if (!consumer.hasMorePolledMessages() - && (total_rows >= kafka_settings->kafka_max_block_size || !check_time_limit() + && (total_rows >= getMaxBlockSize() || !check_time_limit() || failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS || consumer.needsOffsetUpdate())) { + LOG_TRACE( + log, + "Stopped collecting message for current batch. There are {} failed polled attempts, {} total rows and consumer needs " + "offset update is {}", + failed_poll_attempts, + total_rows, + consumer.needsOffsetUpdate()); break; } } From e31df2e29ae4e17f6f73fda383fba628bf626031 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 2 May 2024 09:13:22 +0000 Subject: [PATCH 042/644] Explain some decisions --- src/Storages/Kafka/StorageKafkaCommon.cpp | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp index 1319c871f4d..0b7b877a0f0 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.cpp +++ b/src/Storages/Kafka/StorageKafkaCommon.cpp @@ -378,19 +378,21 @@ void registerStorageKafka(StorageFactory & factory) const auto is_replicated_database = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY && DatabaseCatalog::instance().getDatabase(args.table_id.database_name)->getEngineName() == "Replicated"; - // TODO(antaljanosbenjamin): attach query? - // TODO(antaljanosbenjamin): why not on single atomic database? + // UUID macro is only allowed: + // - with Atomic database only with ON CLUSTER queries, otherwise it is easy to misuse: each replica would have separate uuid generated. + // - with Replicated database + // - with attach queries, as those are used on server startup const auto allow_uuid_macro = is_on_cluster || is_replicated_database || args.query.attach; auto context = args.getContext(); - /// Unfold {database} and {table} macro on table creation, so table can be renamed. + // Unfold {database} and {table} macro on table creation, so table can be renamed. if (!args.attach) { Macros::MacroExpansionInfo info; /// NOTE: it's not recursive info.expand_special_macros_only = true; info.table_id = args.table_id; - // TODO(antaljanosbenjamin): why to skip UUID here? + // We could probably unfold UUID here too, but let's keep it similar to ReplicatedMergeTree, which doesn't do the unfolding. info.table_id.uuid = UUIDHelpers::Nil; kafka_settings->kafka_keeper_path.value = context->getMacros()->expand(kafka_settings->kafka_keeper_path.value, info); @@ -405,14 +407,14 @@ void registerStorageKafka(StorageFactory & factory) settings_query->changes.setSetting("kafka_keeper_path", kafka_settings->kafka_keeper_path.value); settings_query->changes.setSetting("kafka_replica_name", kafka_settings->kafka_replica_name.value); - /// Expand other macros (such as {shard} and {replica}). We do not expand them on previous step - /// to make possible copying metadata files between replicas. + // Expand other macros (such as {replica}). We do not expand them on previous step to make possible copying metadata files between replicas. + // Disable expanding {shard} macro, because it can lead to incorrect behavior and it doesn't make sense to shard Kafka tables. Macros::MacroExpansionInfo info; info.table_id = args.table_id; if (is_replicated_database) { auto database = DatabaseCatalog::instance().getDatabase(args.table_id.database_name); - info.shard = getReplicatedDatabaseShardName(database); + info.shard.reset(); info.replica = getReplicatedDatabaseReplicaName(database); } if (!allow_uuid_macro) From 05e823a1e9eff9d0df0b6473c19eddc03811d016 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 14 May 2024 15:37:20 +0000 Subject: [PATCH 043/644] add chunked wrapper to native protocol --- src/Client/Connection.cpp | 30 ++++-- src/Client/Connection.h | 8 +- src/Core/ProtocolDefines.h | 5 +- src/IO/ReadBufferFromPocoSocket.cpp | 54 +++++++--- src/IO/ReadBufferFromPocoSocket.h | 3 + src/IO/ReadBufferFromPocoSocketChunked.cpp | 114 +++++++++++++++++++++ src/IO/ReadBufferFromPocoSocketChunked.h | 32 ++++++ src/IO/WriteBufferFromPocoSocketChunked.h | 56 ++++++++++ src/Server/TCPHandler.cpp | 50 +++++++-- src/Server/TCPHandler.h | 6 +- 10 files changed, 322 insertions(+), 36 deletions(-) create mode 100644 src/IO/ReadBufferFromPocoSocketChunked.cpp create mode 100644 src/IO/ReadBufferFromPocoSocketChunked.h create mode 100644 src/IO/WriteBufferFromPocoSocketChunked.h diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 483201509c4..970768e515e 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -4,8 +4,6 @@ #include #include #include -#include -#include #include #include #include @@ -191,10 +189,10 @@ void Connection::connect(const ConnectionTimeouts & timeouts) , tcp_keep_alive_timeout_in_sec); } - in = std::make_shared(*socket); + in = std::make_shared(*socket); in->setAsyncCallback(async_callback); - out = std::make_shared(*socket); + out = std::make_shared(*socket); out->setAsyncCallback(async_callback); connected = true; setDescription(); @@ -205,6 +203,12 @@ void Connection::connect(const ConnectionTimeouts & timeouts) if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) sendAddendum(); + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + in->enableChunked(); + out->enableChunked(); + } + LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.", server_name, server_version_major, server_version_minor, server_version_patch); } @@ -567,6 +571,7 @@ bool Connection::ping(const ConnectionTimeouts & timeouts) UInt64 pong = 0; writeVarUInt(Protocol::Client::Ping, *out); + out->finishPacket(); out->next(); if (in->eof()) @@ -611,6 +616,7 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time writeVarUInt(Protocol::Client::TablesStatusRequest, *out); request.write(*out, server_revision); + out->finishPacket(); out->next(); UInt64 response_type = 0; @@ -762,6 +768,8 @@ void Connection::sendQuery( block_profile_events_in.reset(); block_out.reset(); + out->finishPacket(); + /// Send empty block which means end of data. if (!with_pending_data) { @@ -778,6 +786,7 @@ void Connection::sendCancel() return; writeVarUInt(Protocol::Client::Cancel, *out); + out->finishPacket(); out->next(); } @@ -804,6 +813,8 @@ void Connection::sendData(const Block & block, const String & name, bool scalar) block_out->write(block); maybe_compressed_out->next(); + if (!block) + out->finishPacket(); out->next(); if (throttler) @@ -814,6 +825,7 @@ void Connection::sendIgnoredPartUUIDs(const std::vector & uuids) { writeVarUInt(Protocol::Client::IgnoredPartUUIDs, *out); writeVectorBinary(uuids, *out); + out->finishPacket(); out->next(); } @@ -823,6 +835,7 @@ void Connection::sendReadTaskResponse(const String & response) writeVarUInt(Protocol::Client::ReadTaskResponse, *out); writeVarUInt(DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION, *out); writeStringBinary(response, *out); + out->finishPacket(); out->next(); } @@ -831,6 +844,7 @@ void Connection::sendMergeTreeReadTaskResponse(const ParallelReadResponse & resp { writeVarUInt(Protocol::Client::MergeTreeReadTaskResponse, *out); response.serialize(*out); + out->finishPacket(); out->next(); } @@ -848,6 +862,8 @@ void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String copyData(input, *out); else copyData(input, *out, size); + + out->finishPacket(); out->next(); } @@ -876,6 +892,8 @@ void Connection::sendScalarsData(Scalars & data) sendData(elem.second, elem.first, true /* scalar */); } + out->finishPacket(); + out_bytes = out->count() - out_bytes; maybe_compressed_out_bytes = maybe_compressed_out->count() - maybe_compressed_out_bytes; double elapsed = watch.elapsedSeconds(); @@ -1018,13 +1036,13 @@ std::optional Connection::getResolvedAddress() const bool Connection::poll(size_t timeout_microseconds) { - return static_cast(*in).poll(timeout_microseconds); + return in->poll(timeout_microseconds); } bool Connection::hasReadPendingData() const { - return last_input_packet_type.has_value() || static_cast(*in).hasPendingData(); + return last_input_packet_type.has_value() || in->hasPendingData(); } diff --git a/src/Client/Connection.h b/src/Client/Connection.h index 9632eb9d948..e7a6d948204 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -8,8 +8,8 @@ #include -#include -#include +#include +#include #include #include @@ -207,8 +207,8 @@ private: String server_display_name; std::unique_ptr socket; - std::shared_ptr in; - std::shared_ptr out; + std::shared_ptr in; + std::shared_ptr out; std::optional last_input_packet_type; String query_id; diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 159a4c28b6d..837801edcbb 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -79,6 +79,9 @@ static constexpr auto DBMS_MIN_REVISION_WITH_SSH_AUTHENTICATION = 54466; /// Send read-only flag for Replicated tables as well static constexpr auto DBMS_MIN_REVISION_WITH_TABLE_READ_ONLY_CHECK = 54467; +/// Packets size header +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS = 54468; + /// Version of ClickHouse TCP protocol. /// /// Should be incremented manually on protocol changes. @@ -86,6 +89,6 @@ static constexpr auto DBMS_MIN_REVISION_WITH_TABLE_READ_ONLY_CHECK = 54467; /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54467; +static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54468; } diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 26cdee4140c..5fb7ea0440c 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -32,25 +32,13 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -bool ReadBufferFromPocoSocket::nextImpl() +size_t ReadBufferFromPocoSocket::readSocket(Position begin, size_t size) { ssize_t bytes_read = 0; - Stopwatch watch; - - SCOPE_EXIT({ - /// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one - ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds()); - ProfileEvents::increment(ProfileEvents::NetworkReceiveBytes, bytes_read); - }); /// Add more details to exceptions. try { - CurrentMetrics::Increment metric_increment(CurrentMetrics::NetworkReceive); - - if (internal_buffer.size() > INT_MAX) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); - /// If async_callback is specified, set socket to non-blocking mode /// and try to read data from it, if socket is not ready for reading, /// run async_callback and try again later. @@ -61,7 +49,7 @@ bool ReadBufferFromPocoSocket::nextImpl() socket.setBlocking(false); SCOPE_EXIT(socket.setBlocking(true)); bool secure = socket.secure(); - bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); + bytes_read = socket.impl()->receiveBytes(begin, static_cast(size)); /// Check EAGAIN and ERR_SSL_WANT_READ/ERR_SSL_WANT_WRITE for secure socket (reading from secure socket can write too). while (bytes_read < 0 && (errno == EAGAIN || (secure && (checkSSLWantRead(bytes_read) || checkSSLWantWrite(bytes_read))))) @@ -73,12 +61,12 @@ bool ReadBufferFromPocoSocket::nextImpl() async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), AsyncEventTimeoutType::RECEIVE, socket_description, AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); /// Try to read again. - bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); + bytes_read = socket.impl()->receiveBytes(begin, static_cast(size)); } } else { - bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); + bytes_read = socket.impl()->receiveBytes(begin, static_cast(size)); } } catch (const Poco::Net::NetException & e) @@ -99,6 +87,40 @@ bool ReadBufferFromPocoSocket::nextImpl() if (bytes_read < 0) throw NetException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot read from socket (peer: {}, local: {})", peer_address.toString(), socket.address().toString()); + return bytes_read; +} + +bool ReadBufferFromPocoSocket::readSocketExact(Position begin, size_t size) +{ + for (size_t bytes_left = size; bytes_left > 0;) + { + size_t ret = readSocket(begin + size - bytes_left, bytes_left); + if (ret == 0) + return false; + bytes_left -= ret; + } + + return true; +} + +bool ReadBufferFromPocoSocket::nextImpl() +{ + ssize_t bytes_read = 0; + Stopwatch watch; + + SCOPE_EXIT({ + /// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one + ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds()); + ProfileEvents::increment(ProfileEvents::NetworkReceiveBytes, bytes_read); + }); + + CurrentMetrics::Increment metric_increment(CurrentMetrics::NetworkReceive); + + if (internal_buffer.size() > INT_MAX) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); + + bytes_read = readSocket(internal_buffer.begin(), internal_buffer.size()); + if (read_event != ProfileEvents::end()) ProfileEvents::increment(read_event, bytes_read); diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index 76156612764..c40a54ed7ae 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -32,6 +32,9 @@ public: void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); } + size_t readSocket(Position begin, size_t size); + bool readSocketExact(Position begin, size_t size); + private: AsyncCallback async_callback; std::string socket_description; diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp new file mode 100644 index 00000000000..f0a157a7e1c --- /dev/null +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -0,0 +1,114 @@ +#include +#include + + +namespace DB::ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace DB +{ +ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size) + : ReadBufferFromPocoSocketChunked(socket_, ProfileEvents::end(), buf_size) +{} + +ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size) + : ReadBuffer(nullptr, 0), log(getLogger("Protocol")), buffer_socket(socket_, read_event_, buf_size) +{ + chassert(buf_size <= std::numeric_limits::max()); + + working_buffer = buffer_socket.buffer(); + pos = buffer_socket.position(); +} + +void ReadBufferFromPocoSocketChunked::enableChunked() +{ + chunked = true; +} + +bool ReadBufferFromPocoSocketChunked::poll(size_t timeout_microseconds) +{ + buffer_socket.position() = pos + skip_next; + return buffer_socket.poll(timeout_microseconds); +} + +void ReadBufferFromPocoSocketChunked::setAsyncCallback(AsyncCallback async_callback_) +{ + buffer_socket.setAsyncCallback(async_callback_); +} + +bool ReadBufferFromPocoSocketChunked::startChunk() +{ + do { + if (buffer_socket.read(reinterpret_cast(&chunk_left), sizeof(chunk_left)) == 0) + return false; + if (chunk_left == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: empty chunk received"); + } while (chunk_left == 0); + + return nextChunk(); +} + +bool ReadBufferFromPocoSocketChunked::nextChunk() +{ + static bool start = false; + + if (chunk_left == 0) { + start = true; + return startChunk(); + } + + if (buffer_socket.available() == 0) + if (!buffer_socket.next()) + return false; + if (start) + LOG_TEST(log, "Packet recieve started. Message {}, size {}", static_cast(*buffer_socket.position()), chunk_left); + else + LOG_TEST(log, "Packet recieve continued. Size {}", chunk_left); + + start = false; + + nextimpl_working_buffer_offset = buffer_socket.offset(); + + if (buffer_socket.available() < chunk_left) + { + working_buffer.resize(buffer_socket.offset() + buffer_socket.available()); + chunk_left -= buffer_socket.available(); + return true; + } + + working_buffer.resize(buffer_socket.offset() + chunk_left); + skip_next = std::min(static_cast(4), buffer_socket.available() - chunk_left); + + if (skip_next > 0) + std::memcpy(&chunk_left, buffer_socket.position() + chunk_left, skip_next); + if (4 > skip_next) + if (!buffer_socket.readSocketExact(reinterpret_cast(&chunk_left) + skip_next, 4 - skip_next)) + return false; + + if (chunk_left == 0) + LOG_TEST(log, "Packet recieve ended."); + + return true; +} + + +bool ReadBufferFromPocoSocketChunked::nextImpl() +{ + buffer_socket.position() = pos + skip_next; + skip_next = 0; + + if (chunked) + return nextChunk(); + + if (!buffer_socket.next()) + return false; + + pos = buffer_socket.position(); + working_buffer.resize(offset() + buffer_socket.available()); + + return true; +} + +} diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h new file mode 100644 index 00000000000..3d7d91ac93a --- /dev/null +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ReadBufferFromPocoSocketChunked: public ReadBuffer +{ +public: + explicit ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + explicit ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + + void enableChunked(); + bool poll(size_t timeout_microseconds); + void setAsyncCallback(AsyncCallback async_callback_); + +protected: + bool startChunk(); + bool nextChunk(); + bool nextImpl() override; + +private: + LoggerPtr log; + ReadBufferFromPocoSocket buffer_socket; + bool chunked = false; + UInt32 chunk_left = 0; // chunk left to read from socket + UInt8 skip_next = 0; // skip already processed bytes in buffer_socket +}; + +} diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h new file mode 100644 index 00000000000..b316393aab6 --- /dev/null +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -0,0 +1,56 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class WriteBufferFromPocoSocketChunked: public WriteBufferFromPocoSocket +{ +public: + explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, buf_size), log(getLogger("Protocol")) {} + explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, write_event_, buf_size), log(getLogger("Protocol")) {} + + void enableChunked() { chunked = true; } + void finishPacket() + { + if (!chunked) + return; + + next(); + + if (finished) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: attempt to send empty chunk"); + + LOG_TEST(log, "Packet send ended."); + finished = true; + + UInt32 s = 0; + socketSendBytes(reinterpret_cast(&s), sizeof(s)); + } +protected: + void nextImpl() override + { + if (chunked) + { + UInt32 s = static_cast(offset()); + if (finished) + LOG_TEST(log, "Packet send started. Message {}, size {}", static_cast(*buffer().begin()), s); + else + LOG_TEST(log, "Packet send continued. Size {}", s); + + finished = false; + socketSendBytes(reinterpret_cast(&s), sizeof(s)); + } + + WriteBufferFromPocoSocket::nextImpl(); + } +private: + LoggerPtr log; + bool chunked = false; + bool finished = true; +}; + +} diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index ae2f150c4a1..aa33988fdc4 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -19,8 +19,6 @@ #include #include #include -#include -#include #include #include #include @@ -253,8 +251,8 @@ void TCPHandler::runImpl() socket().setSendTimeout(send_timeout); socket().setNoDelay(true); - in = std::make_shared(socket(), read_event); - out = std::make_shared(socket(), write_event); + in = std::make_shared(socket(), read_event); + out = std::make_shared(socket(), write_event); /// Support for PROXY protocol if (parse_proxy_protocol && !receiveProxyHeader()) @@ -289,6 +287,12 @@ void TCPHandler::runImpl() if (!default_database.empty()) session->sessionContext()->setCurrentDatabase(default_database); } + + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + in->enableChunked(); + out->enableChunked(); + } } catch (const Exception & e) /// Typical for an incorrect username, password, or address. { @@ -320,7 +324,7 @@ void TCPHandler::runImpl() { Stopwatch idle_time; UInt64 timeout_ms = std::min(poll_interval, idle_connection_timeout) * 1000000; - while (tcp_server.isOpen() && !server.isCancelled() && !static_cast(*in).poll(timeout_ms)) + while (tcp_server.isOpen() && !server.isCancelled() && !in->poll(timeout_ms)) { if (idle_time.elapsedSeconds() > idle_connection_timeout) { @@ -788,7 +792,7 @@ bool TCPHandler::readDataNext() /// We are waiting for a packet from the client. Thus, every `POLL_INTERVAL` seconds check whether we need to shut down. while (true) { - if (static_cast(*in).poll(timeout_us)) + if (in->poll(timeout_us)) { /// If client disconnected. if (in->eof()) @@ -1154,6 +1158,8 @@ void TCPHandler::processTablesStatusRequest() } response.write(*out, client_tcp_protocol_version); + + out->finishPacket(); } void TCPHandler::receiveUnexpectedTablesStatusRequest() @@ -1174,6 +1180,8 @@ void TCPHandler::sendPartUUIDs() writeVarUInt(Protocol::Server::PartUUIDs, *out); writeVectorBinary(uuids, *out); + + out->finishPacket(); out->next(); } } @@ -1182,6 +1190,8 @@ void TCPHandler::sendPartUUIDs() void TCPHandler::sendReadTaskRequestAssumeLocked() { writeVarUInt(Protocol::Server::ReadTaskRequest, *out); + + out->finishPacket(); out->next(); } @@ -1190,6 +1200,8 @@ void TCPHandler::sendMergeTreeAllRangesAnnouncementAssumeLocked(InitialAllRanges { writeVarUInt(Protocol::Server::MergeTreeAllRangesAnnouncement, *out); announcement.serialize(*out); + + out->finishPacket(); out->next(); } @@ -1198,6 +1210,8 @@ void TCPHandler::sendMergeTreeReadTaskRequestAssumeLocked(ParallelReadRequest re { writeVarUInt(Protocol::Server::MergeTreeReadTaskRequest, *out); request.serialize(*out); + + out->finishPacket(); out->next(); } @@ -1206,6 +1220,8 @@ void TCPHandler::sendProfileInfo(const ProfileInfo & info) { writeVarUInt(Protocol::Server::ProfileInfo, *out); info.write(*out); + + out->finishPacket(); out->next(); } @@ -1221,6 +1237,8 @@ void TCPHandler::sendTotals(const Block & totals) state.block_out->write(totals); state.maybe_compressed_out->next(); + + out->finishPacket(); out->next(); } } @@ -1237,6 +1255,8 @@ void TCPHandler::sendExtremes(const Block & extremes) state.block_out->write(extremes); state.maybe_compressed_out->next(); + + out->finishPacket(); out->next(); } } @@ -1254,6 +1274,8 @@ void TCPHandler::sendProfileEvents() writeStringBinary("", *out); state.profile_events_block_out->write(block); + + out->finishPacket(); out->next(); auto elapsed_milliseconds = stopwatch.elapsedMilliseconds(); @@ -1291,6 +1313,8 @@ void TCPHandler::sendTimezone() LOG_DEBUG(log, "TCPHandler::sendTimezone(): {}", tz); writeVarUInt(Protocol::Server::TimezoneUpdate, *out); writeStringBinary(tz, *out); + + out->finishPacket(); out->next(); } @@ -1636,6 +1660,7 @@ bool TCPHandler::receivePacket() case Protocol::Client::Ping: writeVarUInt(Protocol::Server::Pong, *out); + out->finishPacket(); out->next(); return false; @@ -2152,7 +2177,7 @@ QueryState::CancellationStatus TCPHandler::getQueryCancellationStatus() after_check_cancelled.restart(); /// During request execution the only packet that can come from the client is stopping the query. - if (static_cast(*in).poll(0)) + if (in->poll(0)) { if (in->eof()) { @@ -2216,6 +2241,8 @@ void TCPHandler::sendData(const Block & block) state.block_out->write(block); state.maybe_compressed_out->next(); + + out->finishPacket(); out->next(); } catch (...) @@ -2251,6 +2278,8 @@ void TCPHandler::sendLogData(const Block & block) writeStringBinary("", *out); state.logs_block_out->write(block); + + out->finishPacket(); out->next(); } @@ -2262,6 +2291,7 @@ void TCPHandler::sendTableColumns(const ColumnsDescription & columns) writeStringBinary("", *out); writeStringBinary(columns.toString(), *out); + out->finishPacket(); out->next(); } @@ -2271,6 +2301,8 @@ void TCPHandler::sendException(const Exception & e, bool with_stack_trace) writeVarUInt(Protocol::Server::Exception, *out); writeException(e, *out, with_stack_trace); + + out->finishPacket(); out->next(); } @@ -2281,6 +2313,8 @@ void TCPHandler::sendEndOfStream() state.io.setAllDataSent(); writeVarUInt(Protocol::Server::EndOfStream, *out); + + out->finishPacket(); out->next(); } @@ -2299,6 +2333,8 @@ void TCPHandler::sendProgress() increment.elapsed_ns = current_elapsed_ns - state.prev_elapsed_ns; state.prev_elapsed_ns = current_elapsed_ns; increment.write(*out, client_tcp_protocol_version); + + out->finishPacket(); out->next(); } diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 191617f1905..67d77381167 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -19,6 +19,8 @@ #include #include #include +#include +#include #include "IServer.h" #include "Interpreters/AsynchronousInsertQueue.h" @@ -204,8 +206,8 @@ private: ClientInfo::QueryKind query_kind = ClientInfo::QueryKind::NO_QUERY; /// Streams for reading/writing from/to client connection socket. - std::shared_ptr in; - std::shared_ptr out; + std::shared_ptr in; + std::shared_ptr out; ProfileEvents::Event read_event; ProfileEvents::Event write_event; From daf8277e55058e42fddafc49416164d5cb0ab601 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 14 May 2024 16:00:58 +0000 Subject: [PATCH 044/644] fix --- src/IO/ReadBufferFromPocoSocketChunked.cpp | 17 ++++++++--------- src/IO/ReadBufferFromPocoSocketChunked.h | 1 + 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index f0a157a7e1c..33bed2a32c4 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -52,22 +52,21 @@ bool ReadBufferFromPocoSocketChunked::startChunk() bool ReadBufferFromPocoSocketChunked::nextChunk() { - static bool start = false; - - if (chunk_left == 0) { - start = true; + if (chunk_left == 0) + { + started = true; return startChunk(); } if (buffer_socket.available() == 0) if (!buffer_socket.next()) return false; - if (start) - LOG_TEST(log, "Packet recieve started. Message {}, size {}", static_cast(*buffer_socket.position()), chunk_left); + if (started) + LOG_TEST(log, "Packet receive started. Message {}, size {}", static_cast(*buffer_socket.position()), chunk_left); else - LOG_TEST(log, "Packet recieve continued. Size {}", chunk_left); + LOG_TEST(log, "Packet receive continued. Size {}", chunk_left); - start = false; + started = false; nextimpl_working_buffer_offset = buffer_socket.offset(); @@ -88,7 +87,7 @@ bool ReadBufferFromPocoSocketChunked::nextChunk() return false; if (chunk_left == 0) - LOG_TEST(log, "Packet recieve ended."); + LOG_TEST(log, "Packet receive ended."); return true; } diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index 3d7d91ac93a..5930285e18a 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -27,6 +27,7 @@ private: bool chunked = false; UInt32 chunk_left = 0; // chunk left to read from socket UInt8 skip_next = 0; // skip already processed bytes in buffer_socket + bool started = false; }; } From dfdf31f1b6efbbda847a693a22969c2187a949f7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 14 May 2024 18:09:11 +0000 Subject: [PATCH 045/644] host-net conversion --- src/IO/NetUtils.h | 26 ++++++++++++++++++++++ src/IO/ReadBufferFromPocoSocketChunked.cpp | 16 ++++++++----- src/IO/WriteBufferFromPocoSocketChunked.h | 2 ++ 3 files changed, 38 insertions(+), 6 deletions(-) create mode 100644 src/IO/NetUtils.h diff --git a/src/IO/NetUtils.h b/src/IO/NetUtils.h new file mode 100644 index 00000000000..ac6b5eec9a7 --- /dev/null +++ b/src/IO/NetUtils.h @@ -0,0 +1,26 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +template +constexpr T netToHost(T value) noexcept +{ + if constexpr (std::endian::native != std::endian::big) + return std::byteswap(value); + return value; +} + +template +constexpr T hostToNet(T value) noexcept +{ + if constexpr (std::endian::native != std::endian::big) + return std::byteswap(value); + return value; +} + +} diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 33bed2a32c4..27903761934 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB::ErrorCodes @@ -9,6 +10,7 @@ namespace DB::ErrorCodes namespace DB { + ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size) : ReadBufferFromPocoSocketChunked(socket_, ProfileEvents::end(), buf_size) {} @@ -40,12 +42,12 @@ void ReadBufferFromPocoSocketChunked::setAsyncCallback(AsyncCallback async_callb bool ReadBufferFromPocoSocketChunked::startChunk() { - do { - if (buffer_socket.read(reinterpret_cast(&chunk_left), sizeof(chunk_left)) == 0) - return false; - if (chunk_left == 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: empty chunk received"); - } while (chunk_left == 0); + if (buffer_socket.read(reinterpret_cast(&chunk_left), sizeof(chunk_left)) == 0) + return false; + if (chunk_left == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: empty chunk received"); + + chunk_left = netToHost(chunk_left); return nextChunk(); } @@ -86,6 +88,8 @@ bool ReadBufferFromPocoSocketChunked::nextChunk() if (!buffer_socket.readSocketExact(reinterpret_cast(&chunk_left) + skip_next, 4 - skip_next)) return false; + chunk_left = netToHost(chunk_left); + if (chunk_left == 0) LOG_TEST(log, "Packet receive ended."); diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index b316393aab6..4481dfdedfc 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -42,6 +43,7 @@ protected: LOG_TEST(log, "Packet send continued. Size {}", s); finished = false; + s = hostToNet(s); socketSendBytes(reinterpret_cast(&s), sizeof(s)); } From 88a833335f7e7e9fae85e74d250677f415905292 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 14 May 2024 20:25:26 +0000 Subject: [PATCH 046/644] fix --- src/IO/WriteBufferFromPocoSocketChunked.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 4481dfdedfc..39cdd93501b 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -8,6 +8,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + class WriteBufferFromPocoSocketChunked: public WriteBufferFromPocoSocket { public: From ad204887a2516e5053035c709735bf6c99ddba21 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 15 May 2024 20:47:54 +0000 Subject: [PATCH 047/644] bugs fixed, switch chunk length to little endian --- src/Client/Connection.cpp | 2 +- src/IO/NetUtils.h | 32 +++++++++++++++++ src/IO/ReadBufferFromPocoSocketChunked.cpp | 42 +++++++++++++++------- src/IO/ReadBufferFromPocoSocketChunked.h | 3 +- src/IO/WriteBufferFromPocoSocketChunked.h | 2 +- 5 files changed, 65 insertions(+), 16 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 970768e515e..3a0f3771e7a 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -1042,7 +1042,7 @@ bool Connection::poll(size_t timeout_microseconds) bool Connection::hasReadPendingData() const { - return last_input_packet_type.has_value() || in->hasPendingData(); + return last_input_packet_type.has_value() || in->hasBufferedData(); } diff --git a/src/IO/NetUtils.h b/src/IO/NetUtils.h index ac6b5eec9a7..12f09524ae7 100644 --- a/src/IO/NetUtils.h +++ b/src/IO/NetUtils.h @@ -23,4 +23,36 @@ constexpr T hostToNet(T value) noexcept return value; } +template +constexpr T toLittleEndian(T value) noexcept +{ + if constexpr (std::endian::native == std::endian::big) + return std::byteswap(value); + return value; +} + +template +constexpr T toBigEndian(T value) noexcept +{ + if constexpr (std::endian::native != std::endian::big) + return std::byteswap(value); + return value; +} + +template +constexpr T fromLittleEndian(T value) noexcept +{ + if constexpr (std::endian::native == std::endian::big) + return std::byteswap(value); + return value; +} + +template +constexpr T fromBigEndian(T value) noexcept +{ + if constexpr (std::endian::native != std::endian::big) + return std::byteswap(value); + return value; +} + } diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 27903761934..247d8c8ec6a 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -27,11 +27,14 @@ ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Sock void ReadBufferFromPocoSocketChunked::enableChunked() { chunked = true; + buffer_socket.position() = pos; } bool ReadBufferFromPocoSocketChunked::poll(size_t timeout_microseconds) { - buffer_socket.position() = pos + skip_next; + if (!chunked) + buffer_socket.position() = pos; + return buffer_socket.poll(timeout_microseconds); } @@ -42,12 +45,12 @@ void ReadBufferFromPocoSocketChunked::setAsyncCallback(AsyncCallback async_callb bool ReadBufferFromPocoSocketChunked::startChunk() { - if (buffer_socket.read(reinterpret_cast(&chunk_left), sizeof(chunk_left)) == 0) + if (buffer_socket.read(reinterpret_cast(&chunk_left), sizeof(chunk_left)) < sizeof(chunk_left)) return false; if (chunk_left == 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: empty chunk received"); - chunk_left = netToHost(chunk_left); + chunk_left = fromLittleEndian(chunk_left); return nextChunk(); } @@ -76,19 +79,23 @@ bool ReadBufferFromPocoSocketChunked::nextChunk() { working_buffer.resize(buffer_socket.offset() + buffer_socket.available()); chunk_left -= buffer_socket.available(); + buffer_socket.position() += buffer_socket.available(); return true; } working_buffer.resize(buffer_socket.offset() + chunk_left); - skip_next = std::min(static_cast(4), buffer_socket.available() - chunk_left); + UInt8 buffered = std::min(static_cast(4), buffer_socket.available() - chunk_left); - if (skip_next > 0) - std::memcpy(&chunk_left, buffer_socket.position() + chunk_left, skip_next); - if (4 > skip_next) - if (!buffer_socket.readSocketExact(reinterpret_cast(&chunk_left) + skip_next, 4 - skip_next)) + buffer_socket.position() += chunk_left; + if (buffered > 0) + std::memcpy(&chunk_left, buffer_socket.position(), buffered); + buffer_socket.position() += buffered; + + if (4 > buffered) + if (!buffer_socket.readSocketExact(reinterpret_cast(&chunk_left) + buffered, 4 - buffered)) return false; - chunk_left = netToHost(chunk_left); + chunk_left = fromLittleEndian(chunk_left); if (chunk_left == 0) LOG_TEST(log, "Packet receive ended."); @@ -99,14 +106,23 @@ bool ReadBufferFromPocoSocketChunked::nextChunk() bool ReadBufferFromPocoSocketChunked::nextImpl() { - buffer_socket.position() = pos + skip_next; - skip_next = 0; - if (chunked) - return nextChunk(); + { + if (!nextChunk()) + { + pos = buffer_socket.position(); + return false; + } + return true; + } + + buffer_socket.position() = pos; if (!buffer_socket.next()) + { + pos = buffer_socket.position(); return false; + } pos = buffer_socket.position(); working_buffer.resize(offset() + buffer_socket.available()); diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index 5930285e18a..6f99db4489a 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -16,6 +16,8 @@ public: bool poll(size_t timeout_microseconds); void setAsyncCallback(AsyncCallback async_callback_); + bool hasBufferedData() const { return hasPendingData() || buffer_socket.hasPendingData(); } + protected: bool startChunk(); bool nextChunk(); @@ -26,7 +28,6 @@ private: ReadBufferFromPocoSocket buffer_socket; bool chunked = false; UInt32 chunk_left = 0; // chunk left to read from socket - UInt8 skip_next = 0; // skip already processed bytes in buffer_socket bool started = false; }; diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 39cdd93501b..070e87feff2 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -48,7 +48,7 @@ protected: LOG_TEST(log, "Packet send continued. Size {}", s); finished = false; - s = hostToNet(s); + s = toLittleEndian(s); socketSendBytes(reinterpret_cast(&s), sizeof(s)); } From 6378184c7f004e211d86c3fd7a4f482e45b01a59 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 20 May 2024 14:15:47 +0000 Subject: [PATCH 048/644] fix, add some introspection functionality --- src/IO/ReadBufferFromPocoSocketChunked.cpp | 3 ++- src/IO/ReadBufferFromPocoSocketChunked.h | 5 +++++ src/IO/WriteBufferFromPocoSocketChunked.h | 10 ++++++++++ 3 files changed, 17 insertions(+), 1 deletion(-) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 247d8c8ec6a..4d40d8b4f14 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -16,7 +16,7 @@ ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Sock {} ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size) - : ReadBuffer(nullptr, 0), log(getLogger("Protocol")), buffer_socket(socket_, read_event_, buf_size) + : ReadBuffer(nullptr, 0), log(getLogger("Protocol")), peer_address(socket_.peerAddress()), our_address(socket_.address()), buffer_socket(socket_, read_event_, buf_size) { chassert(buf_size <= std::numeric_limits::max()); @@ -28,6 +28,7 @@ void ReadBufferFromPocoSocketChunked::enableChunked() { chunked = true; buffer_socket.position() = pos; + working_buffer.resize(offset()); } bool ReadBufferFromPocoSocketChunked::poll(size_t timeout_microseconds) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index 6f99db4489a..c70363cf7d8 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -18,6 +18,9 @@ public: bool hasBufferedData() const { return hasPendingData() || buffer_socket.hasPendingData(); } + Poco::Net::SocketAddress peerAddress() { return peer_address; } + Poco::Net::SocketAddress ourAddress() { return our_address; } + protected: bool startChunk(); bool nextChunk(); @@ -25,6 +28,8 @@ protected: private: LoggerPtr log; + Poco::Net::SocketAddress peer_address; + Poco::Net::SocketAddress our_address; ReadBufferFromPocoSocket buffer_socket; bool chunked = false; UInt32 chunk_left = 0; // chunk left to read from socket diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 070e87feff2..6c35db62c0c 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -54,6 +54,16 @@ protected: WriteBufferFromPocoSocket::nextImpl(); } + + Poco::Net::SocketAddress peerAddress() + { + return peer_address; + } + + Poco::Net::SocketAddress ourAddress() + { + return our_address; + } private: LoggerPtr log; bool chunked = false; From 5308256c67c5781916018c321273f04fd21c4545 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 20 May 2024 16:25:19 +0000 Subject: [PATCH 049/644] enable chunked before processing defaul database --- src/Server/TCPHandler.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b3dbd118d8b..070cd0e3247 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -277,6 +277,12 @@ void TCPHandler::runImpl() if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) receiveAddendum(); + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + in->enableChunked(); + out->enableChunked(); + } + if (!is_interserver_mode) { /// If session created, then settings in session context has been updated. @@ -287,12 +293,6 @@ void TCPHandler::runImpl() if (!default_database.empty()) session->sessionContext()->setCurrentDatabase(default_database); } - - if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) - { - in->enableChunked(); - out->enableChunked(); - } } catch (const Exception & e) /// Typical for an incorrect username, password, or address. { From 9e747cd45312302935cbf15ea518808d4ac9c8c8 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 22 May 2024 01:20:00 +0000 Subject: [PATCH 050/644] fix bug with profile stats in WriteBufferFromPocoSocket --- src/IO/WriteBufferFromPocoSocket.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index 10d9fd131cd..e29b3b2cddd 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -183,6 +183,7 @@ WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_ , socket(socket_) , peer_address(socket.peerAddress()) , our_address(socket.address()) + , write_event(ProfileEvents::end()) , socket_description("socket (" + peer_address.toString() + ")") { } From 34702b30bcfe3401991fe7c792c02a80185acdf2 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 22 May 2024 03:21:10 +0000 Subject: [PATCH 051/644] fix test --- .../0_stateless/02532_send_logs_level_test.reference | 3 --- tests/queries/0_stateless/02532_send_logs_level_test.sh | 8 ++++++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02532_send_logs_level_test.reference b/tests/queries/0_stateless/02532_send_logs_level_test.reference index 7e51b888d9c..e69de29bb2d 100644 --- a/tests/queries/0_stateless/02532_send_logs_level_test.reference +++ b/tests/queries/0_stateless/02532_send_logs_level_test.reference @@ -1,3 +0,0 @@ - MergeTreeMarksLoader: Loading marks from path data.cmrk3 - MergeTreeRangeReader: First reader returned: num_rows: 1, columns: 1, total_rows_per_granule: 1, no filter, column[0]: Int32(size = 1), requested columns: key - MergeTreeRangeReader: read() returned num_rows: 1, columns: 1, total_rows_per_granule: 1, no filter, column[0]: Int32(size = 1), sample block key diff --git a/tests/queries/0_stateless/02532_send_logs_level_test.sh b/tests/queries/0_stateless/02532_send_logs_level_test.sh index 4afc6d4496b..f2940e9c005 100755 --- a/tests/queries/0_stateless/02532_send_logs_level_test.sh +++ b/tests/queries/0_stateless/02532_send_logs_level_test.sh @@ -17,6 +17,10 @@ $CLICKHOUSE_CLIENT -nm -q " # instead of "last" value, hence you cannot simply append another # --send_logs_level here. CLICKHOUSE_CLIENT_CLEAN=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=test/g') -$CLICKHOUSE_CLIENT_CLEAN -q "select * from data SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" |& grep -o -e '.*' -e '.*' -$CLICKHOUSE_CLIENT -q "drop table data" +set -e + +trap "$CLICKHOUSE_CLIENT -q 'drop table data'" EXIT + +$CLICKHOUSE_CLIENT_CLEAN -q "select * from data SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" |& (! grep -q -o -e '.*') +$CLICKHOUSE_CLIENT_CLEAN -q "select * from data SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" |& grep -q -o -e '.*' From 6c3556dfda92ea9d04ff5db8427a58aa7ab35750 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 22 May 2024 04:07:52 +0000 Subject: [PATCH 052/644] fix test --- tests/queries/0_stateless/02532_send_logs_level_test.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02532_send_logs_level_test.sh b/tests/queries/0_stateless/02532_send_logs_level_test.sh index f2940e9c005..b74fcf78ad1 100755 --- a/tests/queries/0_stateless/02532_send_logs_level_test.sh +++ b/tests/queries/0_stateless/02532_send_logs_level_test.sh @@ -20,7 +20,7 @@ CLICKHOUSE_CLIENT_CLEAN=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level set -e -trap "$CLICKHOUSE_CLIENT -q 'drop table data'" EXIT +trap '$CLICKHOUSE_CLIENT -q "drop table data"' EXIT $CLICKHOUSE_CLIENT_CLEAN -q "select * from data SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" |& (! grep -q -o -e '.*') $CLICKHOUSE_CLIENT_CLEAN -q "select * from data SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" |& grep -q -o -e '.*' From 69cd5ae549cf7acc4de756a70c9b632d139e50fe Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 22 May 2024 16:39:25 +0000 Subject: [PATCH 053/644] process possibly remaining message after network error --- src/Client/ClientBase.cpp | 35 +++++++++++++++++++++++------------ 1 file changed, 23 insertions(+), 12 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index b6f821794f1..f3e53efd994 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -951,6 +951,8 @@ void ClientBase::processTextAsSingleQuery(const String & full_query) } catch (Exception & e) { + if (server_exception) + server_exception->rethrow(); if (!is_interactive) e.addMessage("(in query: {})", full_query); throw; @@ -1069,19 +1071,28 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa QueryInterruptHandler::start(signals_before_stop); SCOPE_EXIT({ QueryInterruptHandler::stop(); }); - connection->sendQuery( - connection_parameters.timeouts, - query, - query_parameters, - global_context->getCurrentQueryId(), - query_processing_stage, - &global_context->getSettingsRef(), - &global_context->getClientInfo(), - true, - [&](const Progress & progress) { onProgress(progress); }); + try { + connection->sendQuery( + connection_parameters.timeouts, + query, + query_parameters, + global_context->getCurrentQueryId(), + query_processing_stage, + &global_context->getSettingsRef(), + &global_context->getClientInfo(), + true, + [&](const Progress & progress) { onProgress(progress); }); + + if (send_external_tables) + sendExternalTables(parsed_query); + } + catch (const NetException &) + { + // We still want to attempt to process whatever we already recieved or can recieve (socket receive buffer can be not empty) + receiveResult(parsed_query, signals_before_stop, settings.partial_result_on_first_cancel); + throw; + } - if (send_external_tables) - sendExternalTables(parsed_query); receiveResult(parsed_query, signals_before_stop, settings.partial_result_on_first_cancel); break; From 99bd796011aee169f3c4de25b07b330094c4a41a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 22 May 2024 16:58:50 +0000 Subject: [PATCH 054/644] fix spelling --- 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 f3e53efd994..1b8fe83eb51 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1088,7 +1088,7 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa } catch (const NetException &) { - // We still want to attempt to process whatever we already recieved or can recieve (socket receive buffer can be not empty) + // We still want to attempt to process whatever we already received or can receive (socket receive buffer can be not empty) receiveResult(parsed_query, signals_before_stop, settings.partial_result_on_first_cancel); throw; } From 94bc0a1e966d95b8a2180f9504ed93592d2026ed Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 23 May 2024 22:01:32 +0000 Subject: [PATCH 055/644] add config parameters and client arguments, make default notchunked_optional --- programs/benchmark/Benchmark.cpp | 52 +++++++++++++++++- src/Client/ClientBase.cpp | 39 ++++++++++++++ src/Client/Connection.cpp | 54 ++++++++++++++++++- src/Client/Connection.h | 5 ++ src/Client/ConnectionParameters.cpp | 3 ++ src/Client/ConnectionParameters.h | 2 + src/Client/ConnectionPool.cpp | 6 ++- src/Client/ConnectionPool.h | 15 +++++- .../ClickHouseDictionarySource.cpp | 8 ++- src/Dictionaries/ClickHouseDictionarySource.h | 2 + src/Interpreters/Cluster.cpp | 11 +++- src/Interpreters/Cluster.h | 2 + src/Server/TCPHandler.cpp | 44 ++++++++++++++- src/Server/TCPHandler.h | 2 + .../DistributedAsyncInsertDirectoryQueue.cpp | 2 + src/Storages/StorageReplicatedMergeTree.cpp | 3 +- 16 files changed, 240 insertions(+), 10 deletions(-) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 48dca82eb2b..251761e0bad 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -75,6 +75,8 @@ public: const String & default_database_, const String & user_, const String & password_, + const String & proto_send_chunked_, + const String & proto_recv_chunked_, const String & quota_key_, const String & stage, bool randomize_, @@ -128,7 +130,9 @@ public: connections.emplace_back(std::make_unique( concurrency, cur_host, cur_port, - default_database_, user_, password_, quota_key_, + default_database_, user_, password_, + proto_send_chunked_, proto_recv_chunked_, + quota_key_, /* cluster_= */ "", /* cluster_secret_= */ "", /* client_name_= */ std::string(DEFAULT_CLIENT_NAME), @@ -662,6 +666,50 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) Strings hosts = options.count("host") ? options["host"].as() : Strings({"localhost"}); + String proto_send_chunked {"notchunked_optional"}; + String proto_recv_chunked {"notchunked_optional"}; + + if (options.count("proto_caps")) + { + std::string proto_caps_str = options["proto_caps"].as(); + + std::vector proto_caps; + splitInto<','>(proto_caps, proto_caps_str); + + for (auto cap_str : proto_caps) + { + std::string direction; + + if (cap_str.starts_with("send_")) + { + direction = "send"; + cap_str = cap_str.substr(std::string_view("send_").size()); + } + else if (cap_str.starts_with("recv_")) + { + direction = "recv"; + cap_str = cap_str.substr(std::string_view("recv_").size()); + } + + if (cap_str != "chunked" && cap_str != "notchunked" && cap_str != "chunked_optional" && cap_str != "notchunked_optional") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "proto_caps option is incorrect ({})", proto_caps_str); + + if (direction.empty()) + { + proto_send_chunked = cap_str; + proto_recv_chunked = cap_str; + } + else + { + if (direction == "send") + proto_send_chunked = cap_str; + else + proto_recv_chunked = cap_str; + } + } + } + + Benchmark benchmark( options["concurrency"].as(), options["delay"].as(), @@ -673,6 +721,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) options["database"].as(), options["user"].as(), options["password"].as(), + proto_send_chunked, + proto_recv_chunked, options["quota_key"].as(), options["stage"].as(), options.count("randomize"), diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 1b8fe83eb51..0bceee6ea4d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -75,9 +75,11 @@ #include #include #include +#include #include #include +#include #include "config.h" namespace fs = std::filesystem; @@ -2993,6 +2995,8 @@ void ClientBase::init(int argc, char ** argv) ("config-file,C", po::value(), "config-file path") + ("proto_caps", po::value(), "enable/disable chunked protocol: chunked_optional, notchunked, notchunked_optional, send_chunked, send_chunked_optional, send_notchunked, send_notchunked_optional, recv_chunked, recv_chunked_optional, recv_notchunked, recv_notchunked_optional") + ("query,q", po::value>()->multitoken(), R"(query; can be specified multiple times (--query "SELECT 1" --query "SELECT 2"...))") ("queries-file", po::value>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)") ("multiquery,n", "If specified, multiple queries separated by semicolons can be listed after --query. For convenience, it is also possible to omit --query and pass the queries directly after --multiquery.") @@ -3162,6 +3166,41 @@ void ClientBase::init(int argc, char ** argv) if (options.count("server_logs_file")) server_logs_file = options["server_logs_file"].as(); + if (options.count("proto_caps")) + { + std::string proto_caps_str = options["proto_caps"].as(); + + std::vector proto_caps; + splitInto<','>(proto_caps, proto_caps_str); + + for (auto cap_str : proto_caps) + { + std::string direction; + + if (cap_str.starts_with("send_")) + { + direction = "send"; + cap_str = cap_str.substr(std::string_view("send_").size()); + } + else if (cap_str.starts_with("recv_")) + { + direction = "recv"; + cap_str = cap_str.substr(std::string_view("recv_").size()); + } + + if (cap_str != "chunked" && cap_str != "notchunked" && cap_str != "chunked_optional" && cap_str != "notchunked_optional") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "proto_caps option is incorrect ({})", proto_caps_str); + + if (direction.empty()) + { + config().setString("proto_caps.send", std::string(cap_str)); + config().setString("proto_caps.recv", std::string(cap_str)); + } + else + config().setString("proto_caps." + direction, std::string(cap_str)); + } + } + query_processing_stage = QueryProcessingStage::fromString(options["stage"].as()); query_kind = parseQueryKind(options["query_kind"].as()); profile_events.print = options.count("print-profile-events"); diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 082fe8d5098..9327b694d29 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -71,6 +71,7 @@ Connection::~Connection() = default; Connection::Connection(const String & host_, UInt16 port_, const String & default_database_, const String & user_, const String & password_, + const String & proto_send_chunked_, const String & proto_recv_chunked_, [[maybe_unused]] const SSHKey & ssh_private_key_, const String & quota_key_, const String & cluster_, @@ -80,6 +81,7 @@ Connection::Connection(const String & host_, UInt16 port_, Protocol::Secure secure_) : host(host_), port(port_), default_database(default_database_) , user(user_), password(password_) + , proto_send_chunked(proto_send_chunked_), proto_recv_chunked(proto_recv_chunked_) #if USE_SSH , ssh_private_key(ssh_private_key_) #endif @@ -206,13 +208,46 @@ void Connection::connect(const ConnectionTimeouts & timeouts) sendHello(); receiveHello(timeouts.handshake_timeout); + bool out_chunked = false; + bool in_chunked = false; + + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + auto is_chunked = [](const String & chunked_srv_str, const String & chunked_cl_str, const String & direction) + { + bool chunked_srv = chunked_srv_str.starts_with("chunked"); + bool optional_srv = chunked_srv_str.ends_with("_optional"); + bool chunked_cl = chunked_cl_str.starts_with("chunked"); + bool optional_cl = chunked_cl_str.ends_with("_optional"); + + if (optional_srv) + return chunked_cl; + if (optional_cl) + return chunked_srv; + if (chunked_cl != chunked_srv) + throw NetException( + ErrorCodes::NETWORK_ERROR, + "Incompatible protocol: {} set to {}, server requires {}", + direction, + chunked_cl ? "chunked" : "notchunked", + chunked_srv ? "chunked" : "notchunked"); + + return chunked_srv; + }; + + out_chunked = is_chunked(proto_recv_chunked_srv, proto_send_chunked, "send"); + in_chunked = is_chunked(proto_send_chunked_srv, proto_recv_chunked, "recv"); + } + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) sendAddendum(); if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) { - in->enableChunked(); - out->enableChunked(); + if (out_chunked) + out->enableChunked(); + if (in_chunked) + in->enableChunked(); } LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.", @@ -359,6 +394,13 @@ void Connection::sendAddendum() { if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_QUOTA_KEY) writeStringBinary(quota_key, *out); + + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + writeStringBinary(proto_send_chunked, *out); + writeStringBinary(proto_recv_chunked, *out); + } + out->next(); } @@ -438,6 +480,12 @@ void Connection::receiveHello(const Poco::Timespan & handshake_timeout) else server_version_patch = server_revision; + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + readStringBinary(proto_send_chunked_srv, *in); + readStringBinary(proto_recv_chunked_srv, *in); + } + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES) { UInt64 rules_size; @@ -1327,6 +1375,8 @@ ServerConnectionPtr Connection::createConnection(const ConnectionParameters & pa parameters.default_database, parameters.user, parameters.password, + parameters.proto_send_chunked, + parameters.proto_recv_chunked, parameters.ssh_private_key, parameters.quota_key, "", /* cluster */ diff --git a/src/Client/Connection.h b/src/Client/Connection.h index e7a6d948204..a04ccd44627 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -52,6 +52,7 @@ public: Connection(const String & host_, UInt16 port_, const String & default_database_, const String & user_, const String & password_, + const String & proto_send_chunked_, const String & proto_recv_chunked_, const SSHKey & ssh_private_key_, const String & quota_key_, const String & cluster_, @@ -169,6 +170,10 @@ private: String default_database; String user; String password; + String proto_send_chunked; + String proto_recv_chunked; + String proto_send_chunked_srv; + String proto_recv_chunked_srv; #if USE_SSH SSHKey ssh_private_key; #endif diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index 774f3375f63..430c462084a 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -103,6 +103,9 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati #endif } + proto_send_chunked = config.getString("proto_caps.send", "notchunked_optional"); + proto_recv_chunked = config.getString("proto_caps.recv", "notchunked_optional"); + quota_key = config.getString("quota_key", ""); /// By default compression is disabled if address looks like localhost. diff --git a/src/Client/ConnectionParameters.h b/src/Client/ConnectionParameters.h index f23522d48b3..85174924016 100644 --- a/src/Client/ConnectionParameters.h +++ b/src/Client/ConnectionParameters.h @@ -20,6 +20,8 @@ struct ConnectionParameters std::string default_database; std::string user; std::string password; + std::string proto_send_chunked; + std::string proto_recv_chunked; std::string quota_key; SSHKey ssh_private_key; Protocol::Secure security = Protocol::Secure::Disable; diff --git a/src/Client/ConnectionPool.cpp b/src/Client/ConnectionPool.cpp index 5cabb1465d1..05cb97cadc7 100644 --- a/src/Client/ConnectionPool.cpp +++ b/src/Client/ConnectionPool.cpp @@ -12,6 +12,8 @@ ConnectionPoolPtr ConnectionPoolFactory::get( String default_database, String user, String password, + String proto_send_chunked, + String proto_recv_chunked, String quota_key, String cluster, String cluster_secret, @@ -21,7 +23,7 @@ ConnectionPoolPtr ConnectionPoolFactory::get( Priority priority) { Key key{ - max_connections, host, port, default_database, user, password, quota_key, cluster, cluster_secret, client_name, compression, secure, priority}; + max_connections, host, port, default_database, user, password, proto_send_chunked, proto_recv_chunked, quota_key, cluster, cluster_secret, client_name, compression, secure, priority}; std::lock_guard lock(mutex); auto [it, inserted] = pools.emplace(key, ConnectionPoolPtr{}); @@ -38,6 +40,8 @@ ConnectionPoolPtr ConnectionPoolFactory::get( default_database, user, password, + proto_send_chunked, + proto_recv_chunked, quota_key, cluster, cluster_secret, diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h index d35c2552461..2df97dfb454 100644 --- a/src/Client/ConnectionPool.h +++ b/src/Client/ConnectionPool.h @@ -72,6 +72,8 @@ public: const String & default_database_, const String & user_, const String & password_, + const String & proto_send_chunked_, + const String & proto_recv_chunked_, const String & quota_key_, const String & cluster_, const String & cluster_secret_, @@ -84,6 +86,8 @@ public: , default_database(default_database_) , user(user_) , password(password_) + , proto_send_chunked(proto_send_chunked_) + , proto_recv_chunked(proto_recv_chunked_) , quota_key(quota_key_) , cluster(cluster_) , cluster_secret(cluster_secret_) @@ -123,7 +127,9 @@ protected: { return std::make_shared( host, port, - default_database, user, password, SSHKey(), quota_key, + default_database, user, password, + proto_send_chunked, proto_recv_chunked, + SSHKey(), quota_key, cluster, cluster_secret, client_name, compression, secure); } @@ -132,6 +138,8 @@ private: String default_database; String user; String password; + String proto_send_chunked; + String proto_recv_chunked; String quota_key; /// For inter-server authorization @@ -157,6 +165,8 @@ public: String default_database; String user; String password; + String proto_send_chunked; + String proto_recv_chunked; String quota_key; String cluster; String cluster_secret; @@ -180,6 +190,8 @@ public: String default_database, String user, String password, + String proto_send_chunked, + String proto_recv_chunked, String quota_key, String cluster, String cluster_secret, @@ -197,6 +209,7 @@ inline bool operator==(const ConnectionPoolFactory::Key & lhs, const ConnectionP { return lhs.max_connections == rhs.max_connections && lhs.host == rhs.host && lhs.port == rhs.port && lhs.default_database == rhs.default_database && lhs.user == rhs.user && lhs.password == rhs.password + && lhs.proto_send_chunked == rhs.proto_send_chunked && lhs.proto_recv_chunked == rhs.proto_recv_chunked && lhs.quota_key == rhs.quota_key && lhs.cluster == rhs.cluster && lhs.cluster_secret == rhs.cluster_secret && lhs.client_name == rhs.client_name && lhs.compression == rhs.compression && lhs.secure == rhs.secure && lhs.priority == rhs.priority; diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index bf16f315ddf..3b096da92c6 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -51,6 +51,8 @@ namespace configuration.db, configuration.user, configuration.password, + configuration.proto_send_chunked, + configuration.proto_recv_chunked, configuration.quota_key, "", /* cluster */ "", /* cluster_secret */ @@ -222,7 +224,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) { validateNamedCollection( *named_collection, {}, ValidateKeysMultiset{ - "secure", "host", "hostname", "port", "user", "username", "password", "quota_key", "name", + "secure", "host", "hostname", "port", "user", "username", "password", "proto_send_chunked", "proto_recv_chunked", "quota_key", "name", "db", "database", "table","query", "where", "invalidate_query", "update_field", "update_lag"}); const auto secure = named_collection->getOrDefault("secure", false); @@ -234,6 +236,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .host = host, .user = named_collection->getAnyOrDefault({"user", "username"}, "default"), .password = named_collection->getOrDefault("password", ""), + .proto_send_chunked = named_collection->getOrDefault("proto_send_chunked", "notchunked_optional"), + .proto_recv_chunked = named_collection->getOrDefault("proto_recv_chunked", "notchunked_optional"), .quota_key = named_collection->getOrDefault("quota_key", ""), .db = named_collection->getAnyOrDefault({"db", "database"}, default_database), .table = named_collection->getOrDefault("table", ""), @@ -258,6 +262,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .host = host, .user = config.getString(settings_config_prefix + ".user", "default"), .password = config.getString(settings_config_prefix + ".password", ""), + .proto_send_chunked = config.getString(settings_config_prefix + ".proto_caps.send", "notchunked_optional"), + .proto_recv_chunked = config.getString(settings_config_prefix + ".proto_caps.recv", "notchunked_optional"), .quota_key = config.getString(settings_config_prefix + ".quota_key", ""), .db = config.getString(settings_config_prefix + ".db", default_database), .table = config.getString(settings_config_prefix + ".table", ""), diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index 3357514eab2..faf9e5f8009 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -23,6 +23,8 @@ public: const std::string host; const std::string user; const std::string password; + const std::string proto_send_chunked; + const std::string proto_recv_chunked; const std::string quota_key; const std::string db; const std::string table; diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 59c98491c14..1d7ccd484d0 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -113,6 +113,9 @@ Cluster::Address::Address( secure = ConfigHelper::getBool(config, config_prefix + ".secure", false, /* empty_as */true) ? Protocol::Secure::Enable : Protocol::Secure::Disable; priority = Priority{config.getInt(config_prefix + ".priority", 1)}; + proto_send_chunked = config.getString(config_prefix + ".proto_caps.send", "notchunked_optional"); + proto_recv_chunked = config.getString(config_prefix + ".proto_caps.recv", "notchunked_optional"); + const char * port_type = secure == Protocol::Secure::Enable ? "tcp_port_secure" : "tcp_port"; auto default_port = config.getInt(port_type, 0); @@ -425,7 +428,9 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, auto pool = ConnectionPoolFactory::instance().get( static_cast(settings.distributed_connections_pool_size), address.host_name, address.port, - address.default_database, address.user, address.password, address.quota_key, + address.default_database, address.user, address.password, + address.proto_send_chunked, address.proto_recv_chunked, + address.quota_key, address.cluster, address.cluster_secret, "server", address.compression, address.secure, address.priority); @@ -589,6 +594,8 @@ void Cluster::addShard( replica.default_database, replica.user, replica.password, + replica.proto_send_chunked, + replica.proto_recv_chunked, replica.quota_key, replica.cluster, replica.cluster_secret, @@ -744,6 +751,8 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti address.default_database, address.user, address.password, + address.proto_send_chunked, + address.proto_recv_chunked, address.quota_key, address.cluster, address.cluster_secret, diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index dc5790ac339..c993af5fc5e 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -114,6 +114,8 @@ public: UInt16 port{0}; String user; String password; + String proto_send_chunked; + String proto_recv_chunked; String quota_key; /// For inter-server authorization diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 2071eac3a68..c7db25c4c3a 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1,6 +1,7 @@ #include "Interpreters/AsynchronousInsertQueue.h" #include "Interpreters/SquashingTransform.h" #include "Parsers/ASTInsertQuery.h" +#include #include #include #include @@ -99,6 +100,7 @@ namespace DB::ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int UNSUPPORTED_METHOD; extern const int USER_EXPIRED; + extern const int NETWORK_ERROR; } namespace @@ -279,8 +281,35 @@ void TCPHandler::runImpl() if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) { - in->enableChunked(); - out->enableChunked(); + auto is_chunked = [](const String & chunked_srv_str, const String & chunked_cl_str, const String & direction) + { + bool chunked_srv = chunked_srv_str.starts_with("chunked"); + bool optional_srv = chunked_srv_str.ends_with("_optional"); + bool chunked_cl = chunked_cl_str.starts_with("chunked"); + bool optional_cl = chunked_cl_str.ends_with("_optional"); + + if (optional_srv) + return chunked_cl; + if (optional_cl) + return chunked_srv; + if (chunked_cl != chunked_srv) + throw NetException( + ErrorCodes::NETWORK_ERROR, + "Incompatible protocol: {} is {}, client requested {}", + direction, + chunked_srv ? "chunked" : "notchunked", + chunked_cl ? "chunked" : "notchunked"); + + return chunked_srv; + }; + + bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "notchunked_optional"), proto_recv_chunked_cl, "send"); + bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "notchunked_optional"), proto_send_chunked_cl, "recv"); + + if (out_chunked) + out->enableChunked(); + if (in_chunked) + in->enableChunked(); } if (!is_interserver_mode) @@ -1575,6 +1604,12 @@ void TCPHandler::receiveAddendum() if (!is_interserver_mode) session->setQuotaClientKey(quota_key); + + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + readStringBinary(proto_send_chunked_cl, *in); + readStringBinary(proto_recv_chunked_cl, *in); + } } @@ -1608,6 +1643,11 @@ void TCPHandler::sendHello() writeStringBinary(server_display_name, *out); if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_VERSION_PATCH) writeVarUInt(VERSION_PATCH, *out); + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + writeStringBinary(server.config().getString("proto_caps.send", "notchunked_optional"), *out); + writeStringBinary(server.config().getString("proto_caps.recv", "notchunked_optional"), *out); + } if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES) { auto rules = server.context()->getAccessControl().getPasswordComplexityRules(); diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 67d77381167..baef92b9fa0 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -188,6 +188,8 @@ private: UInt64 client_version_minor = 0; UInt64 client_version_patch = 0; UInt32 client_tcp_protocol_version = 0; + String proto_send_chunked_cl; + String proto_recv_chunked_cl; String quota_key; /// Connection settings, which are extracted from a context. diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index d471c67553d..dd318f34148 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -273,6 +273,8 @@ ConnectionPoolWithFailoverPtr DistributedAsyncInsertDirectoryQueue::createPool(c address.default_database, address.user, address.password, + address.proto_send_chunked, + address.proto_recv_chunked, address.quota_key, address.cluster, address.cluster_secret, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 378b81c6d18..4475e265395 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5664,7 +5664,8 @@ std::optional StorageReplicatedMergeTree::distributedWriteFromClu { auto connection = std::make_shared( node.host_name, node.port, query_context->getGlobalContext()->getCurrentDatabase(), - node.user, node.password, SSHKey(), node.quota_key, node.cluster, node.cluster_secret, + node.user, node.password, node.proto_send_chunked, node.proto_recv_chunked, + SSHKey(), node.quota_key, node.cluster, node.cluster_secret, "ParallelInsertSelectInititiator", node.compression, node.secure From 147ad42df09f374df971d6bed36ccf67c97d87a9 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 24 May 2024 03:36:29 +0000 Subject: [PATCH 056/644] fix notchunked mode in ReadBufferFromPocoSocketChunked --- src/IO/ReadBufferFromPocoSocketChunked.cpp | 7 +++++++ src/IO/ReadBufferFromPocoSocketChunked.h | 2 +- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 4d40d8b4f14..a67a5bb41a9 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -44,6 +44,13 @@ void ReadBufferFromPocoSocketChunked::setAsyncCallback(AsyncCallback async_callb buffer_socket.setAsyncCallback(async_callback_); } +bool ReadBufferFromPocoSocketChunked::hasBufferedData() const +{ + if (chunked) + return hasPendingData() || buffer_socket.hasPendingData(); + return hasPendingData(); +} + bool ReadBufferFromPocoSocketChunked::startChunk() { if (buffer_socket.read(reinterpret_cast(&chunk_left), sizeof(chunk_left)) < sizeof(chunk_left)) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index c70363cf7d8..b0f5dd7dc5f 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -16,7 +16,7 @@ public: bool poll(size_t timeout_microseconds); void setAsyncCallback(AsyncCallback async_callback_); - bool hasBufferedData() const { return hasPendingData() || buffer_socket.hasPendingData(); } + bool hasBufferedData() const; Poco::Net::SocketAddress peerAddress() { return peer_address; } Poco::Net::SocketAddress ourAddress() { return our_address; } From 89205d78a68879399129b64f78cd27f7602bf373 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 31 May 2024 04:18:36 +0000 Subject: [PATCH 057/644] major refactoring --- src/Client/Connection.cpp | 2 +- src/IO/ReadBufferFromPocoSocket.cpp | 51 ++---- src/IO/ReadBufferFromPocoSocket.h | 20 ++- src/IO/ReadBufferFromPocoSocketChunked.cpp | 183 ++++++++++++--------- src/IO/ReadBufferFromPocoSocketChunked.h | 98 +++++++++-- 5 files changed, 222 insertions(+), 132 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 9327b694d29..c221124932a 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -1101,7 +1101,7 @@ bool Connection::poll(size_t timeout_microseconds) bool Connection::hasReadPendingData() const { - return last_input_packet_type.has_value() || in->hasBufferedData(); + return last_input_packet_type.has_value() || in->hasPendingData(); } diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 5fb7ea0440c..5c338ef18bc 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -32,9 +32,16 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -size_t ReadBufferFromPocoSocket::readSocket(Position begin, size_t size) +ssize_t ReadBufferFromPocoSocketBase::socketReceiveBytesImpl(char * ptr, size_t size) { ssize_t bytes_read = 0; + Stopwatch watch; + + SCOPE_EXIT({ + /// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one + ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds()); + ProfileEvents::increment(ProfileEvents::NetworkReceiveBytes, bytes_read); + }); /// Add more details to exceptions. try @@ -49,7 +56,7 @@ size_t ReadBufferFromPocoSocket::readSocket(Position begin, size_t size) socket.setBlocking(false); SCOPE_EXIT(socket.setBlocking(true)); bool secure = socket.secure(); - bytes_read = socket.impl()->receiveBytes(begin, static_cast(size)); + bytes_read = socket.impl()->receiveBytes(ptr, static_cast(size)); /// Check EAGAIN and ERR_SSL_WANT_READ/ERR_SSL_WANT_WRITE for secure socket (reading from secure socket can write too). while (bytes_read < 0 && (errno == EAGAIN || (secure && (checkSSLWantRead(bytes_read) || checkSSLWantWrite(bytes_read))))) @@ -61,12 +68,12 @@ size_t ReadBufferFromPocoSocket::readSocket(Position begin, size_t size) async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), AsyncEventTimeoutType::RECEIVE, socket_description, AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); /// Try to read again. - bytes_read = socket.impl()->receiveBytes(begin, static_cast(size)); + bytes_read = socket.impl()->receiveBytes(ptr, static_cast(size)); } } else { - bytes_read = socket.impl()->receiveBytes(begin, static_cast(size)); + bytes_read = socket.impl()->receiveBytes(ptr, static_cast(size)); } } catch (const Poco::Net::NetException & e) @@ -90,36 +97,12 @@ size_t ReadBufferFromPocoSocket::readSocket(Position begin, size_t size) return bytes_read; } -bool ReadBufferFromPocoSocket::readSocketExact(Position begin, size_t size) +bool ReadBufferFromPocoSocketBase::nextImpl() { - for (size_t bytes_left = size; bytes_left > 0;) - { - size_t ret = readSocket(begin + size - bytes_left, bytes_left); - if (ret == 0) - return false; - bytes_left -= ret; - } - - return true; -} - -bool ReadBufferFromPocoSocket::nextImpl() -{ - ssize_t bytes_read = 0; - Stopwatch watch; - - SCOPE_EXIT({ - /// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one - ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds()); - ProfileEvents::increment(ProfileEvents::NetworkReceiveBytes, bytes_read); - }); - - CurrentMetrics::Increment metric_increment(CurrentMetrics::NetworkReceive); - if (internal_buffer.size() > INT_MAX) throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); - bytes_read = readSocket(internal_buffer.begin(), internal_buffer.size()); + ssize_t bytes_read = socketReceiveBytesImpl(internal_buffer.begin(), internal_buffer.size()); if (read_event != ProfileEvents::end()) ProfileEvents::increment(read_event, bytes_read); @@ -132,7 +115,7 @@ bool ReadBufferFromPocoSocket::nextImpl() return true; } -ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size) +ReadBufferFromPocoSocketBase::ReadBufferFromPocoSocketBase(Poco::Net::Socket & socket_, size_t buf_size) : BufferWithOwnMemory(buf_size) , socket(socket_) , peer_address(socket.peerAddress()) @@ -141,13 +124,13 @@ ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, { } -ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size) - : ReadBufferFromPocoSocket(socket_, buf_size) +ReadBufferFromPocoSocketBase::ReadBufferFromPocoSocketBase(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size) + : ReadBufferFromPocoSocketBase(socket_, buf_size) { read_event = read_event_; } -bool ReadBufferFromPocoSocket::poll(size_t timeout_microseconds) const +bool ReadBufferFromPocoSocketBase::poll(size_t timeout_microseconds) const { if (available()) return true; diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index c40a54ed7ae..a36bea6d679 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -9,7 +9,7 @@ namespace DB { /// Works with the ready Poco::Net::Socket. Blocking operations. -class ReadBufferFromPocoSocket : public BufferWithOwnMemory +class ReadBufferFromPocoSocketBase : public BufferWithOwnMemory { protected: Poco::Net::Socket & socket; @@ -25,19 +25,29 @@ protected: bool nextImpl() override; public: - explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); - explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + explicit ReadBufferFromPocoSocketBase(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + explicit ReadBufferFromPocoSocketBase(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); bool poll(size_t timeout_microseconds) const; void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); } - size_t readSocket(Position begin, size_t size); - bool readSocketExact(Position begin, size_t size); + ssize_t socketReceiveBytesImpl(char * ptr, size_t size); private: AsyncCallback async_callback; std::string socket_description; }; +class ReadBufferFromPocoSocket : public ReadBufferFromPocoSocketBase +{ +public: + explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) + : ReadBufferFromPocoSocketBase(socket_, buf_size = DBMS_DEFAULT_BUFFER_SIZE) + {} + explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) + : ReadBufferFromPocoSocketBase(socket_, read_event_, buf_size) + {} +}; + } diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index a67a5bb41a9..3cc8710407e 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -16,126 +16,149 @@ ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Sock {} ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size) - : ReadBuffer(nullptr, 0), log(getLogger("Protocol")), peer_address(socket_.peerAddress()), our_address(socket_.address()), buffer_socket(socket_, read_event_, buf_size) + : ReadBufferFromPocoSocketBase(socket_, read_event_, buf_size), our_address(socket_.address()), log(getLogger("Protocol")) + { chassert(buf_size <= std::numeric_limits::max()); - - working_buffer = buffer_socket.buffer(); - pos = buffer_socket.position(); } void ReadBufferFromPocoSocketChunked::enableChunked() { - chunked = true; - buffer_socket.position() = pos; + if (chunked) + return; + chunked = 1; + data_end = buffer().end(); working_buffer.resize(offset()); + chunk_left = 0; + next_chunk = 0; } -bool ReadBufferFromPocoSocketChunked::poll(size_t timeout_microseconds) -{ - if (!chunked) - buffer_socket.position() = pos; - - return buffer_socket.poll(timeout_microseconds); -} - -void ReadBufferFromPocoSocketChunked::setAsyncCallback(AsyncCallback async_callback_) -{ - buffer_socket.setAsyncCallback(async_callback_); -} - -bool ReadBufferFromPocoSocketChunked::hasBufferedData() const +bool ReadBufferFromPocoSocketChunked::hasPendingData() const { if (chunked) - return hasPendingData() || buffer_socket.hasPendingData(); - return hasPendingData(); + return available() || static_cast(data_end - working_buffer.end()) > sizeof(next_chunk); + + return ReadBufferFromPocoSocketBase::hasPendingData(); } -bool ReadBufferFromPocoSocketChunked::startChunk() +bool ReadBufferFromPocoSocketChunked::poll(size_t timeout_microseconds) const { - if (buffer_socket.read(reinterpret_cast(&chunk_left), sizeof(chunk_left)) < sizeof(chunk_left)) - return false; - if (chunk_left == 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: empty chunk received"); + if (chunked) + if (available() || static_cast(data_end - working_buffer.end()) > sizeof(next_chunk)) + return true; - chunk_left = fromLittleEndian(chunk_left); - - return nextChunk(); + return ReadBufferFromPocoSocketBase::poll(timeout_microseconds); } -bool ReadBufferFromPocoSocketChunked::nextChunk() -{ - if (chunk_left == 0) - { - started = true; - return startChunk(); - } - if (buffer_socket.available() == 0) - if (!buffer_socket.next()) +bool ReadBufferFromPocoSocketChunked::load_next_chunk(Position c_pos, bool cont) +{ + auto buffered = std::min(static_cast(data_end - c_pos), sizeof(next_chunk)); + + if (buffered) + std::memcpy(&next_chunk, c_pos, buffered); + if (buffered < sizeof(next_chunk)) + if (socketReceiveBytesImpl(reinterpret_cast(&next_chunk) + buffered, sizeof(next_chunk) - buffered) < static_cast(sizeof(next_chunk) - buffered)) return false; - if (started) - LOG_TEST(log, "Packet receive started. Message {}, size {}", static_cast(*buffer_socket.position()), chunk_left); - else - LOG_TEST(log, "Packet receive continued. Size {}", chunk_left); + next_chunk = fromLittleEndian(next_chunk); - started = false; - - nextimpl_working_buffer_offset = buffer_socket.offset(); - - if (buffer_socket.available() < chunk_left) + if (next_chunk) { - working_buffer.resize(buffer_socket.offset() + buffer_socket.available()); - chunk_left -= buffer_socket.available(); - buffer_socket.position() += buffer_socket.available(); + if (cont) + LOG_TEST(log, "Packet receive continued. Size {}", next_chunk); + } + else + LOG_TEST(log, "Packet receive ended."); + + return true; +} + +bool ReadBufferFromPocoSocketChunked::process_chunk_left(Position c_pos) +{ + if (data_end - c_pos < chunk_left) + { + working_buffer.resize(data_end - buffer().begin()); + nextimpl_working_buffer_offset = c_pos - buffer().begin(); + chunk_left -= (data_end - c_pos); return true; } - working_buffer.resize(buffer_socket.offset() + chunk_left); - UInt8 buffered = std::min(static_cast(4), buffer_socket.available() - chunk_left); + nextimpl_working_buffer_offset = c_pos - buffer().begin(); + working_buffer.resize(nextimpl_working_buffer_offset + chunk_left); - buffer_socket.position() += chunk_left; - if (buffered > 0) - std::memcpy(&chunk_left, buffer_socket.position(), buffered); - buffer_socket.position() += buffered; + c_pos += chunk_left; - if (4 > buffered) - if (!buffer_socket.readSocketExact(reinterpret_cast(&chunk_left) + buffered, 4 - buffered)) - return false; - - chunk_left = fromLittleEndian(chunk_left); - - if (chunk_left == 0) - LOG_TEST(log, "Packet receive ended."); + if (!load_next_chunk(c_pos, true)) + return false; + chunk_left = 0; return true; } bool ReadBufferFromPocoSocketChunked::nextImpl() { - if (chunked) + if (!chunked) + return ReadBufferFromPocoSocketBase::nextImpl(); + + auto c_pos = pos; + + if (chunk_left == 0) { - if (!nextChunk()) + if (next_chunk == 0) { - pos = buffer_socket.position(); - return false; + if (chunked == 1) + chunked = 2; // first chunked block - no end marker + else + c_pos = pos + sizeof(next_chunk); // bypass chunk end marker + + if (c_pos > data_end) + c_pos = data_end; + + if (!load_next_chunk(c_pos)) + return false; + + chunk_left = next_chunk; + next_chunk = 0; + + c_pos += sizeof(next_chunk); + + if (c_pos >= data_end) + { + if (!ReadBufferFromPocoSocketBase::nextImpl()) + return false; + data_end = buffer().end(); + c_pos = buffer().begin(); + } + + LOG_TEST(log, "Packet receive started. Message {}, size {}", static_cast(*c_pos), chunk_left); + } + else + { + c_pos += sizeof(next_chunk); + if (c_pos >= data_end) + { + if (!ReadBufferFromPocoSocketBase::nextImpl()) + return false; + data_end = buffer().end(); + c_pos = buffer().begin(); + } + + chunk_left = next_chunk; + next_chunk = 0; } - return true; } - - buffer_socket.position() = pos; - - if (!buffer_socket.next()) + else { - pos = buffer_socket.position(); - return false; + chassert(c_pos == data_end); + + if (!ReadBufferFromPocoSocketBase::nextImpl()) + return false; + data_end = buffer().end(); + c_pos = buffer().begin(); } - pos = buffer_socket.position(); - working_buffer.resize(offset() + buffer_socket.available()); - - return true; + return process_chunk_left(c_pos); } } diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index b0f5dd7dc5f..851a90042ac 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -3,37 +3,111 @@ #include #include +/* + +Handshake +============= + | 'Hello' type + | handshake exchange + | chunked protocol negotiation + +============= + + +Basic chunk: + +============= +Chunk begins | 0x12345678 chunk size, 4 bytes little endian + +------------- + | Packet type always follows beginning of the chunk + | packet data + +------------- +Chunk ends | 0x00000000 4 zero bytes + +============= + + + + +Datastream chunk: + +============= +Chunk begins | 0x12345678 + +------------- + | Packet type + | packet data + +------------- + | Packet type + | packet data + +------------- +...arbitrary number ..... +of packets... ..... + +------------- + | Packet type + | packet data + +------------- +Chunk ends | 0x00000000 + +============= + + + +Multipart chunk: + +============= +Chunk begins | 0x12345678 chunk part size, 4 bytes little endian + +------------- + | Packet type + | packet data + +------------- + | Packet type + | (partial) packet data + +============= +Chunk continues | 0x12345678 chunk next part size, 4 bytes little endian + +============= + | possibly previous packet's data + +------------- + | Packet type + | packet data + +------------- +...arbitrary number ..... +of chunk parts... ..... + +------------- + | Packet type + | packet data + +------------- +Chunk ends | 0x00000000 + +============= + +*/ + namespace DB { -class ReadBufferFromPocoSocketChunked: public ReadBuffer +class ReadBufferFromPocoSocketChunked: public ReadBufferFromPocoSocketBase { public: + using ReadBufferFromPocoSocketBase::setAsyncCallback; + explicit ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); explicit ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); void enableChunked(); - bool poll(size_t timeout_microseconds); - void setAsyncCallback(AsyncCallback async_callback_); - bool hasBufferedData() const; + bool hasPendingData() const; + + bool poll(size_t timeout_microseconds) const; Poco::Net::SocketAddress peerAddress() { return peer_address; } Poco::Net::SocketAddress ourAddress() { return our_address; } protected: - bool startChunk(); - bool nextChunk(); + bool load_next_chunk(Position c_pos, bool cont = false); + bool process_chunk_left(Position c_pos); bool nextImpl() override; +protected: + Poco::Net::SocketAddress our_address; + private: LoggerPtr log; - Poco::Net::SocketAddress peer_address; - Poco::Net::SocketAddress our_address; - ReadBufferFromPocoSocket buffer_socket; - bool chunked = false; - UInt32 chunk_left = 0; // chunk left to read from socket - bool started = false; + Position data_end = nullptr; // end position of data in the internal_buffer + UInt32 chunk_left = 0; // chunk left to read from socket + UInt32 next_chunk = 0; // size of the next cnunk + UInt8 chunked = 0; // 0 - disabled; 1 - started; 2 - enabled; }; } From 4545f3af52d8046cd2a1b54fc22fd0d592a48a31 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 31 May 2024 04:35:01 +0000 Subject: [PATCH 058/644] fix --- src/IO/ReadBufferFromPocoSocketChunked.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 3cc8710407e..59c56b9d008 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -121,6 +121,9 @@ bool ReadBufferFromPocoSocketChunked::nextImpl() chunk_left = next_chunk; next_chunk = 0; + if (chunk_left == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: empty chunk received"); + c_pos += sizeof(next_chunk); if (c_pos >= data_end) From d1bc58f23254ca781b6645bafb9c7cdf00326a04 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 31 May 2024 05:05:18 +0000 Subject: [PATCH 059/644] fix --- src/IO/ReadBufferFromPocoSocket.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 5c338ef18bc..af58efc7e10 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -43,6 +43,8 @@ ssize_t ReadBufferFromPocoSocketBase::socketReceiveBytesImpl(char * ptr, size_t ProfileEvents::increment(ProfileEvents::NetworkReceiveBytes, bytes_read); }); + CurrentMetrics::Increment metric_increment(CurrentMetrics::NetworkReceive); + /// Add more details to exceptions. try { From 1dc381dbc1f0b7b53d8707b9515a0d3f6ad3f442 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 31 May 2024 05:07:40 +0000 Subject: [PATCH 060/644] fix --- src/IO/ReadBufferFromPocoSocketChunked.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index 851a90042ac..749ee042a7c 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -23,8 +23,6 @@ Chunk ends | 0x00000000 4 zero bytes +============= - - Datastream chunk: +============= Chunk begins | 0x12345678 @@ -45,7 +43,6 @@ Chunk ends | 0x00000000 +============= - Multipart chunk: +============= Chunk begins | 0x12345678 chunk part size, 4 bytes little endian From fdccba97a3c7d1097034bc6b0994b7f37bc5721e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 31 May 2024 06:35:04 +0000 Subject: [PATCH 061/644] set chunked for testing --- src/Client/ConnectionParameters.cpp | 4 ++-- src/Server/TCPHandler.cpp | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index 430c462084a..b6ed242acd4 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -103,8 +103,8 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati #endif } - proto_send_chunked = config.getString("proto_caps.send", "notchunked_optional"); - proto_recv_chunked = config.getString("proto_caps.recv", "notchunked_optional"); + proto_send_chunked = config.getString("proto_caps.send", "chunked"); + proto_recv_chunked = config.getString("proto_caps.recv", "chunked"); quota_key = config.getString("quota_key", ""); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index c7db25c4c3a..47e5f982a93 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -303,8 +303,8 @@ void TCPHandler::runImpl() return chunked_srv; }; - bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "notchunked_optional"), proto_recv_chunked_cl, "send"); - bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "notchunked_optional"), proto_send_chunked_cl, "recv"); + bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "chunked"), proto_recv_chunked_cl, "send"); + bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "chunked"), proto_send_chunked_cl, "recv"); if (out_chunked) out->enableChunked(); @@ -1645,8 +1645,8 @@ void TCPHandler::sendHello() writeVarUInt(VERSION_PATCH, *out); if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) { - writeStringBinary(server.config().getString("proto_caps.send", "notchunked_optional"), *out); - writeStringBinary(server.config().getString("proto_caps.recv", "notchunked_optional"), *out); + writeStringBinary(server.config().getString("proto_caps.send", "chunked"), *out); + writeStringBinary(server.config().getString("proto_caps.recv", "chunked"), *out); } if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES) { From e3d57ab117391c3b99a8937783320a8c59e0b196 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 31 May 2024 16:05:42 +0000 Subject: [PATCH 062/644] set default protocol to notchunked_optional for cluster clients --- src/Interpreters/Cluster.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index c993af5fc5e..f3146ac0134 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -114,8 +114,8 @@ public: UInt16 port{0}; String user; String password; - String proto_send_chunked; - String proto_recv_chunked; + String proto_send_chunked = "notchunked_optional"; + String proto_recv_chunked = "notchunked_optional"; String quota_key; /// For inter-server authorization From f11f41491087099c63ee9f98b6bf8a27a8e87ed9 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 2 Jun 2024 07:25:48 +0000 Subject: [PATCH 063/644] fix special case of testing feature for chunked protocol --- src/Server/TCPHandler.cpp | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 47e5f982a93..da276e1c404 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -2268,16 +2268,26 @@ void TCPHandler::sendData(const Block & block) } writeVarUInt(Protocol::Server::Data, *out); - /// Send external table name (empty name is the main table) - writeStringBinary("", *out); /// For testing hedged requests if (block.rows() > 0 && query_context->getSettingsRef().sleep_in_send_data_ms.totalMilliseconds()) { + /// This strange sequence is needed in case of chunked protocol is enabled, in order for client not to + /// hang on recieving of at least packet type - chunk will not be processed unless either chunk footer + /// or chunk continuation header is recieved - first 'next' is sending starting chunk containing packet type + /// and second 'next' is sending chunk continuation header. + out->next(); + /// Send external table name (empty name is the main table) + writeStringBinary("", *out); out->next(); std::chrono::milliseconds ms(query_context->getSettingsRef().sleep_in_send_data_ms.totalMilliseconds()); std::this_thread::sleep_for(ms); } + else + { + /// Send external table name (empty name is the main table) + writeStringBinary("", *out); + } state.block_out->write(block); state.maybe_compressed_out->next(); From eaeabd8d374e2e28a6208fb9ea1ea7835676c7e5 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 2 Jun 2024 13:03:48 +0000 Subject: [PATCH 064/644] fix typos --- src/Server/TCPHandler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index da276e1c404..1a64ec1dd10 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -2273,8 +2273,8 @@ void TCPHandler::sendData(const Block & block) if (block.rows() > 0 && query_context->getSettingsRef().sleep_in_send_data_ms.totalMilliseconds()) { /// This strange sequence is needed in case of chunked protocol is enabled, in order for client not to - /// hang on recieving of at least packet type - chunk will not be processed unless either chunk footer - /// or chunk continuation header is recieved - first 'next' is sending starting chunk containing packet type + /// hang on receiving of at least packet type - chunk will not be processed unless either chunk footer + /// or chunk continuation header is received - first 'next' is sending starting chunk containing packet type /// and second 'next' is sending chunk continuation header. out->next(); /// Send external table name (empty name is the main table) From e0be652f4de803198b406dcbda5b1f1ac6938a9c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 3 Jun 2024 07:24:28 +0000 Subject: [PATCH 065/644] fix test, better log, fix defaults for client --- src/Client/ConnectionParameters.cpp | 4 ++-- src/Client/ConnectionParameters.h | 4 ++-- src/IO/ReadBufferFromPocoSocketChunked.cpp | 6 +++--- src/IO/WriteBufferFromPocoSocketChunked.h | 6 +++--- tests/integration/test_hedged_requests/test.py | 2 +- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index b6ed242acd4..430c462084a 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -103,8 +103,8 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati #endif } - proto_send_chunked = config.getString("proto_caps.send", "chunked"); - proto_recv_chunked = config.getString("proto_caps.recv", "chunked"); + proto_send_chunked = config.getString("proto_caps.send", "notchunked_optional"); + proto_recv_chunked = config.getString("proto_caps.recv", "notchunked_optional"); quota_key = config.getString("quota_key", ""); diff --git a/src/Client/ConnectionParameters.h b/src/Client/ConnectionParameters.h index 85174924016..52fe7bd9b2b 100644 --- a/src/Client/ConnectionParameters.h +++ b/src/Client/ConnectionParameters.h @@ -20,8 +20,8 @@ struct ConnectionParameters std::string default_database; std::string user; std::string password; - std::string proto_send_chunked; - std::string proto_recv_chunked; + std::string proto_send_chunked = "notchunked_optional"; + std::string proto_recv_chunked = "notchunked_optional"; std::string quota_key; SSHKey ssh_private_key; Protocol::Secure security = Protocol::Secure::Disable; diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 59c56b9d008..328b70bdb9b 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -65,10 +65,10 @@ bool ReadBufferFromPocoSocketChunked::load_next_chunk(Position c_pos, bool cont) if (next_chunk) { if (cont) - LOG_TEST(log, "Packet receive continued. Size {}", next_chunk); + LOG_TEST(log, "{} <- {} Chunk receive continued. Size {}", ourAddress().toString(), peerAddress().toString(), next_chunk); } else - LOG_TEST(log, "Packet receive ended."); + LOG_TEST(log, "{} <- {} Chunk receive ended.", ourAddress().toString(), peerAddress().toString()); return true; } @@ -134,7 +134,7 @@ bool ReadBufferFromPocoSocketChunked::nextImpl() c_pos = buffer().begin(); } - LOG_TEST(log, "Packet receive started. Message {}, size {}", static_cast(*c_pos), chunk_left); + LOG_TEST(log, "{} <- {} Chunk receive started. Message {}, size {}", ourAddress().toString(), peerAddress().toString(), static_cast(*c_pos), chunk_left); } else { diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 6c35db62c0c..7c6ab53dc91 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -30,7 +30,7 @@ public: if (finished) throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: attempt to send empty chunk"); - LOG_TEST(log, "Packet send ended."); + LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); finished = true; UInt32 s = 0; @@ -43,9 +43,9 @@ protected: { UInt32 s = static_cast(offset()); if (finished) - LOG_TEST(log, "Packet send started. Message {}, size {}", static_cast(*buffer().begin()), s); + LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", ourAddress().toString(), peerAddress().toString(), static_cast(*buffer().begin()), s); else - LOG_TEST(log, "Packet send continued. Size {}", s); + LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), s); finished = false; s = toLittleEndian(s); diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 02ecf3c1367..0d72f7c45b1 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -333,7 +333,7 @@ def test_receive_timeout2(started_cluster): # in packet receiving but there are replicas in process of # connection establishing. update_configs( - node_1_sleep_in_send_data=4000, + node_1_sleep_in_send_data=5000, node_2_sleep_in_send_tables_status=2000, node_3_sleep_in_send_tables_status=2000, ) From 66e387562659e9712088e09427d4c050e9f22c1f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 3 Jun 2024 09:55:51 +0000 Subject: [PATCH 066/644] fix tidy build --- src/IO/ReadBufferFromPocoSocket.h | 2 +- src/IO/ReadBufferFromPocoSocketChunked.h | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index a36bea6d679..912388adaac 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -43,7 +43,7 @@ class ReadBufferFromPocoSocket : public ReadBufferFromPocoSocketBase { public: explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) - : ReadBufferFromPocoSocketBase(socket_, buf_size = DBMS_DEFAULT_BUFFER_SIZE) + : ReadBufferFromPocoSocketBase(socket_, buf_size) {} explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : ReadBufferFromPocoSocketBase(socket_, read_event_, buf_size) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index 749ee042a7c..acf0edafe0a 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -96,7 +96,6 @@ protected: bool process_chunk_left(Position c_pos); bool nextImpl() override; -protected: Poco::Net::SocketAddress our_address; private: From 1cda4596adfc9ca384a28da80a91159641952e36 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 3 Jun 2024 11:51:01 +0000 Subject: [PATCH 067/644] fix tidy build --- src/IO/ReadBufferFromPocoSocketChunked.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 328b70bdb9b..6ed6b63289c 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -101,7 +101,7 @@ bool ReadBufferFromPocoSocketChunked::nextImpl() if (!chunked) return ReadBufferFromPocoSocketBase::nextImpl(); - auto c_pos = pos; + auto * c_pos = pos; if (chunk_left == 0) { From 00ffb48924366de453ae6bd416d56be7d0a9568d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 4 Jun 2024 14:27:08 +0000 Subject: [PATCH 068/644] Handle table drop properly --- src/Storages/Kafka/KafkaConsumer2.cpp | 2 +- src/Storages/Kafka/KafkaConsumer2.h | 7 +- src/Storages/Kafka/StorageKafka2.cpp | 326 +++++++++++++++++++++----- src/Storages/Kafka/StorageKafka2.h | 13 +- 4 files changed, 281 insertions(+), 67 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index d2c2d7d8022..e32db78fb65 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -58,7 +58,7 @@ bool KafkaConsumer2::TopicPartition::operator<(const TopicPartition & other) con KafkaConsumer2::KafkaConsumer2( ConsumerPtr consumer_, - Poco::Logger * log_, + LoggerPtr log_, size_t max_batch_size, size_t poll_timeout_, const std::atomic & stopped_, diff --git a/src/Storages/Kafka/KafkaConsumer2.h b/src/Storages/Kafka/KafkaConsumer2.h index cde23ebf812..8eb21cf0364 100644 --- a/src/Storages/Kafka/KafkaConsumer2.h +++ b/src/Storages/Kafka/KafkaConsumer2.h @@ -10,8 +10,6 @@ #include #include -#include - namespace CurrentMetrics { extern const Metric KafkaConsumers; @@ -80,7 +78,7 @@ public: KafkaConsumer2( ConsumerPtr consumer_, - Poco::Logger * log_, + LoggerPtr log_, size_t max_batch_size, size_t poll_timeout_, const std::atomic & stopped_, @@ -110,7 +108,6 @@ public: /// Polls batch of messages from Kafka and returns read buffer containing the next message or /// nullptr when there are no messages to process. - /// TODO(antaljanosbenjamin): add batch size param ReadBufferPtr consume(const TopicPartition & topic_partition, const std::optional & message_count); // Return values for the message that's being read. @@ -136,7 +133,7 @@ private: }; ConsumerPtr consumer; - Poco::Logger * log; + LoggerPtr log; const size_t batch_size = 1; const size_t poll_timeout = 0; diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index ee038ae118b..b971ed4b42b 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -60,6 +60,7 @@ #include #include +#include #include namespace CurrentMetrics @@ -84,18 +85,22 @@ extern const Event KafkaWrites; namespace DB { +namespace fs = std::filesystem; + namespace ErrorCodes { extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; extern const int QUERY_NOT_ALLOWED; +extern const int REPLICA_ALREADY_EXISTS; +extern const int TABLE_IS_DROPPED; +extern const int TABLE_WAS_NOT_DROPPED; } namespace { constexpr auto MAX_FAILED_POLL_ATTEMPTS = 10; } -// TODO(antaljanosbenjamin): check performance StorageKafka2::StorageKafka2( const StorageID & table_id_, @@ -119,7 +124,7 @@ StorageKafka2::StorageKafka2( , max_rows_per_message(kafka_settings->kafka_max_rows_per_message.value) , schema_name(getContext()->getMacros()->expand(kafka_settings->kafka_schema.value, macros_info)) , num_consumers(kafka_settings->kafka_num_consumers.value) - , log(&Poco::Logger::get("StorageKafka (" + table_id_.table_name + ")")) + , log(getLogger("StorageKafka2 (" + table_id_.table_name + ")")) , semaphore(0, static_cast(num_consumers)) , settings_adjustments(createSettingsAdjustments()) , thread_per_consumer(kafka_settings->kafka_thread_per_consumer.value) @@ -145,6 +150,39 @@ StorageKafka2::StorageKafka2( task->deactivate(); tasks.emplace_back(std::make_shared(std::move(task))); } + + for (size_t i = 0; i < num_consumers; ++i) + { + try + { + consumers.push_back(ConsumerAndAssignmentInfo{.consumer = createConsumer(i), .keeper = keeper}); + ++num_created_consumers; + } + catch (const cppkafka::Exception &) + { + tryLogCurrentException(log); + } + } + for (auto try_count = 0; try_count < 5; ++try_count) + { + bool all_had_assignment = true; + for (auto & consumer_info : consumers) + { + if (nullptr == consumer_info.consumer->getKafkaAssignment()) + { + all_had_assignment = false; + consumer_info.consumer->pollEvents(); + } + } + + if (all_had_assignment) + break; + } + + const auto first_replica = createTableIfNotExists(consumers.front().consumer); + + if (!first_replica) + createReplica(); } VirtualColumnsDescription StorageKafka2::createVirtuals(StreamingHandleErrorMode handle_error_mode) @@ -257,31 +295,6 @@ StorageKafka2::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapsho void StorageKafka2::startup() { - for (size_t i = 0; i < num_consumers; ++i) - { - try - { - consumers.push_back(ConsumerAndAssignmentInfo{.consumer = createConsumer(i), .keeper = keeper}); - ++num_created_consumers; - } - catch (const cppkafka::Exception &) - { - tryLogCurrentException(log); - } - } - - try - { - createKeeperNodes(consumers.front().consumer); - } - catch (const Exception & ex) - { - if (ex.code() == ErrorCodes::LOGICAL_ERROR) - throw; - - tryLogCurrentException(log, __PRETTY_FUNCTION__); - } - // Start the reader thread for (auto & task : tasks) task->holder->activateAndSchedule(); @@ -306,7 +319,7 @@ void StorageKafka2::shutdown(bool) void StorageKafka2::drop() { - getZooKeeper()->removeRecursive(kafka_settings->kafka_keeper_path); + dropReplica(); } KafkaConsumer2Ptr StorageKafka2::createConsumer(size_t consumer_number) @@ -548,48 +561,244 @@ std::optional getNumber(zkutil::ZooKeeper & keeper, const std::string & } } -void StorageKafka2::createKeeperNodes(const KafkaConsumer2Ptr & consumer) +bool StorageKafka2::createTableIfNotExists(const KafkaConsumer2Ptr & consumer) { - // TODO(antaljanosbenjamin): check config with other StorageKafkas - // TODO(antaljanosbenjamin): maybe also create a node in `keeper_path/replicas/` to note that this replica has the table? - const auto & keeper_path = kafka_settings->kafka_keeper_path.value; + const auto & keeper_path = fs::path(kafka_settings->kafka_keeper_path.value); - if (keeper->exists(keeper_path)) - return; + const auto & replicas_path = keeper_path / "replicas"; - keeper->createAncestors(keeper_path); - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCreateRequest(keeper_path, "", zkutil::CreateMode::Persistent)); - - ops.emplace_back(zkutil::makeCreateRequest(keeper_path + "/topics", "", zkutil::CreateMode::Persistent)); - - const auto topics_prefix = keeper_path + "/topics/"; - - const auto topic_partition_counts = consumer->getPartitionCounts(); - for (const auto & topic_partition_count : topic_partition_counts) + for (auto i = 0; i < 1000; ++i) { - ops.emplace_back(zkutil::makeCreateRequest(topics_prefix + topic_partition_count.topic, "", zkutil::CreateMode::Persistent)); + if (keeper->exists(replicas_path)) + { + LOG_DEBUG(log, "This table {} is already created, will add new replica", String(keeper_path)); + return false; + } + + /// There are leftovers from incompletely dropped table. + if (keeper->exists(keeper_path / "dropped")) + { + /// This condition may happen when the previous drop attempt was not completed + /// or when table is dropped by another replica right now. + /// This is Ok because another replica is definitely going to drop the table. + + LOG_WARNING(log, "Removing leftovers from table {} (this might take several minutes)", String(keeper_path)); + String drop_lock_path = keeper_path / "dropped" / "lock"; + Coordination::Error code = keeper->tryCreate(drop_lock_path, "", zkutil::CreateMode::Ephemeral); + + if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) + { + LOG_WARNING(log, "The leftovers from table {} were removed by another replica", String(keeper_path)); + } + else if (code != Coordination::Error::ZOK) + { + throw Coordination::Exception::fromPath(code, drop_lock_path); + } + else + { + auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(drop_lock_path, *keeper); + if (!removeTableNodesFromZooKeeper(metadata_drop_lock)) + { + /// Someone is recursively removing table right now, we cannot create new table until old one is removed + continue; + } + } + } + + keeper->createAncestors(keeper_path); + Coordination::Requests ops; + + ops.emplace_back(zkutil::makeCreateRequest(keeper_path, "", zkutil::CreateMode::Persistent)); + + const auto topics_path = keeper_path / "topics"; + ops.emplace_back(zkutil::makeCreateRequest(topics_path, "", zkutil::CreateMode::Persistent)); + + + const auto topic_partition_counts = consumer->getPartitionCounts(); + for (const auto & topic_partition_count : topic_partition_counts) + { + LOG_DEBUG( + log, + "Creating path in keeper for topic {} with {} partitions", + topic_partition_count.topic, + topic_partition_count.partition_count); + ops.emplace_back(zkutil::makeCreateRequest(topics_path / topic_partition_count.topic, "", zkutil::CreateMode::Persistent)); + + const auto partitions_path = topics_path / topic_partition_count.topic / "partitions"; + ops.emplace_back(zkutil::makeCreateRequest(partitions_path, "", zkutil::CreateMode::Persistent)); + // TODO(antaljanosbenjamin): handle changing number of partitions + for (auto partition_id{0U}; partition_id < topic_partition_count.partition_count; ++partition_id) + ops.emplace_back(zkutil::makeCreateRequest(partitions_path / toString(partition_id), "", zkutil::CreateMode::Persistent)); + } + + // Create the first replica + ops.emplace_back(zkutil::makeCreateRequest(replicas_path, "", zkutil::CreateMode::Persistent)); ops.emplace_back( - zkutil::makeCreateRequest(topics_prefix + topic_partition_count.topic + "/partitions", "", zkutil::CreateMode::Persistent)); - const auto partitions_prefix = topics_prefix + topic_partition_count.topic + "/partitions/"; - // TODO(antaljanosbenjamin): handle changing number of partitions - for (auto partition_id{0U}; partition_id < topic_partition_count.partition_count; ++partition_id) - ops.emplace_back(zkutil::makeCreateRequest(partitions_prefix + toString(partition_id), "", zkutil::CreateMode::Persistent)); + zkutil::makeCreateRequest(replicas_path / kafka_settings->kafka_replica_name.value, "", zkutil::CreateMode::Persistent)); + + + Coordination::Responses responses; + const auto code = keeper->tryMulti(ops, responses); + if (code == Coordination::Error::ZNODEEXISTS) + { + LOG_INFO(log, "It looks like the table {} was created by another replica at the same moment, will retry", String(keeper_path)); + continue; + } + else if (code != Coordination::Error::ZOK) + { + zkutil::KeeperMultiException::check(code, ops, responses); + } + + LOG_INFO(log, "Table {} created successfully ", String(keeper_path)); + + return true; } + throw Exception( + ErrorCodes::REPLICA_ALREADY_EXISTS, + "Cannot create table, because it is created concurrently every time or because " + "of wrong zookeeper_path or because of logical error"); +} + +bool StorageKafka2::removeTableNodesFromZooKeeper(const zkutil::EphemeralNodeHolder::Ptr & drop_lock) +{ + bool completely_removed = false; + + Strings children; + if (const auto code = keeper->tryGetChildren(kafka_settings->kafka_keeper_path.value, children); code == Coordination::Error::ZNONODE) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is a race condition between creation and removal. It's a bug"); + + const auto keeper_path = fs::path(kafka_settings->kafka_keeper_path.value); + for (const auto & child : children) + if (child != "dropped") + keeper->tryRemoveRecursive(keeper_path / child); + + Coordination::Requests ops; Coordination::Responses responses; - const auto code = keeper->tryMulti(ops, responses); - if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNODEEXISTS) + ops.emplace_back(zkutil::makeRemoveRequest(drop_lock->getPath(), -1)); + ops.emplace_back(zkutil::makeRemoveRequest(keeper_path / "dropped", -1)); + ops.emplace_back(zkutil::makeRemoveRequest(keeper_path, -1)); + const auto code = keeper->tryMulti(ops, responses, /* check_session_valid */ true); + + if (code == Coordination::Error::ZNONODE) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, "There is a race condition between creation and removal of replicated table. It's a bug"); + } + else if (code == Coordination::Error::ZNOTEMPTY) + { + LOG_ERROR( + log, + "Table was not completely removed from Keeper, {} still exists and may contain some garbage," + "but someone is removing it right now.", + kafka_settings->kafka_keeper_path.value); + } + else if (code != Coordination::Error::ZOK) + { + /// It is still possible that ZooKeeper session is expired or server is killed in the middle of the delete operation. zkutil::KeeperMultiException::check(code, ops, responses); + } + else + { + drop_lock->setAlreadyRemoved(); + completely_removed = true; + LOG_INFO(log, "Table {} was successfully removed from ZooKeeper", kafka_settings->kafka_keeper_path.value); + } + + return completely_removed; +} + +void StorageKafka2::createReplica() +{ + const auto replica_path = kafka_settings->kafka_keeper_path.value + "/replicas/" + kafka_settings->kafka_replica_name.value; + const auto code = keeper->tryCreate(replica_path, "", zkutil::CreateMode::Persistent); + if (code == Coordination::Error::ZNODEEXISTS) + throw Exception(ErrorCodes::REPLICA_ALREADY_EXISTS, "Replica {} already exists", replica_path); + else if (code == Coordination::Error::ZNONODE) + throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} was suddenly removed", kafka_settings->kafka_keeper_path.value); + else if (code != Coordination::Error::ZOK) + throw Coordination::Exception::fromPath(code, replica_path); + + LOG_INFO(log, "Replica {} created", replica_path); +} + + +void StorageKafka2::dropReplica() +{ + if (keeper->expired()) + throw Exception(ErrorCodes::TABLE_WAS_NOT_DROPPED, "Table was not dropped because ZooKeeper session has expired."); + + auto replica_path = kafka_settings->kafka_keeper_path.value + "/replicas/" + kafka_settings->kafka_replica_name.value; + + LOG_INFO(log, "Removing replica {}", replica_path); + + if (!keeper->exists(replica_path)) + { + LOG_INFO(log, "Removing replica {} does not exist", replica_path); + return; + } + + { + keeper->tryRemoveChildrenRecursive(replica_path); + + if (keeper->tryRemove(replica_path) != Coordination::Error::ZOK) + LOG_ERROR(log, "Replica was not completely removed from Keeper, {} still exists and may contain some garbage.", replica_path); + } + + /// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line. + Strings replicas; + if (Coordination::Error::ZOK != keeper->tryGetChildren(kafka_settings->kafka_keeper_path.value + "/replicas", replicas) + || !replicas.empty()) + return; + + LOG_INFO(log, "{} is the last replica, will remove table", replica_path); + + /** At this moment, another replica can be created and we cannot remove the table. + * Try to remove /replicas node first. If we successfully removed it, + * it guarantees that we are the only replica that proceed to remove the table + * and no new replicas can be created after that moment (it requires the existence of /replicas node). + * and table cannot be recreated with new /replicas node on another servers while we are removing data, + * because table creation is executed in single transaction that will conflict with remaining nodes. + */ + + /// Node /dropped works like a lock that protects from concurrent removal of old table and creation of new table. + /// But recursive removal may fail in the middle of operation leaving some garbage in zookeeper_path, so + /// we remove it on table creation if there is /dropped node. Creating thread may remove /dropped node created by + /// removing thread, and it causes race condition if removing thread is not finished yet. + /// To avoid this we also create ephemeral child before starting recursive removal. + /// (The existence of child node does not allow to remove parent node). + Coordination::Requests ops; + Coordination::Responses responses; + String drop_lock_path = kafka_settings->kafka_keeper_path.value + "/dropped/lock"; + ops.emplace_back(zkutil::makeRemoveRequest(kafka_settings->kafka_keeper_path.value + "/replicas", -1)); + ops.emplace_back(zkutil::makeCreateRequest(kafka_settings->kafka_keeper_path.value + "/dropped", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(drop_lock_path, "", zkutil::CreateMode::Ephemeral)); + Coordination::Error code = keeper->tryMulti(ops, responses); + + if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) + { + LOG_WARNING(log, "Table {} is already started to be removing by another replica right now", replica_path); + } + else if (code == Coordination::Error::ZNOTEMPTY) + { + LOG_WARNING(log, "Another replica was suddenly created, will keep the table {}", replica_path); + } + else if (code != Coordination::Error::ZOK) + { + zkutil::KeeperMultiException::check(code, ops, responses); + } + else + { + auto drop_lock = zkutil::EphemeralNodeHolder::existing(drop_lock_path, *keeper); + LOG_INFO(log, "Removing table {} (this might take several minutes)", kafka_settings->kafka_keeper_path.value); + removeTableNodesFromZooKeeper(drop_lock); + } } std::optional StorageKafka2::lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const TopicPartitions & topic_partitions) { - // TODO(antaljanosbenjamin): Review this function with somebody who knows keeper better than me - const auto uuid_as_string = toString(uuid); - std::vector topic_partition_paths; topic_partition_paths.reserve(topic_partitions.size()); for (const auto & topic_partition : topic_partitions) @@ -598,8 +807,11 @@ StorageKafka2::lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const Topi Coordination::Requests ops; for (const auto & topic_partition_path : topic_partition_paths) - ops.push_back(zkutil::makeCreateRequest(topic_partition_path + lock_file_name, uuid_as_string, zkutil::CreateMode::Ephemeral)); - + { + LOG_TRACE(log, "Creating locking ops for: {}", topic_partition_path + lock_file_name); + ops.push_back(zkutil::makeCreateRequest( + topic_partition_path + lock_file_name, kafka_settings->kafka_replica_name.value, zkutil::CreateMode::Ephemeral)); + } Coordination::Responses responses; if (const auto code = keeper_to_use.tryMulti(ops, responses); code != Coordination::Error::ZOK) diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index 3cec473c746..c383357d79f 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -139,7 +138,7 @@ private: const size_t max_rows_per_message; const String schema_name; const size_t num_consumers; /// total number of consumers - Poco::Logger * log; + LoggerPtr log; Poco::Semaphore semaphore; const SettingsChanges settings_adjustments; std::atomic mv_attached = false; @@ -176,8 +175,6 @@ private: // Load Kafka properties from producer configuration void updateProducerConfiguration(cppkafka::Configuration & kafka_config); - UUID uuid{UUIDHelpers::generateV4()}; - String getConfigPrefix() const; void threadFunc(size_t idx); @@ -194,6 +191,14 @@ private: bool checkDependencies(const StorageID & table_id); + // Returns true if this is the first replica + bool createTableIfNotExists(const KafkaConsumer2Ptr & consumer); + // Returns true if all of the nodes were cleaned up + bool removeTableNodesFromZooKeeper(const zkutil::EphemeralNodeHolder::Ptr & drop_lock); + // Creates only the replica in ZooKeeper. Shouldn't be called on the first replica as it is created in createTableIfNotExists + void createReplica(); + void dropReplica(); + // Takes lock over topic partitions and set's the committed offset in topic_partitions void createKeeperNodes(const KafkaConsumer2Ptr & consumer); From e38e7d806c30ec5a295aa14fcadc53e2b9bc0ff2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 4 Jun 2024 14:28:38 +0000 Subject: [PATCH 069/644] Save offsets properly --- src/Storages/Kafka/StorageKafka2.cpp | 14 ++++++++++++-- src/Storages/Kafka/StorageKafka2.h | 5 +---- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index b971ed4b42b..aed27c27e62 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -851,16 +851,26 @@ StorageKafka2::lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const Topi } -void StorageKafka2::saveCommittedOffset(zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition, int64_t committed_offset) +void StorageKafka2::saveCommittedOffset( + zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition, const int64_t last_read_offset) { + const auto committed_offset = last_read_offset + 1; const auto partition_prefix = getTopicPartitionPath(topic_partition); keeper_to_use.createOrUpdate(partition_prefix + commit_file_name, toString(committed_offset), zkutil::CreateMode::Persistent); // This is best effort, if it fails we will try to remove in the next round keeper_to_use.tryRemove(partition_prefix + intent_file_name, -1); + LOG_TEST(log, "Saved offset {} for topic-partition [{}:{}]", committed_offset, topic_partition.topic, topic_partition.partition_id); } void StorageKafka2::saveIntent(zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition, int64_t intent) { + LOG_TEST( + log, + "Saving intent of {} for topic-partition [{}:{}] at offset {}", + intent, + topic_partition.topic, + topic_partition.partition_id, + topic_partition.offset); keeper_to_use.createOrUpdate( getTopicPartitionPath(topic_partition) + intent_file_name, toString(intent), zkutil::CreateMode::Persistent); } @@ -1189,7 +1199,7 @@ bool StorageKafka2::streamToViews(size_t idx) TopicPartition topic_partition_copy{topic_partition}; if (const auto & maybe_committed_offset = consumer_info.locks.at(topic_partition).committed_offset; maybe_committed_offset.has_value()) - topic_partition_copy.offset = *maybe_committed_offset + 1; + topic_partition_copy.offset = *maybe_committed_offset; else topic_partition_copy.offset = KafkaConsumer2::BEGINNING_OFFSET; diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index c383357d79f..c184053fc4d 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -99,7 +99,6 @@ private: KafkaConsumer2Ptr consumer; /// available consumers size_t consume_from_topic_partition_index{0}; TopicPartitions topic_partitions; - // TODO(antaljanosbenjamin): maybe recreate the ephemeral node zkutil::ZooKeeperPtr keeper; TopicPartitionLocks locks; }; @@ -200,10 +199,8 @@ private: void dropReplica(); // Takes lock over topic partitions and set's the committed offset in topic_partitions - void createKeeperNodes(const KafkaConsumer2Ptr & consumer); - std::optional lockTopicPartitions(zkutil::ZooKeeper& keeper_to_use, const TopicPartitions & topic_partitions); - void saveCommittedOffset(zkutil::ZooKeeper& keeper_to_use,const TopicPartition & topic_partition, int64_t committed_offset); + void saveCommittedOffset(zkutil::ZooKeeper& keeper_to_use,const TopicPartition & topic_partition, int64_t last_read_offset); void saveIntent(zkutil::ZooKeeper& keeper_to_use,const TopicPartition & topic_partition, int64_t intent); PolledBatchInfo pollConsumer( From b59cb914cd1a10e2c1753505674d44f05553d218 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 4 Jun 2024 14:40:16 +0000 Subject: [PATCH 070/644] Limit polls in `pollEvents` --- src/Storages/Kafka/KafkaConsumer2.cpp | 24 ++++++++++++++---------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index e32db78fb65..dabef7702cf 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -45,9 +45,9 @@ namespace ErrorCodes } using namespace std::chrono_literals; -const auto MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS = 15000; -const auto POLL_TIMEOUT_WO_ASSIGNMENT = 50ms; -const auto DRAIN_TIMEOUT_MS = 5000ms; +static constexpr auto MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS = 15000; +static constexpr auto EVENT_POLL_TIMEOUT = 50ms; +static constexpr auto DRAIN_TIMEOUT_MS = 5000ms; bool KafkaConsumer2::TopicPartition::operator<(const TopicPartition & other) const @@ -201,17 +201,21 @@ void KafkaConsumer2::drainConsumerQueue() void KafkaConsumer2::pollEvents() { - // POLL_TIMEOUT_WO_ASSIGNMENT_MS (50ms) is 100% enough just to check if we got assignment - // (see https://github.com/ClickHouse/ClickHouse/issues/11218) - auto msg = consumer->poll(POLL_TIMEOUT_WO_ASSIGNMENT); - + static constexpr int64_t max_tries = 5; + auto consumer_has_subscription = !consumer->get_subscription().empty(); + for(auto i = 0; i < max_tries && !consumer_has_subscription; ++i) + { + consumer->subscribe(topics); + consumer_has_subscription = !consumer->get_subscription().empty(); + } + auto msg = consumer->poll(EVENT_POLL_TIMEOUT); + LOG_TRACE(log, "Consumer has subscription: {}", consumer_has_subscription); // All the partition queues are detached, so the consumer shouldn't be able to poll any messages chassert(!msg && "Consumer returned a message when it was not expected"); auto consumer_queue = consumer->get_consumer_queue(); - // There should be events in the queue, so let's consume them all - while (consumer_queue.get_length() > 0) - consumer->poll(); + for(auto i = 0; i < max_tries && consumer_queue.get_length() > 0; ++i) + consumer->poll(EVENT_POLL_TIMEOUT); }; KafkaConsumer2::TopicPartitionCounts KafkaConsumer2::getPartitionCounts() const From 4a0a2d8ca17f4c2a477a5510e3ad09ec573e03b7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 4 Jun 2024 15:51:26 +0000 Subject: [PATCH 071/644] Save the offset to Kafka also to not duplicate messages when table is recreated --- src/Storages/Kafka/KafkaConsumer2.cpp | 66 +++++++++++++++++++++++++-- src/Storages/Kafka/KafkaConsumer2.h | 2 + src/Storages/Kafka/StorageKafka2.cpp | 20 ++++---- src/Storages/Kafka/StorageKafka2.h | 2 +- 4 files changed, 76 insertions(+), 14 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index dabef7702cf..7ae816f1a0f 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -95,7 +95,8 @@ KafkaConsumer2::KafkaConsumer2( needs_offset_update = true; for (const auto & topic_partition : topic_partitions) { - assignment->push_back(TopicPartition{topic_partition.get_topic(), topic_partition.get_partition(), INVALID_OFFSET}); + assignment->push_back( + TopicPartition{topic_partition.get_topic(), topic_partition.get_partition(), topic_partition.get_offset()}); } // We need to initialize the queues here in order to detach them from the consumer queue. Otherwise `pollEvents` might eventually poll actual messages also. @@ -376,6 +377,67 @@ ReadBufferPtr KafkaConsumer2::consume(const TopicPartition & topic_partition, co return getNextMessage(); } +void KafkaConsumer2::commit(const TopicPartition & topic_partition) +{ + static constexpr auto max_retries = 5; + bool committed = false; + + LOG_TEST( + log, + "Trying to commit offset {} to Kafka for topic-partition [{}:{}]", + topic_partition.offset, + topic_partition.topic, + topic_partition.partition_id); + + const auto topic_partition_list = std::vector{cppkafka::TopicPartition{ + topic_partition.topic, + topic_partition.partition_id, + topic_partition.offset, + }}; + for (auto try_count = 0; try_count < max_retries && !committed; ++try_count) + { + try + { + // See https://github.com/edenhill/librdkafka/issues/1470 + // broker may reject commit if during offsets.commit.timeout.ms (5000 by default), + // there were not enough replicas available for the __consumer_offsets topic. + // also some other temporary issues like client-server connectivity problems are possible + + consumer->commit(topic_partition_list); + committed = true; + LOG_INFO( + log, + "Committed offset {} to Kafka for topic-partition [{}:{}]", + topic_partition.offset, + topic_partition.topic, + topic_partition.partition_id); + } + catch (const cppkafka::HandleException & e) + { + // If there were actually no offsets to commit, return. Retrying won't solve + // anything here + if (e.get_error() == RD_KAFKA_RESP_ERR__NO_OFFSET) + committed = true; + else + LOG_ERROR(log, "Exception during commit attempt: {}", e.what()); + } + } + + if (!committed) + { + // The failure is not the biggest issue, it only counts when a table is dropped and recreated, otherwise the offsets are taken from keeper. + ProfileEvents::increment(ProfileEvents::KafkaCommitFailures); + LOG_INFO( + log, + "All commit attempts failed. Last block was already written to target table(s), " + "but was not committed to Kafka."); + } + else + { + ProfileEvents::increment(ProfileEvents::KafkaCommits); + } +} + ReadBufferPtr KafkaConsumer2::getNextMessage() { while (current != messages.end()) @@ -417,8 +479,6 @@ size_t KafkaConsumer2::filterMessageErrors() void KafkaConsumer2::resetIfStopped() { - // we can react on stop only during fetching data - // after block is formed (i.e. during copying data to MV / committing) we ignore stop attempts if (stopped) { stalled_status = StalledStatus::CONSUMER_STOPPED; diff --git a/src/Storages/Kafka/KafkaConsumer2.h b/src/Storages/Kafka/KafkaConsumer2.h index 8eb21cf0364..16d12c8723d 100644 --- a/src/Storages/Kafka/KafkaConsumer2.h +++ b/src/Storages/Kafka/KafkaConsumer2.h @@ -110,6 +110,8 @@ public: /// nullptr when there are no messages to process. ReadBufferPtr consume(const TopicPartition & topic_partition, const std::optional & message_count); + void commit(const TopicPartition& topic_partition); + // Return values for the message that's being read. String currentTopic() const { return current[-1].get_topic(); } String currentKey() const { return current[-1].get_key(); } diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index aed27c27e62..c5033be519f 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -852,14 +852,13 @@ StorageKafka2::lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const Topi void StorageKafka2::saveCommittedOffset( - zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition, const int64_t last_read_offset) + zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition) { - const auto committed_offset = last_read_offset + 1; const auto partition_prefix = getTopicPartitionPath(topic_partition); - keeper_to_use.createOrUpdate(partition_prefix + commit_file_name, toString(committed_offset), zkutil::CreateMode::Persistent); + keeper_to_use.createOrUpdate(partition_prefix + commit_file_name, toString(topic_partition.offset), zkutil::CreateMode::Persistent); // This is best effort, if it fails we will try to remove in the next round keeper_to_use.tryRemove(partition_prefix + intent_file_name, -1); - LOG_TEST(log, "Saved offset {} for topic-partition [{}:{}]", committed_offset, topic_partition.topic, topic_partition.partition_id); + LOG_TEST(log, "Saved offset {} for topic-partition [{}:{}]", topic_partition.offset, topic_partition.topic, topic_partition.partition_id); } void StorageKafka2::saveIntent(zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition, int64_t intent) @@ -1199,9 +1198,10 @@ bool StorageKafka2::streamToViews(size_t idx) TopicPartition topic_partition_copy{topic_partition}; if (const auto & maybe_committed_offset = consumer_info.locks.at(topic_partition).committed_offset; maybe_committed_offset.has_value()) + { topic_partition_copy.offset = *maybe_committed_offset; - else - topic_partition_copy.offset = KafkaConsumer2::BEGINNING_OFFSET; + } + // in case no saved offset, we will get the offset from Kafka as a best effort. This is important to not to duplicate message when recreating the table. consumer_info.topic_partitions.push_back(std::move(topic_partition_copy)); } @@ -1303,11 +1303,11 @@ std::optional StorageKafka2::streamFromConsumer(ConsumerAndAssignmentInf CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); } - - saveCommittedOffset(keeper_to_use, topic_partition, last_read_offset); - lock_info.intent_size.reset(); - lock_info.committed_offset = last_read_offset; + lock_info.committed_offset = last_read_offset + 1; topic_partition.offset = last_read_offset + 1; + consumer_info.consumer->commit(topic_partition); + saveCommittedOffset(keeper_to_use, topic_partition); + lock_info.intent_size.reset(); needs_offset_reset = false; return rows; diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index c184053fc4d..e8cfcac2689 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -200,7 +200,7 @@ private: // Takes lock over topic partitions and set's the committed offset in topic_partitions std::optional lockTopicPartitions(zkutil::ZooKeeper& keeper_to_use, const TopicPartitions & topic_partitions); - void saveCommittedOffset(zkutil::ZooKeeper& keeper_to_use,const TopicPartition & topic_partition, int64_t last_read_offset); + void saveCommittedOffset(zkutil::ZooKeeper& keeper_to_use,const TopicPartition & topic_partition); void saveIntent(zkutil::ZooKeeper& keeper_to_use,const TopicPartition & topic_partition, int64_t intent); PolledBatchInfo pollConsumer( From c0eea71ab387602ec0d760e21fc4954079552ae6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 4 Jun 2024 15:52:59 +0000 Subject: [PATCH 072/644] Make some tests work with the new storage kafka too --- tests/integration/test_storage_kafka/test.py | 1987 ++++++++++-------- 1 file changed, 1052 insertions(+), 935 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 8393e88db88..96438b5efa1 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -29,6 +29,7 @@ from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer, BrokerConnecti from kafka.protocol.admin import DescribeGroupsRequest_v1 from kafka.protocol.group import MemberAssignment from kafka.admin import NewTopic +from contextlib import contextmanager # protoc --version @@ -46,6 +47,11 @@ if is_arm(): # TODO: add test for run-time offset update in CH, if we manually update it on Kafka side. # TODO: add test for SELECT LIMIT is working. + +KAFKA_TOPIC_NEW = "new_t" +KAFKA_CONSUMER_GROUP_NEW = "new_cg" + + cluster = ClickHouseCluster(__file__) instance = cluster.add_instance( "instance", @@ -57,8 +63,8 @@ instance = cluster.add_instance( "kafka_broker": "kafka1", "kafka_topic_old": "old", "kafka_group_name_old": "old", - "kafka_topic_new": "new", - "kafka_group_name_new": "new", + "kafka_topic_new": KAFKA_TOPIC_NEW, + "kafka_group_name_new": KAFKA_CONSUMER_GROUP_NEW, "kafka_client_id": "instance", "kafka_format_json_each_row": "JSONEachRow", }, @@ -142,6 +148,22 @@ def kafka_delete_topic(admin_client, topic, max_retries=50): raise Exception(f"Failed to delete topics {topic}, {result}") +@contextmanager +def kafka_topic( + admin_client, + topic_name, + num_partitions=1, + replication_factor=1, + max_retries=50, + config=None +): + kafka_create_topic(admin_client, topic_name, num_partitions, replication_factor, max_retries, config) + try: + yield None + finally: + # Code to release resource, e.g.: + kafka_delete_topic(admin_client, topic_name, max_retries) + def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=15): logging.debug( "kafka_produce server:{}:{} topic:{}".format( @@ -283,11 +305,82 @@ def avro_confluent_message(schema_registry_client, value): ) return serializer.encode_record_with_schema("test_subject", schema, value) +def create_settings_string(settings): + if settings is None: + return "" + + def format_value(value): + if isinstance(value, str): + return f"'{value}'" + return str(value) + + settings_string = "SETTINGS " + keys = settings.keys() + first_key = next(iter(settings)) + settings_string += str(first_key) + " = " + format_value(settings[first_key]) + for key in keys: + if key == first_key: + continue + settings_string +=", " + str(key) + " = " + format_value(settings[key]) + return settings_string + + +def generate_old_create_table_query( + table_name, + columns_def, + database="test", + brokers="{kafka_broker}:19092", + topic_list="{kafka_topic_new}", + consumer_group="{kafka_group_name_new}", + format="{kafka_format_json_each_row}", + row_delimiter= "\\n", + keeper_path=None, # it is not used, but it is easier to handle keeper_path and replica_name like this + replica_name=None, + settings=None): + + settings_string=create_settings_string(settings) + query = f"""CREATE TABLE {database}.{table_name} ({columns_def}) ENGINE = Kafka('{brokers}', '{topic_list}', '{consumer_group}', '{format}', '{row_delimiter}') +{settings_string}""" + logging.debug(f"Generated old create query: {query}") + return query + +def generate_new_create_table_query( + table_name, + columns_def, + database="test", + brokers="{kafka_broker}:19092", + topic_list="{kafka_topic_new}", + consumer_group="{kafka_group_name_new}", + format="{kafka_format_json_each_row}", + row_delimiter= "\\n", + keeper_path=None, + replica_name=None, + settings=None): + if settings is None: + settings = {} + if keeper_path is None: + keeper_path = f"/clickhouse/{{database}}/{table_name}" + if replica_name is None: + replica_name = "r1" + settings["kafka_keeper_path"] = keeper_path + settings["kafka_replica_name"] = replica_name + settings_string=create_settings_string(settings) + query = f"""CREATE TABLE {database}.{table_name} ({columns_def}) ENGINE = Kafka('{brokers}', '{topic_list}', '{consumer_group}', '{format}', '{row_delimiter}') +{settings_string} +SETTINGS allow_experimental_kafka_store_offsets_in_keeper=1""" + logging.debug(f"Generated new create query: {query}") + return query + +def get_topic_postfix(generator): + if generator == generate_old_create_table_query: + return "old" + if generator == generate_new_create_table_query: + return "new" + raise "Unexpected generator" # Tests - - -def test_kafka_column_types(kafka_cluster): +@pytest.mark.parametrize('create_query_generator, do_direct_read', [(generate_old_create_table_query, True), (generate_new_create_table_query, False)]) +def test_kafka_column_types(kafka_cluster, create_query_generator, do_direct_read): def assert_returned_exception(e): assert e.value.returncode == 36 assert ( @@ -297,57 +390,14 @@ def test_kafka_column_types(kafka_cluster): # check column with DEFAULT expression with pytest.raises(QueryRuntimeException) as exception: - instance.query( - """ - CREATE TABLE test.kafka (a Int, b Int DEFAULT 0) - ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_new}', '{kafka_group_name_new}', '{kafka_format_json_each_row}', '\\n') - """ - ) + instance.query(create_query_generator('kafka', 'a Int, b Int DEFAULT 0')) assert_returned_exception(exception) # check EPHEMERAL with pytest.raises(QueryRuntimeException) as exception: - instance.query( - """ - CREATE TABLE test.kafka (a Int, b Int EPHEMERAL) - ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_new}', '{kafka_group_name_new}', '{kafka_format_json_each_row}', '\\n') - """ - ) + instance.query(create_query_generator('kafka', 'a Int, b Int EPHEMERAL')) assert_returned_exception(exception) - # check ALIAS - instance.query( - """ - CREATE TABLE test.kafka (a Int, b String Alias toString(a)) - ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_new}', '{kafka_group_name_new}', '{kafka_format_json_each_row}', '\\n') - SETTINGS kafka_commit_on_select = 1; - """ - ) - messages = [] - for i in range(5): - messages.append(json.dumps({"a": i})) - kafka_produce(kafka_cluster, "new", messages) - result = "" - expected = TSV( - """ -0\t0 -1\t1 -2\t2 -3\t3 -4\t4 - """ - ) - retries = 50 - while retries > 0: - result += instance.query("SELECT a, b FROM test.kafka", ignore_error=True) - if TSV(result) == expected: - break - retries -= 1 - - assert TSV(result) == expected - - instance.query("DROP TABLE test.kafka SYNC") - # check MATERIALIZED with pytest.raises(QueryRuntimeException) as exception: instance.query( @@ -358,6 +408,35 @@ def test_kafka_column_types(kafka_cluster): ) assert_returned_exception(exception) + if do_direct_read: + # check ALIAS + instance.query(create_query_generator("kafka", "a Int, b String Alias toString(a)", settings={"kafka_commit_on_select":1})) + messages = [] + for i in range(5): + messages.append(json.dumps({"a": i})) + kafka_produce(kafka_cluster, KAFKA_TOPIC_NEW, messages) + result = "" + expected = TSV( + """ + 0\t0 + 1\t1 + 2\t2 + 3\t3 + 4\t4 + """ + ) + retries = 50 + while retries > 0: + result += instance.query("SELECT a, b FROM test.kafka", ignore_error=True) + if TSV(result) == expected: + break + retries -= 1 + time.sleep(0.5) + + assert TSV(result) == expected + + instance.query("DROP TABLE test.kafka SYNC") + def test_kafka_settings_old_syntax(kafka_cluster): assert TSV( @@ -423,16 +502,16 @@ def test_kafka_settings_new_syntax(kafka_cluster): messages = [] for i in range(25): messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "new", messages) + kafka_produce(kafka_cluster, KAFKA_TOPIC_NEW, messages) # Insert couple of malformed messages. - kafka_produce(kafka_cluster, "new", ["}{very_broken_message,"]) - kafka_produce(kafka_cluster, "new", ["}another{very_broken_message,"]) + kafka_produce(kafka_cluster, KAFKA_TOPIC_NEW, ["}{very_broken_message,"]) + kafka_produce(kafka_cluster, KAFKA_TOPIC_NEW, ["}another{very_broken_message,"]) messages = [] for i in range(25, 50): messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "new", messages) + kafka_produce(kafka_cluster, KAFKA_TOPIC_NEW, messages) result = "" while True: @@ -519,8 +598,8 @@ def test_kafka_json_as_string(kafka_cluster): "Parsing of message (topic: kafka_json_as_string, partition: 0, offset: [0-9]*) return no rows" ) - -def test_kafka_formats(kafka_cluster): +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_formats(kafka_cluster, create_query_generator): schema_registry_client = CachedSchemaRegistryClient( "http://localhost:{}".format(kafka_cluster.schema_registry_port) ) @@ -649,7 +728,7 @@ def test_kafka_formats(kafka_cluster): '(id = 1, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 2, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 3, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 4, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 5, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 6, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 7, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 8, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 9, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 10, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 11, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 12, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 13, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 14, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 15, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)', '(id = 0, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)', ], - "extra_settings": ", format_template_row='template_row.format'", + "extra_settings": {"format_template_row":"template_row.format"}, }, "Regexp": { "data_sample": [ @@ -660,7 +739,7 @@ def test_kafka_formats(kafka_cluster): # On empty message exception happens: Line "" doesn't match the regexp.: (at row 1) # /src/Processors/Formats/Impl/RegexpRowInputFormat.cpp:140: DB::RegexpRowInputFormat::readRow(std::__1::vector::mutable_ptr, std::__1::allocator::mutable_ptr > >&, DB::RowReadExtension&) @ 0x1df82fcb in /usr/bin/clickhouse ], - "extra_settings": r", format_regexp='\(id = (.+?), blockNo = (.+?), val1 = \"(.+?)\", val2 = (.+?), val3 = (.+?)\)', format_regexp_escaping_rule='Escaped'", + "extra_settings": {"format_regexp":r"\(id = (.+?), blockNo = (.+?), val1 = \"(.+?)\", val2 = (.+?), val3 = (.+?)\)", "format_regexp_escaping_rule": "Escaped"}, }, ## BINARY FORMATS # dumped with @@ -732,7 +811,7 @@ def test_kafka_formats(kafka_cluster): # /src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp:25: DB::ProtobufRowInputFormat::readRow(std::__1::vector::mutable_ptr, std::__1::allocator::mutable_ptr > >&, DB::RowReadExtension&) @ 0x1df4cc71 in /usr/bin/clickhouse # /src/Processors/Formats/IRowInputFormat.cpp:64: DB::IRowInputFormat::generate() @ 0x1de727cf in /usr/bin/clickhouse ], - "extra_settings": ", kafka_schema='test:TestMessage'", + "extra_settings": {"kafka_schema":"test:TestMessage"}, }, "ORC": { "data_sample": [ @@ -756,7 +835,7 @@ def test_kafka_formats(kafka_cluster): # /src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp:212: DB::CapnProtoRowInputFormat::readMessage() @ 0x1ded1cab in /usr/bin/clickhouse # /src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp:241: DB::CapnProtoRowInputFormat::readRow(std::__1::vector::mutable_ptr, std::__1::allocator::mutable_ptr > >&, DB::RowReadExtension&) @ 0x1ded205d in /usr/bin/clickhouse ], - "extra_settings": ", kafka_schema='test:TestRecordStruct'", + "extra_settings": {"kafka_schema":"test:TestRecordStruct"}, }, "Parquet": { "data_sample": [ @@ -791,9 +870,9 @@ def test_kafka_formats(kafka_cluster): {"id": 0, "blockNo": 0, "val1": str("AM"), "val2": 0.5, "val3": 1}, ), ], - "extra_settings": ", format_avro_schema_registry_url='http://{}:{}'".format( + "extra_settings": {"format_avro_schema_registry_url":"http://{}:{}".format( kafka_cluster.schema_registry_host, kafka_cluster.schema_registry_port - ), + )}, "supports_empty_value": True, }, "Avro": { @@ -837,31 +916,25 @@ def test_kafka_formats(kafka_cluster): }, } + topic_postfix = str(hash(create_query_generator)) for format_name, format_opts in list(all_formats.items()): - logging.debug(("Set up {}".format(format_name))) - topic_name = "format_tests_{}".format(format_name) + logging.debug(f"Set up {format_name}") + topic_name = f"format_tests_{format_name}-{topic_postfix}" data_sample = format_opts["data_sample"] data_prefix = [] # prepend empty value when supported if format_opts.get("supports_empty_value", False): data_prefix = data_prefix + [""] kafka_produce(kafka_cluster, topic_name, data_prefix + data_sample) + + extra_settings = format_opts.get("extra_settings") or {} + extra_settings["kafka_flush_interval_ms"] = 1000 + instance.query( """ DROP TABLE IF EXISTS test.kafka_{format_name}; - CREATE TABLE test.kafka_{format_name} ( - id Int64, - blockNo UInt16, - val1 String, - val2 Float32, - val3 UInt8 - ) ENGINE = Kafka() - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic_name}', - kafka_group_name = '{topic_name}_group', - kafka_format = '{format_name}', - kafka_flush_interval_ms = 1000 {extra_settings}; + {create_query}; DROP TABLE IF EXISTS test.kafka_{format_name}_mv; @@ -870,7 +943,13 @@ def test_kafka_formats(kafka_cluster): """.format( topic_name=topic_name, format_name=format_name, - extra_settings=format_opts.get("extra_settings") or "", + create_query=create_query_generator( + f"kafka_{format_name}", + "id Int64, blockNo UInt16, val1 String, val2 Float32, val3 UInt8", + topic_list=f"{topic_name}", + consumer_group=f"{topic_name}_group", + format=format_name, + settings=extra_settings), ) ) raw_expected = """\ @@ -905,13 +984,13 @@ def test_kafka_formats(kafka_cluster): for format_name, format_opts in list(all_formats.items()): logging.debug(("Checking {}".format(format_name))) - topic_name = f"format_tests_{format_name}" + topic_name = f"format_tests_{format_name}-{topic_postfix}" # shift offsets by 1 if format supports empty value offsets = ( [1, 2, 3] if format_opts.get("supports_empty_value", False) else [0, 1, 2] ) - result = instance.query( - "SELECT * FROM test.kafka_{format_name}_mv;".format(format_name=format_name) + result = instance.query_with_retry( + "SELECT * FROM test.kafka_{format_name}_mv;".format(format_name=format_name), check_callback=lambda x: x.count('\n') == raw_expected.count('\n') ) expected = raw_expected.format( topic_name=topic_name, @@ -1591,134 +1670,132 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): """ assert TSV(result) == TSV(expected) - -def test_kafka_materialized_view(kafka_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'mv', - kafka_group_name = 'mv', - kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka; - """ - ) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "mv", messages) - - while True: - result = instance.query("SELECT * FROM test.view") - if kafka_check_result(result): - break - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) - - kafka_check_result(result, True) - - -def test_kafka_recreate_kafka_table(kafka_cluster): - """ - Checks that materialized view work properly after dropping and recreating the Kafka table. - """ - # line for backporting: - # admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_materialized_view(kafka_cluster, create_query_generator): + topic_name="mv" admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) + with kafka_topic(admin_client, topic_name): + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.kafka; + + {create_query_generator("kafka", "key UInt64, value UInt64", topic_list=topic_name, consumer_group="mv")}; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; + """ + ) + + messages = [] + for i in range(50): + messages.append(json.dumps({"key": i, "value": i})) + kafka_produce(kafka_cluster, topic_name, messages) + + result = instance.query_with_retry("SELECT * FROM test.view", check_callback=kafka_check_result) + + kafka_check_result(result, True) + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; + DROP TABLE test.kafka; + """ + ) + + +# TODO(antaljanosbenjamin): fails with the new, because it doesn't store the offsets... +@pytest.mark.parametrize('create_query_generator, thread_per_consumer, log_line', [ + (generate_new_create_table_query,1,r"kafka.*Saved offset [0-9]+ for topic-partition \[recreate_kafka_table:[0-9]+"), + (generate_old_create_table_query,0,"kafka.*Committed offset [0-9]+.*recreate_kafka_table"), +]) +def test_kafka_recreate_kafka_table(kafka_cluster, create_query_generator, thread_per_consumer, log_line): + """ + Checks that materialized view work properly after dropping and recreating the Kafka table. + """ + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) topic_name = "recreate_kafka_table" - kafka_create_topic(admin_client, topic_name, num_partitions=6) - instance.query( + with kafka_topic(admin_client, topic_name, num_partitions=6): + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group="recreate_kafka_table_group", + settings={ + "kafka_num_consumers": 4, + "kafka_flush_interval_ms": 1000, + "kafka_skip_broken_messages": 1048577, + "kafka_thread_per_consumer": thread_per_consumer, + }) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + {create_query}; + + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'recreate_kafka_table', - kafka_group_name = 'recreate_kafka_table_group', - kafka_format = 'JSONEachRow', - kafka_num_consumers = 6, - kafka_flush_interval_ms = 1000, - kafka_skip_broken_messages = 1048577; + ) - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka; - """ - ) + messages = [] + for i in range(120): + messages.append(json.dumps({"key": i, "value": i})) + kafka_produce(kafka_cluster, "recreate_kafka_table", messages) - messages = [] - for i in range(120): - messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "recreate_kafka_table", messages) + instance.wait_for_log_line( + log_line, + repetitions=6, + look_behind_lines=100, + ) - instance.wait_for_log_line( - "kafka.*Committed offset [0-9]+.*recreate_kafka_table", - repetitions=6, - look_behind_lines=100, - ) - - instance.query( + instance.query( + """ + DROP TABLE test.kafka; """ - DROP TABLE test.kafka; - """ - ) + ) - kafka_produce(kafka_cluster, "recreate_kafka_table", messages) + kafka_produce(kafka_cluster, "recreate_kafka_table", messages) - instance.query( + instance.query(create_query) + + instance.wait_for_log_line( + log_line, + repetitions=6, + look_behind_lines=100, + ) + + # data was not flushed yet (it will be flushed 7.5 sec after creating MV) + assert int(instance.query("SELECT count() FROM test.view")) == 240 + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.kafka; + DROP TABLE test.view; """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'recreate_kafka_table', - kafka_group_name = 'recreate_kafka_table_group', - kafka_format = 'JSONEachRow', - kafka_num_consumers = 6, - kafka_flush_interval_ms = 1000, - kafka_skip_broken_messages = 1048577; - """ - ) - - instance.wait_for_log_line( - "kafka.*Committed offset [0-9]+.*recreate_kafka_table", - repetitions=6, - look_behind_lines=100, - ) - - # data was not flushed yet (it will be flushed 7.5 sec after creating MV) - assert int(instance.query("SELECT count() FROM test.view")) == 240 - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) - kafka_delete_topic(admin_client, topic_name) + ) -def test_librdkafka_compression(kafka_cluster): +@pytest.mark.parametrize('create_query_generator, log_line', [ + (generate_old_create_table_query, "Committed offset {offset}"), + (generate_new_create_table_query, r"kafka.*Saved offset [0-9]+ for topic-partition \[{topic}:[0-9]+\]") +]) +def test_librdkafka_compression(kafka_cluster, create_query_generator, log_line): """ Regression for UB in snappy-c (that is used in librdkafka), backport pr is [1]. @@ -1754,139 +1831,144 @@ def test_librdkafka_compression(kafka_cluster): expected = "\n".join(expected) + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + for compression_type in supported_compression_types: logging.debug(("Check compression {}".format(compression_type))) topic_name = "test_librdkafka_compression_{}".format(compression_type) - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + topic_config = {"compression.type": compression_type} + with kafka_topic(admin_client, topic_name, config=topic_config): + instance.query( + """{create_query}; + + CREATE MATERIALIZED VIEW test.consumer Engine=Log AS + SELECT * FROM test.kafka; + """.format( + create_query=create_query_generator( + "kafka", + "key UInt64, value String", + topic_list=topic_name, + #brokers="kafka1:19092", + #consumer_group=f"{topic_name}_group", + format="JSONEachRow", + settings={"kafka_flush_interval_ms": 1000}), + ) + ) + + kafka_produce(kafka_cluster, topic_name, messages) + + instance.wait_for_log_line(log_line.format(offset=number_of_messages, topic=topic_name)) + + result = instance.query("SELECT * FROM test.consumer") + assert TSV(result) == TSV(expected) + + instance.query("DROP TABLE test.kafka SYNC") + instance.query("DROP TABLE test.consumer SYNC") + + +# TODO(antaljanosbenjamin): It fails with the new if the topic is not created explicitly +@pytest.mark.parametrize('create_query_generator', [generate_new_create_table_query, generate_old_create_table_query]) +def test_kafka_materialized_view_with_subquery(kafka_cluster, create_query_generator): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + topic_name = "mysq" + logging.debug(f"Using topic {topic_name}") + + with kafka_topic(admin_client, topic_name): + create_query = create_query_generator("kafka", "key UInt64, value UInt64", topic_list=topic_name, consumer_group=topic_name) + instance.query( + f""" + DROP TABLE IF EXISTS test.kafka; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + + {create_query}; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM (SELECT * FROM test.kafka); + """ ) - kafka_create_topic( - admin_client, topic_name, config={"compression.type": compression_type} - ) + messages = [] + for i in range(50): + messages.append(json.dumps({"key": i, "value": i})) + kafka_produce(kafka_cluster, topic_name, messages) + + result = instance.query_with_retry("SELECT * FROM test.view", check_callback=kafka_check_result) instance.query( """ - CREATE TABLE test.kafka (key UInt64, value String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic_name}', - kafka_group_name = '{topic_name}_group', - kafka_format = 'JSONEachRow', - kafka_flush_interval_ms = 1000; - CREATE MATERIALIZED VIEW test.consumer Engine=Log AS - SELECT * FROM test.kafka; - """.format( - topic_name=topic_name - ) + DROP TABLE test.consumer; + DROP TABLE test.view; + """ ) + kafka_check_result(result, True) + + +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_many_materialized_views(kafka_cluster, create_query_generator): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + topic_name = f"mmv-{get_topic_postfix(create_query_generator)}" + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=f"{topic_name}-group" + ) + with kafka_topic(admin_client, topic_name): + instance.query( + f""" + DROP TABLE IF EXISTS test.view1; + DROP TABLE IF EXISTS test.view2; + DROP TABLE IF EXISTS test.consumer1; + DROP TABLE IF EXISTS test.consumer2; + {create_query}; + CREATE TABLE test.view1 (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE TABLE test.view2 (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer1 TO test.view1 AS + SELECT * FROM test.kafka; + CREATE MATERIALIZED VIEW test.consumer2 TO test.view2 AS + SELECT * FROM test.kafka; + """ + ) + + messages = [] + for i in range(50): + messages.append(json.dumps({"key": i, "value": i})) kafka_produce(kafka_cluster, topic_name, messages) - instance.wait_for_log_line("Committed offset {}".format(number_of_messages)) + result1 = instance.query_with_retry("SELECT * FROM test.view1", check_callback=kafka_check_result) + result2 = instance.query_with_retry("SELECT * FROM test.view2", check_callback=kafka_check_result) - result = instance.query("SELECT * FROM test.consumer") - assert TSV(result) == TSV(expected) - - instance.query("DROP TABLE test.kafka SYNC") - instance.query("DROP TABLE test.consumer SYNC") - kafka_delete_topic(admin_client, topic_name) - - -def test_kafka_materialized_view_with_subquery(kafka_cluster): - instance.query( + instance.query( + """ + DROP TABLE test.consumer1; + DROP TABLE test.consumer2; + DROP TABLE test.view1; + DROP TABLE test.view2; """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'mvsq', - kafka_group_name = 'mvsq', - kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM (SELECT * FROM test.kafka); - """ - ) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "mvsq", messages) - - while True: - result = instance.query("SELECT * FROM test.view") - if kafka_check_result(result): - break - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) - - kafka_check_result(result, True) - - -def test_kafka_many_materialized_views(kafka_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.view1; - DROP TABLE IF EXISTS test.view2; - DROP TABLE IF EXISTS test.consumer1; - DROP TABLE IF EXISTS test.consumer2; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'mmv', - kafka_group_name = 'mmv', - kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\\n'; - CREATE TABLE test.view1 (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE TABLE test.view2 (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer1 TO test.view1 AS - SELECT * FROM test.kafka; - CREATE MATERIALIZED VIEW test.consumer2 TO test.view2 AS - SELECT * FROM test.kafka; - """ - ) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "mmv", messages) - - while True: - result1 = instance.query("SELECT * FROM test.view1") - result2 = instance.query("SELECT * FROM test.view2") - if kafka_check_result(result1) and kafka_check_result(result2): - break - - instance.query( - """ - DROP TABLE test.consumer1; - DROP TABLE test.consumer2; - DROP TABLE test.view1; - DROP TABLE test.view2; - """ - ) - - kafka_check_result(result1, True) - kafka_check_result(result2, True) + ) + kafka_check_result(result1, True) + kafka_check_result(result2, True) +# TODO(antaljanosbenjamin) def test_kafka_flush_on_big_message(kafka_cluster): - # Create batchs of messages of size ~100Kb + # Create batches of messages of size ~100Kb kafka_messages = 1000 batch_messages = 1000 messages = [ @@ -1989,7 +2071,8 @@ def test_kafka_virtual_columns(kafka_cluster): kafka_check_result(result, True, "test_kafka_virtual1.reference") -def test_kafka_virtual_columns_with_materialized_view(kafka_cluster): +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_virtual_columns_with_materialized_view(kafka_cluster, create_query_generator): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -1997,65 +2080,77 @@ def test_kafka_virtual_columns_with_materialized_view(kafka_cluster): # default retention, since predefined timestamp_ms is used. "retention.ms": "-1", } - kafka_create_topic(admin_client, "virt2", config=topic_config) - - instance.query( - """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'virt2', - kafka_group_name = 'virt2', - kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64, kafka_key String, topic String, offset UInt64, partition UInt64, timestamp Nullable(DateTime('UTC'))) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT *, _key as kafka_key, _topic as topic, _offset as offset, _partition as partition, _timestamp = 0 ? '0000-00-00 00:00:00' : toString(_timestamp) as timestamp FROM test.kafka; - """ + # the topic name is hardcoded in reference, it doesn't worth to create two reference files to have separate topics, + # as the context manager will always clean up the topic + topic_name = "virt2" + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=f"{topic_name}-group" ) + with kafka_topic(admin_client, topic_name, config=topic_config): - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "virt2", messages, 0) + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + {create_query}; + CREATE TABLE test.view (key UInt64, value UInt64, kafka_key String, topic String, offset UInt64, partition UInt64, timestamp Nullable(DateTime('UTC'))) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT *, _key as kafka_key, _topic as topic, _offset as offset, _partition as partition, _timestamp = 0 ? '0000-00-00 00:00:00' : toString(_timestamp) as timestamp FROM test.kafka; + """ + ) - sql = "SELECT kafka_key, key, topic, value, offset, partition, timestamp FROM test.view ORDER BY kafka_key, key" - result = instance.query(sql) - iterations = 0 - while ( - not kafka_check_result(result, False, "test_kafka_virtual2.reference") - and iterations < 10 - ): - time.sleep(3) - iterations += 1 - result = instance.query(sql) + messages = [] + for i in range(50): + messages.append(json.dumps({"key": i, "value": i})) + kafka_produce(kafka_cluster, topic_name, messages, 0) - kafka_check_result(result, True, "test_kafka_virtual2.reference") + def check_callback(result): + return kafka_check_result(result, False, "test_kafka_virtual2.reference") + result = instance.query_with_retry( + "SELECT kafka_key, key, topic, value, offset, partition, timestamp FROM test.view ORDER BY kafka_key, key", + check_callback=check_callback) + + kafka_check_result(result, True, "test_kafka_virtual2.reference") + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; + """ + ) + +def insert_with_retry(instance, values, table_name="kafka", max_try_couunt=5): + try_count = 0 + while True: + logging.debug(f"Inserting, try_count is {try_count}") + try: + try_count += 1 + instance.query("INSERT INTO test.kafka VALUES {}".format(values)) + break + except QueryRuntimeException as e: + if "Local: Timed out." in str(e) and try_count < max_try_couunt: + continue + else: + raise + + +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_insert(kafka_cluster, create_query_generator): + topic_name = "insert1" + get_topic_postfix(create_query_generator) instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) - - -def test_kafka_insert(kafka_cluster): - instance.query( - """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'insert1', - kafka_group_name = 'insert1', - kafka_format = 'TSV', - kafka_commit_on_select = 1, - kafka_row_delimiter = '\\n'; - """ + create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format="TSV" + ) ) values = [] @@ -2063,91 +2158,88 @@ def test_kafka_insert(kafka_cluster): values.append("({i}, {i})".format(i=i)) values = ",".join(values) - while True: - try: - instance.query("INSERT INTO test.kafka VALUES {}".format(values)) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise + insert_with_retry(instance, values) messages = [] - while True: - messages.extend(kafka_consume(kafka_cluster, "insert1")) + try_count = 0 + while True and try_count < 5: + try_count += 1 + messages.extend(kafka_consume(kafka_cluster, topic_name)) if len(messages) == 50: break + time.sleep(0.1) result = "\n".join(messages) kafka_check_result(result, True) -def test_kafka_produce_consume(kafka_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'insert2', - kafka_group_name = 'insert2', - kafka_format = 'TSV', - kafka_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka; - """ +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_produce_consume(kafka_cluster, create_query_generator): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) - messages_num = 10000 + topic_name = "insert2" + get_topic_postfix(create_query_generator) - def insert(): - values = [] - for i in range(messages_num): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - while True: - try: - instance.query("INSERT INTO test.kafka VALUES {}".format(values)) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - - threads = [] - threads_num = 16 - for _ in range(threads_num): - threads.append(threading.Thread(target=insert)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - while True: - result = instance.query("SELECT count() FROM test.view") - time.sleep(1) - if int(result) == messages_num * threads_num: - break - - instance.query( + with kafka_topic(admin_client, topic_name): + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format="TSV" + ) + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + {create_query}; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) + ) - for thread in threads: - thread.join() + messages_num = 10000 - assert ( - int(result) == messages_num * threads_num - ), "ClickHouse lost some messages: {}".format(result) + def insert(): + values = [] + for i in range(messages_num): + values.append("({i}, {i})".format(i=i)) + values = ",".join(values) + + insert_with_retry(instance, values) + + threads = [] + threads_num = 16 + for _ in range(threads_num): + threads.append(threading.Thread(target=insert)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + expected_row_count = messages_num * threads_num + result = instance.query_with_retry( + "SELECT count() FROM test.view", + sleep_time=1, + retry_count=20, + check_callback=lambda result: int(result) == expected_row_count) + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; + """ + ) + + for thread in threads: + thread.join() + + assert ( + int(result) == expected_row_count + ), "ClickHouse lost some messages: {}".format(result) def test_kafka_commit_on_block_write(kafka_cluster): @@ -2226,8 +2318,11 @@ def test_kafka_commit_on_block_write(kafka_cluster): assert result == 1, "Messages from kafka get duplicated!" - -def test_kafka_virtual_columns2(kafka_cluster): +@pytest.mark.parametrize('create_query_generator, thread_per_consumer, log_line', [ + (generate_old_create_table_query,0,"kafka.*Committed offset 2.*virt2_[01]"), + (generate_new_create_table_query,1,r"kafka.*Saved offset 2[0-9]* for topic-partition \[virt2_[01]:[0-9]+"), +]) +def test_kafka_virtual_columns2(kafka_cluster, create_query_generator, thread_per_consumer, log_line): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -2236,139 +2331,142 @@ def test_kafka_virtual_columns2(kafka_cluster): # default retention, since predefined timestamp_ms is used. "retention.ms": "-1", } - kafka_create_topic(admin_client, "virt2_0", num_partitions=2, config=topic_config) - kafka_create_topic(admin_client, "virt2_1", num_partitions=2, config=topic_config) + topic_name_0 = "virt2_0" + topic_name_1 = "virt2_1" + consumer_group = "virt2"+get_topic_postfix(create_query_generator) + with kafka_topic(admin_client, topic_name_0, num_partitions=2, config=topic_config): + with kafka_topic(admin_client, topic_name_1, num_partitions=2, config=topic_config): + create_query = create_query_generator( + "kafka", + "value UInt64", + topic_list=f"{topic_name_0},{topic_name_1}", + consumer_group=consumer_group, + settings={ + "kafka_num_consumers":2, + "kafka_thread_per_consumer": thread_per_consumer, + } + ) - instance.query( + instance.query( + f""" + {create_query}; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT value, _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp), toUnixTimestamp64Milli(_timestamp_ms), _headers.name, _headers.value FROM test.kafka; + """ + ) + + producer = KafkaProducer( + bootstrap_servers="localhost:{}".format(cluster.kafka_port), + value_serializer=producer_serializer, + key_serializer=producer_serializer, + ) + + producer.send( + topic=topic_name_0, + value=json.dumps({"value": 1}), + partition=0, + key="k1", + timestamp_ms=1577836801001, + headers=[("content-encoding", b"base64")], + ) + producer.send( + topic=topic_name_0, + value=json.dumps({"value": 2}), + partition=0, + key="k2", + timestamp_ms=1577836802002, + headers=[ + ("empty_value", b""), + ("", b"empty name"), + ("", b""), + ("repetition", b"1"), + ("repetition", b"2"), + ], + ) + producer.flush() + + producer.send( + topic=topic_name_0, + value=json.dumps({"value": 3}), + partition=1, + key="k3", + timestamp_ms=1577836803003, + headers=[("b", b"b"), ("a", b"a")], + ) + producer.send( + topic=topic_name_0, + value=json.dumps({"value": 4}), + partition=1, + key="k4", + timestamp_ms=1577836804004, + headers=[("a", b"a"), ("b", b"b")], + ) + producer.flush() + + producer.send( + topic=topic_name_1, + value=json.dumps({"value": 5}), + partition=0, + key="k5", + timestamp_ms=1577836805005, + ) + producer.send( + topic=topic_name_1, + value=json.dumps({"value": 6}), + partition=0, + key="k6", + timestamp_ms=1577836806006, + ) + producer.flush() + + producer.send( + topic=topic_name_1, + value=json.dumps({"value": 7}), + partition=1, + key="k7", + timestamp_ms=1577836807007, + ) + producer.send( + topic=topic_name_1, + value=json.dumps({"value": 8}), + partition=1, + key="k8", + timestamp_ms=1577836808008, + ) + producer.flush() + + instance.wait_for_log_line(log_line, repetitions=4, look_behind_lines=6000) + + members = describe_consumer_group(kafka_cluster, consumer_group) + # pprint.pprint(members) + # members[0]['client_id'] = 'ClickHouse-instance-test-kafka-0' + # members[1]['client_id'] = 'ClickHouse-instance-test-kafka-1' + + result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) + + expected = f"""\ + 1 k1 {topic_name_0} 0 0 1577836801 1577836801001 ['content-encoding'] ['base64'] + 2 k2 {topic_name_0} 0 1 1577836802 1577836802002 ['empty_value','','','repetition','repetition'] ['','empty name','','1','2'] + 3 k3 {topic_name_0} 1 0 1577836803 1577836803003 ['b','a'] ['b','a'] + 4 k4 {topic_name_0} 1 1 1577836804 1577836804004 ['a','b'] ['a','b'] + 5 k5 {topic_name_1} 0 0 1577836805 1577836805005 [] [] + 6 k6 {topic_name_1} 0 1 1577836806 1577836806006 [] [] + 7 k7 {topic_name_1} 1 0 1577836807 1577836807007 [] [] + 8 k8 {topic_name_1} 1 1 1577836808 1577836808008 [] [] """ - CREATE TABLE test.kafka (value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'virt2_0,virt2_1', - kafka_group_name = 'virt2', - kafka_num_consumers = 2, - kafka_format = 'JSONEachRow'; - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT value, _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp), toUnixTimestamp64Milli(_timestamp_ms), _headers.name, _headers.value FROM test.kafka; - """ - ) - - producer = KafkaProducer( - bootstrap_servers="localhost:{}".format(cluster.kafka_port), - value_serializer=producer_serializer, - key_serializer=producer_serializer, - ) - - producer.send( - topic="virt2_0", - value=json.dumps({"value": 1}), - partition=0, - key="k1", - timestamp_ms=1577836801001, - headers=[("content-encoding", b"base64")], - ) - producer.send( - topic="virt2_0", - value=json.dumps({"value": 2}), - partition=0, - key="k2", - timestamp_ms=1577836802002, - headers=[ - ("empty_value", b""), - ("", b"empty name"), - ("", b""), - ("repetition", b"1"), - ("repetition", b"2"), - ], - ) - producer.flush() - - producer.send( - topic="virt2_0", - value=json.dumps({"value": 3}), - partition=1, - key="k3", - timestamp_ms=1577836803003, - headers=[("b", b"b"), ("a", b"a")], - ) - producer.send( - topic="virt2_0", - value=json.dumps({"value": 4}), - partition=1, - key="k4", - timestamp_ms=1577836804004, - headers=[("a", b"a"), ("b", b"b")], - ) - producer.flush() - - producer.send( - topic="virt2_1", - value=json.dumps({"value": 5}), - partition=0, - key="k5", - timestamp_ms=1577836805005, - ) - producer.send( - topic="virt2_1", - value=json.dumps({"value": 6}), - partition=0, - key="k6", - timestamp_ms=1577836806006, - ) - producer.flush() - - producer.send( - topic="virt2_1", - value=json.dumps({"value": 7}), - partition=1, - key="k7", - timestamp_ms=1577836807007, - ) - producer.send( - topic="virt2_1", - value=json.dumps({"value": 8}), - partition=1, - key="k8", - timestamp_ms=1577836808008, - ) - producer.flush() - - instance.wait_for_log_line( - "kafka.*Committed offset 2.*virt2_[01]", repetitions=4, look_behind_lines=6000 - ) - - members = describe_consumer_group(kafka_cluster, "virt2") - # pprint.pprint(members) - # members[0]['client_id'] = 'ClickHouse-instance-test-kafka-0' - # members[1]['client_id'] = 'ClickHouse-instance-test-kafka-1' - - result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) - - expected = """\ -1 k1 virt2_0 0 0 1577836801 1577836801001 ['content-encoding'] ['base64'] -2 k2 virt2_0 0 1 1577836802 1577836802002 ['empty_value','','','repetition','repetition'] ['','empty name','','1','2'] -3 k3 virt2_0 1 0 1577836803 1577836803003 ['b','a'] ['b','a'] -4 k4 virt2_0 1 1 1577836804 1577836804004 ['a','b'] ['a','b'] -5 k5 virt2_1 0 0 1577836805 1577836805005 [] [] -6 k6 virt2_1 0 1 1577836806 1577836806006 [] [] -7 k7 virt2_1 1 0 1577836807 1577836807007 [] [] -8 k8 virt2_1 1 1 1577836808 1577836808008 [] [] -""" - - assert TSV(result) == TSV(expected) - - instance.query( - """ - DROP TABLE test.kafka; - DROP TABLE test.view; - """ - ) - kafka_delete_topic(admin_client, "virt2_0") - kafka_delete_topic(admin_client, "virt2_1") - instance.rotate_logs() + assert TSV(result) == TSV(expected) + instance.query( + """ + DROP TABLE test.kafka; + DROP TABLE test.view; + """ + ) + instance.rotate_logs() +# TODO(antaljanosbenjamin) def test_kafka_producer_consumer_separate_settings(kafka_cluster): instance.query( """ @@ -2446,7 +2544,12 @@ def test_kafka_producer_consumer_separate_settings(kafka_cluster): assert property_in_log in kafka_producer_applyed_properties -def test_kafka_produce_key_timestamp(kafka_cluster): +# TODO(antaljanosbenjamin) +@pytest.mark.parametrize('create_query_generator, log_line', [ + #(generate_new_create_table_query,"Saved offset 5"), + (generate_old_create_table_query, "Committed offset 5"), +]) +def test_kafka_produce_key_timestamp(kafka_cluster, create_query_generator, log_line): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -2456,74 +2559,75 @@ def test_kafka_produce_key_timestamp(kafka_cluster): # default retention, since predefined timestamp_ms is used. "retention.ms": "-1", } - kafka_create_topic(admin_client, topic_name, config=topic_config) - instance.query( + with kafka_topic(admin_client, topic_name, config=topic_config): + + writer_create_query = create_query_generator( + "kafka_writer", + "key UInt64, value UInt64, _key String, _timestamp DateTime('UTC')", + topic_list=topic_name, + consumer_group=topic_name, + format="TSV") + reader_create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64, inserted_key String, inserted_timestamp DateTime('UTC')", + topic_list=topic_name, + consumer_group=topic_name, + format="TSV") + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + {writer_create_query}; + {reader_create_query}; + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value, inserted_key, toUnixTimestamp(inserted_timestamp), _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp) FROM test.kafka; """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka_writer (key UInt64, value UInt64, _key String, _timestamp DateTime('UTC')) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'insert3', - kafka_group_name = 'insert3', - kafka_format = 'TSV', - kafka_row_delimiter = '\\n'; + ) - CREATE TABLE test.kafka (key UInt64, value UInt64, inserted_key String, inserted_timestamp DateTime('UTC')) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'insert3', - kafka_group_name = 'insert3', - kafka_format = 'TSV', - kafka_row_delimiter = '\\n'; + instance.query( + "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format( + 1, 1, "k1", 1577836801 + ) + ) + instance.query( + "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format( + 2, 2, "k2", 1577836802 + ) + ) + instance.query( + "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({})),({},{},'{}',toDateTime({}))".format( + 3, 3, "k3", 1577836803, 4, 4, "k4", 1577836804 + ) + ) + instance.query( + "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format( + 5, 5, "k5", 1577836805 + ) + ) - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value, inserted_key, toUnixTimestamp(inserted_timestamp), _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp) FROM test.kafka; + # instance.wait_for_log_line(log_line) + + expected = """\ + 1 1 k1 1577836801 k1 insert3 0 0 1577836801 + 2 2 k2 1577836802 k2 insert3 0 1 1577836802 + 3 3 k3 1577836803 k3 insert3 0 2 1577836803 + 4 4 k4 1577836804 k4 insert3 0 3 1577836804 + 5 5 k5 1577836805 k5 insert3 0 4 1577836805 """ - ) - instance.query( - "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format( - 1, 1, "k1", 1577836801 - ) - ) - instance.query( - "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format( - 2, 2, "k2", 1577836802 - ) - ) - instance.query( - "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({})),({},{},'{}',toDateTime({}))".format( - 3, 3, "k3", 1577836803, 4, 4, "k4", 1577836804 - ) - ) - instance.query( - "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format( - 5, 5, "k5", 1577836805 - ) - ) + result = instance.query_with_retry("SELECT * FROM test.view ORDER BY value", ignore_error=True, check_callback=lambda res: TSV(res) == TSV(expected)) - instance.wait_for_log_line("Committed offset 5") - - result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) - - # logging.debug(result) - - expected = """\ -1 1 k1 1577836801 k1 insert3 0 0 1577836801 -2 2 k2 1577836802 k2 insert3 0 1 1577836802 -3 3 k3 1577836803 k3 insert3 0 2 1577836803 -4 4 k4 1577836804 k4 insert3 0 3 1577836804 -5 5 k5 1577836805 k5 insert3 0 4 1577836805 -""" - - assert TSV(result) == TSV(expected) - - kafka_delete_topic(admin_client, topic_name) + # logging.debug(result) -def test_kafka_insert_avro(kafka_cluster): + + assert TSV(result) == TSV(expected) + + +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_insert_avro(kafka_cluster, create_query_generator): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -2531,49 +2635,51 @@ def test_kafka_insert_avro(kafka_cluster): # default retention, since predefined timestamp_ms is used. "retention.ms": "-1", } - kafka_create_topic(admin_client, "avro1", config=topic_config) - - instance.query( - """ - DROP TABLE IF EXISTS test.kafka; - CREATE TABLE test.kafka (key UInt64, value UInt64, _timestamp DateTime('UTC')) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'avro1', - kafka_group_name = 'avro1', - kafka_commit_on_select = 1, - kafka_format = 'Avro'; - """ - ) - - instance.query( - "INSERT INTO test.kafka select number*10 as key, number*100 as value, 1636505534 as _timestamp from numbers(4) SETTINGS output_format_avro_rows_in_file = 2, output_format_avro_codec = 'deflate'" - ) - - messages = [] - while True: - messages.extend( - kafka_consume( - kafka_cluster, "avro1", needDecode=False, timestamp=1636505534 - ) + topic_name="avro1" + get_topic_postfix(create_query_generator) + with kafka_topic(admin_client, topic_name, config=topic_config): + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64, _timestamp DateTime('UTC')", + topic_list=topic_name, + consumer_group=topic_name, + format="Avro", + ) + instance.query( + f""" + DROP TABLE IF EXISTS test.kafka; + {create_query} + """ ) - if len(messages) == 2: - break - result = "" - for a_message in messages: - result += decode_avro(a_message) + "\n" + instance.query( + "INSERT INTO test.kafka select number*10 as key, number*100 as value, 1636505534 as _timestamp from numbers(4) SETTINGS output_format_avro_rows_in_file = 2, output_format_avro_codec = 'deflate'" + ) - expected_result = """{'key': 0, 'value': 0, '_timestamp': 1636505534} + messages = [] + while True: + messages.extend( + kafka_consume( + kafka_cluster, topic_name, needDecode=False, timestamp=1636505534 + ) + ) + if len(messages) == 2: + break + + result = "" + for a_message in messages: + result += decode_avro(a_message) + "\n" + + expected_result = """{'key': 0, 'value': 0, '_timestamp': 1636505534} {'key': 10, 'value': 100, '_timestamp': 1636505534} {'key': 20, 'value': 200, '_timestamp': 1636505534} {'key': 30, 'value': 300, '_timestamp': 1636505534} """ - assert result == expected_result + assert result == expected_result +# TODO(antaljanosbenjamin) def test_kafka_produce_consume_avro(kafka_cluster): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) @@ -2633,194 +2739,207 @@ def test_kafka_produce_consume_avro(kafka_cluster): kafka_delete_topic(admin_client, topic_name) -def test_kafka_flush_by_time(kafka_cluster): +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_flush_by_time(kafka_cluster, create_query_generator): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) - topic_name = "flush_by_time" - kafka_create_topic(admin_client, topic_name) + topic_name = "flush_by_time" + get_topic_postfix(create_query_generator) - instance.query( - """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'flush_by_time', - kafka_group_name = 'flush_by_time', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 100, - kafka_row_delimiter = '\\n'; - - SELECT * FROM test.kafka; - - CREATE TABLE test.view (key UInt64, value UInt64, ts DateTime64(3) MATERIALIZED now64(3)) - ENGINE = MergeTree() - ORDER BY key; - """ - ) - - cancel = threading.Event() - - def produce(): - while not cancel.is_set(): - messages = [] - messages.append(json.dumps({"key": 0, "value": 0})) - kafka_produce(kafka_cluster, "flush_by_time", messages) - time.sleep(0.8) - - kafka_thread = threading.Thread(target=produce) - kafka_thread.start() - - instance.query( - """ - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka; - """ - ) - - time.sleep(18) - - result = instance.query("SELECT uniqExact(ts) = 2, count() >= 15 FROM test.view") - - cancel.set() - kafka_thread.join() - - # kafka_cluster.open_bash_shell('instance') - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) - - assert TSV(result) == TSV("1 1") - kafka_delete_topic(admin_client, topic_name) - - -def test_kafka_flush_by_block_size(kafka_cluster): - cancel = threading.Event() - - def produce(): - while not cancel.is_set(): - messages = [] - messages.append(json.dumps({"key": 0, "value": 0})) - kafka_produce(kafka_cluster, "flush_by_block_size", messages) - - kafka_thread = threading.Thread(target=produce) - kafka_thread.start() - - instance.query( - """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'flush_by_block_size', - kafka_group_name = 'flush_by_block_size', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 100, - kafka_poll_max_batch_size = 1, - kafka_flush_interval_ms = 120000, /* should not flush by time during test */ - kafka_row_delimiter = '\\n'; - - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka; - """ - ) - - # Wait for Kafka engine to consume this data - while 1 != int( - instance.query( - "SELECT count() FROM system.parts WHERE database = 'test' AND table = 'view' AND name = 'all_1_1_0'" + with kafka_topic(admin_client, topic_name): + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + settings={ + "kafka_max_block_size":100, + } ) - ): - time.sleep(0.5) + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; - cancel.set() - kafka_thread.join() + {create_query}; - # more flushes can happens during test, we need to check only result of first flush (part named all_1_1_0). - result = instance.query("SELECT count() FROM test.view WHERE _part='all_1_1_0'") - # logging.debug(result) - - instance.query( + CREATE TABLE test.view (key UInt64, value UInt64, ts DateTime64(3) MATERIALIZED now64(3)) + ENGINE = MergeTree() + ORDER BY key; """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ + ) + + cancel = threading.Event() + + def produce(): + while not cancel.is_set(): + messages = [json.dumps({"key": 0, "value": 0})] + kafka_produce(kafka_cluster, topic_name, messages) + time.sleep(0.8) + + kafka_thread = threading.Thread(target=produce) + kafka_thread.start() + + instance.query( + """ + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; + """ + ) + + time.sleep(18) + + result = instance.query("SELECT uniqExact(ts) = 2, count() >= 15 FROM test.view") + + cancel.set() + kafka_thread.join() + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; + """ + ) + + assert TSV(result) == TSV("1 1") + + +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_flush_by_block_size(kafka_cluster, create_query_generator): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) + topic_name = "flush_by_block_size" + get_topic_postfix(create_query_generator) - # 100 = first poll should return 100 messages (and rows) - # not waiting for stream_flush_interval_ms - assert ( - int(result) == 100 - ), "Messages from kafka should be flushed when block of size kafka_max_block_size is formed!" + cancel = threading.Event() + def produce(): + while not cancel.is_set(): + messages = [] + messages.append(json.dumps({"key": 0, "value": 0})) + kafka_produce(kafka_cluster, topic_name, messages) + + kafka_thread = threading.Thread(target=produce) + + with kafka_topic(admin_client, topic_name): + kafka_thread.start() + + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + settings={ + "kafka_max_block_size": 100, + "kafka_poll_max_batch_size": 1, + "kafka_flush_interval_ms": 120000, + } + ) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + + {create_query}; + + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; + """ + ) + + # Wait for Kafka engine to consume this data + while 1 != int( + instance.query( + "SELECT count() FROM system.parts WHERE database = 'test' AND table = 'view' AND name = 'all_1_1_0'" + ) + ): + time.sleep(0.5) + + cancel.set() + kafka_thread.join() + + # more flushes can happens during test, we need to check only result of first flush (part named all_1_1_0). + result = instance.query("SELECT count() FROM test.view WHERE _part='all_1_1_0'") + # logging.debug(result) + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; + """ + ) + + # 100 = first poll should return 100 messages (and rows) + # not waiting for stream_flush_interval_ms + assert ( + int(result) == 100 + ), "Messages from kafka should be flushed when block of size kafka_max_block_size is formed!" -def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster): +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster, create_query_generator): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) - topic_name = "topic_with_multiple_partitions2" - kafka_create_topic(admin_client, topic_name, num_partitions=10) - - instance.query( + topic_name = "topic_with_multiple_partitions2" + get_topic_postfix(create_query_generator) + with kafka_topic(admin_client, topic_name): + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + settings={ + "kafka_max_block_size": 211, + "kafka_flush_interval_ms": 500, + } + ) + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + {create_query}; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'topic_with_multiple_partitions2', - kafka_group_name = 'topic_with_multiple_partitions2', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 211, - kafka_flush_interval_ms = 500; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka; - """ - ) + ) - messages = [] - count = 0 - for dummy_msg in range(1000): - rows = [] - for dummy_row in range(random.randrange(3, 10)): - count = count + 1 - rows.append(json.dumps({"key": count, "value": count})) - messages.append("\n".join(rows)) - kafka_produce(kafka_cluster, "topic_with_multiple_partitions2", messages) + messages = [] + count = 0 + for dummy_msg in range(1000): + rows = [] + for dummy_row in range(random.randrange(3, 10)): + count = count + 1 + rows.append(json.dumps({"key": count, "value": count})) + messages.append("\n".join(rows)) + kafka_produce(kafka_cluster, topic_name, messages) - instance.wait_for_log_line("kafka.*Stalled", repetitions=5) + instance.wait_for_log_line("kafka.*Stalled", repetitions=5) - result = instance.query("SELECT count(), uniqExact(key), max(key) FROM test.view") - logging.debug(result) - assert TSV(result) == TSV("{0}\t{0}\t{0}".format(count)) + result = instance.query("SELECT count(), uniqExact(key), max(key) FROM test.view") + logging.debug(result) + assert TSV(result) == TSV("{0}\t{0}\t{0}".format(count)) - instance.query( + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) - kafka_delete_topic(admin_client, topic_name) + ) -def test_kafka_rebalance(kafka_cluster): +@pytest.mark.parametrize('create_query_generator, log_line', [ + (generate_old_create_table_query, "{}.*Polled offset [0-9]+"), + (generate_new_create_table_query, "{}.*Saved offset"), +]) +def test_kafka_rebalance(kafka_cluster, create_query_generator, log_line): NUMBER_OF_CONSURRENT_CONSUMERS = 11 instance.query( @@ -2841,151 +2960,149 @@ def test_kafka_rebalance(kafka_cluster): """ ) - # kafka_cluster.open_bash_shell('instance') - - # time.sleep(2) - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) - topic_name = "topic_with_multiple_partitions" - kafka_create_topic(admin_client, topic_name, num_partitions=11) + topic_name = "topic_with_multiple_partitions" + get_topic_postfix(create_query_generator) + table_name_prefix = "kafka_consumer" + keeper_path = f"/clickhouse/{{database}}/{table_name_prefix}" + with kafka_topic(admin_client, topic_name, num_partitions=11): - cancel = threading.Event() + cancel = threading.Event() - msg_index = [0] + msg_index = [0] - def produce(): - while not cancel.is_set(): - messages = [] - for _ in range(59): - messages.append( - json.dumps({"key": msg_index[0], "value": msg_index[0]}) - ) - msg_index[0] += 1 - kafka_produce(kafka_cluster, "topic_with_multiple_partitions", messages) + def produce(): + while not cancel.is_set(): + messages = [] + for _ in range(59): + messages.append( + json.dumps({"key": msg_index[0], "value": msg_index[0]}) + ) + msg_index[0] += 1 + kafka_produce(kafka_cluster, topic_name, messages) - kafka_thread = threading.Thread(target=produce) - kafka_thread.start() + kafka_thread = threading.Thread(target=produce) + kafka_thread.start() - for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): - table_name = "kafka_consumer{}".format(consumer_index) - logging.debug(("Setting up {}".format(table_name))) + for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): + table_name = f"{table_name_prefix}{consumer_index}" + replica_name = f"r{consumer_index}" + logging.debug(f"Setting up {consumer_index}") - instance.query( - """ - DROP TABLE IF EXISTS test.{0}; - DROP TABLE IF EXISTS test.{0}_mv; - CREATE TABLE test.{0} (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'topic_with_multiple_partitions', - kafka_group_name = 'rebalance_test_group', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 33, - kafka_flush_interval_ms = 500; - CREATE MATERIALIZED VIEW test.{0}_mv TO test.destination AS - SELECT - key, - value, - _topic, - _key, - _offset, - _partition, - _timestamp, - '{0}' as _consumed_by - FROM test.{0}; - """.format( - table_name + create_query = create_query_generator( + table_name, + "key UInt64, value UInt64", + topic_list=topic_name, + keeper_path=keeper_path, + replica_name=replica_name, + settings={ + "kafka_max_block_size": 33, + "kafka_flush_interval_ms": 500, + } ) - ) + instance.query( + f""" + DROP TABLE IF EXISTS test.{table_name}; + DROP TABLE IF EXISTS test.{table_name}_mv; + {create_query}; + CREATE MATERIALIZED VIEW test.{table_name}_mv TO test.destination AS + SELECT + key, + value, + _topic, + _key, + _offset, + _partition, + _timestamp, + '{table_name}' as _consumed_by + FROM test.{table_name}; + """ + ) + # kafka_cluster.open_bash_shell('instance') + # Waiting for test.kafka_consumerX to start consume ... + instance.wait_for_log_line(log_line.format(table_name)) + + cancel.set() + + # I leave last one working by intent (to finish consuming after all rebalances) + for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS - 1): + logging.debug(("Dropping test.kafka_consumer{}".format(consumer_index))) + instance.query( + "DROP TABLE IF EXISTS test.kafka_consumer{} SYNC".format(consumer_index) + ) + + # logging.debug(instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination')) # kafka_cluster.open_bash_shell('instance') - # Waiting for test.kafka_consumerX to start consume ... - instance.wait_for_log_line( - "kafka_consumer{}.*Polled offset [0-9]+".format(consumer_index) - ) - cancel.set() + while 1: + messages_consumed = int( + instance.query("SELECT uniqExact(key) FROM test.destination") + ) + if messages_consumed >= msg_index[0]: + break + time.sleep(1) + logging.debug( + ( + "Waiting for finishing consuming (have {}, should be {})".format( + messages_consumed, msg_index[0] + ) + ) + ) - # I leave last one working by intent (to finish consuming after all rebalances) - for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS - 1): - logging.debug(("Dropping test.kafka_consumer{}".format(consumer_index))) - instance.query( - "DROP TABLE IF EXISTS test.kafka_consumer{} SYNC".format(consumer_index) - ) - - # logging.debug(instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination')) - # kafka_cluster.open_bash_shell('instance') - - while 1: - messages_consumed = int( - instance.query("SELECT uniqExact(key) FROM test.destination") - ) - if messages_consumed >= msg_index[0]: - break - time.sleep(1) logging.debug( ( - "Waiting for finishing consuming (have {}, should be {})".format( - messages_consumed, msg_index[0] + instance.query( + "SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination" ) ) ) - logging.debug( - ( - instance.query( - "SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination" - ) + # Some queries to debug... + # SELECT * FROM test.destination where key in (SELECT key FROM test.destination group by key having count() <> 1) + # select number + 1 as key from numbers(4141) x left join test.destination using (key) where test.destination.key = 0; + # SELECT * FROM test.destination WHERE key between 2360 and 2370 order by key; + # select _partition from test.destination group by _partition having count() <> max(_offset) + 1; + # select toUInt64(0) as _partition, number + 1 as _offset from numbers(400) x left join test.destination using (_partition,_offset) where test.destination.key = 0 order by _offset; + # SELECT * FROM test.destination WHERE _partition = 0 and _offset between 220 and 240 order by _offset; + + # CREATE TABLE test.reference (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', + # kafka_topic_list = 'topic_with_multiple_partitions', + # kafka_group_name = 'rebalance_test_group_reference', + # kafka_format = 'JSONEachRow', + # kafka_max_block_size = 100000; + # + # CREATE MATERIALIZED VIEW test.reference_mv Engine=Log AS + # SELECT key, value, _topic,_key,_offset, _partition, _timestamp, 'reference' as _consumed_by + # FROM test.reference; + # + # select * from test.reference_mv left join test.destination using (key,_topic,_offset,_partition) where test.destination._consumed_by = ''; + + result = int( + instance.query("SELECT count() == uniqExact(key) FROM test.destination") ) - ) - # Some queries to debug... - # SELECT * FROM test.destination where key in (SELECT key FROM test.destination group by key having count() <> 1) - # select number + 1 as key from numbers(4141) x left join test.destination using (key) where test.destination.key = 0; - # SELECT * FROM test.destination WHERE key between 2360 and 2370 order by key; - # select _partition from test.destination group by _partition having count() <> max(_offset) + 1; - # select toUInt64(0) as _partition, number + 1 as _offset from numbers(400) x left join test.destination using (_partition,_offset) where test.destination.key = 0 order by _offset; - # SELECT * FROM test.destination WHERE _partition = 0 and _offset between 220 and 240 order by _offset; + for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): + logging.debug(("kafka_consumer{}".format(consumer_index))) + table_name = "kafka_consumer{}".format(consumer_index) + instance.query( + """ + DROP TABLE IF EXISTS test.{0}; + DROP TABLE IF EXISTS test.{0}_mv; + """.format( + table_name + ) + ) - # CREATE TABLE test.reference (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', - # kafka_topic_list = 'topic_with_multiple_partitions', - # kafka_group_name = 'rebalance_test_group_reference', - # kafka_format = 'JSONEachRow', - # kafka_max_block_size = 100000; - # - # CREATE MATERIALIZED VIEW test.reference_mv Engine=Log AS - # SELECT key, value, _topic,_key,_offset, _partition, _timestamp, 'reference' as _consumed_by - # FROM test.reference; - # - # select * from test.reference_mv left join test.destination using (key,_topic,_offset,_partition) where test.destination._consumed_by = ''; - - result = int( - instance.query("SELECT count() == uniqExact(key) FROM test.destination") - ) - - for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): - logging.debug(("kafka_consumer{}".format(consumer_index))) - table_name = "kafka_consumer{}".format(consumer_index) instance.query( """ - DROP TABLE IF EXISTS test.{0}; - DROP TABLE IF EXISTS test.{0}_mv; - """.format( - table_name - ) + DROP TABLE IF EXISTS test.destination; + """ ) - instance.query( - """ - DROP TABLE IF EXISTS test.destination; - """ - ) + kafka_thread.join() - kafka_thread.join() - - assert result == 1, "Messages from kafka get duplicated!" - kafka_delete_topic(admin_client, topic_name) + assert result == 1, "Messages from kafka get duplicated!" def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): From a562118d2a5b66955f44d393949eccb0e8c3b8b7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 7 Jun 2024 01:45:56 +0000 Subject: [PATCH 073/644] major refactoring of chunked write buffer - more buffering, some bugs fixed --- src/Client/Connection.cpp | 23 +++-- src/IO/ReadBufferFromPocoSocketChunked.cpp | 2 - src/IO/WriteBuffer.h | 8 +- src/IO/WriteBufferFromPocoSocketChunked.h | 114 +++++++++++++++++---- src/Server/TCPHandler.cpp | 38 +++---- 5 files changed, 134 insertions(+), 51 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index c221124932a..9f727b974ee 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -625,7 +625,7 @@ bool Connection::ping(const ConnectionTimeouts & timeouts) UInt64 pong = 0; writeVarUInt(Protocol::Client::Ping, *out); - out->finishPacket(); + out->finishChunk(); out->next(); if (in->eof()) @@ -675,7 +675,7 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time writeVarUInt(Protocol::Client::TablesStatusRequest, *out); request.write(*out, server_revision); - out->finishPacket(); + out->finishChunk(); out->next(); UInt64 response_type = 0; @@ -827,7 +827,7 @@ void Connection::sendQuery( block_profile_events_in.reset(); block_out.reset(); - out->finishPacket(); + out->finishChunk(); /// Send empty block which means end of data. if (!with_pending_data) @@ -845,7 +845,7 @@ void Connection::sendCancel() return; writeVarUInt(Protocol::Client::Cancel, *out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -871,9 +871,10 @@ void Connection::sendData(const Block & block, const String & name, bool scalar) size_t prev_bytes = out->count(); block_out->write(block); - maybe_compressed_out->next(); + if (maybe_compressed_out != out) + maybe_compressed_out->next(); if (!block) - out->finishPacket(); + out->finishChunk(); out->next(); if (throttler) @@ -884,7 +885,7 @@ void Connection::sendIgnoredPartUUIDs(const std::vector & uuids) { writeVarUInt(Protocol::Client::IgnoredPartUUIDs, *out); writeVectorBinary(uuids, *out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -894,7 +895,7 @@ void Connection::sendReadTaskResponse(const String & response) writeVarUInt(Protocol::Client::ReadTaskResponse, *out); writeVarUInt(DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION, *out); writeStringBinary(response, *out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -903,7 +904,7 @@ void Connection::sendMergeTreeReadTaskResponse(const ParallelReadResponse & resp { writeVarUInt(Protocol::Client::MergeTreeReadTaskResponse, *out); response.serialize(*out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -922,7 +923,7 @@ void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String else copyData(input, *out, size); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -951,7 +952,7 @@ void Connection::sendScalarsData(Scalars & data) sendData(elem.second, elem.first, true /* scalar */); } - out->finishPacket(); + out->finishChunk(); out_bytes = out->count() - out_bytes; maybe_compressed_out_bytes = maybe_compressed_out->count() - maybe_compressed_out_bytes; diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 6ed6b63289c..798be547e99 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -153,8 +153,6 @@ bool ReadBufferFromPocoSocketChunked::nextImpl() } else { - chassert(c_pos == data_end); - if (!ReadBufferFromPocoSocketBase::nextImpl()) return false; data_end = buffer().end(); diff --git a/src/IO/WriteBuffer.h b/src/IO/WriteBuffer.h index 1ceb938e454..bb3200d2e54 100644 --- a/src/IO/WriteBuffer.h +++ b/src/IO/WriteBuffer.h @@ -63,7 +63,8 @@ public: } bytes += bytes_in_buffer; - pos = working_buffer.begin(); + pos = working_buffer.begin() + nextimpl_working_buffer_offset; + nextimpl_working_buffer_offset = 0; } /// Calling finalize() in the destructor of derived classes is a bad practice. @@ -152,6 +153,11 @@ protected: bool finalized = false; + /// The number of bytes to preserve from the initial position of `working_buffer` + /// buffer. Apparently this is an additional out-parameter for nextImpl(), + /// not a real field. + size_t nextimpl_working_buffer_offset = 0; + private: /** Write the data in the buffer (from the beginning of the buffer to the current position). * Throw an exception if something is wrong. diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 7c6ab53dc91..3fe39487923 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -19,40 +19,114 @@ public: explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, buf_size), log(getLogger("Protocol")) {} explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, write_event_, buf_size), log(getLogger("Protocol")) {} - void enableChunked() { chunked = true; } - void finishPacket() + void enableChunked() + { + chunked = true; + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(pos); + pos += std::min(available(), sizeof(*chunk_size_ptr)); + } + + void finishChunk() { if (!chunked) return; - next(); - - if (finished) + if (pos <= reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: attempt to send empty chunk"); - LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); - finished = true; + /// Fill up current chunk size + *chunk_size_ptr = toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr))); - UInt32 s = 0; - socketSendBytes(reinterpret_cast(&s), sizeof(s)); + if (!chunk_started) + LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", + ourAddress().toString(), peerAddress().toString(), + static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), + *chunk_size_ptr); + else + chunk_started = false; + + LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); + + if (available() < sizeof(*chunk_size_ptr)) + { + finishing = available(); + pos += available(); + chunk_size_ptr = reinterpret_cast(pos); + return; + } + + /// Buffer end-of-chunk + *reinterpret_cast(pos) = 0; + pos += sizeof(*chunk_size_ptr); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(pos); + pos += std::min(available(), sizeof(*chunk_size_ptr)); } + protected: void nextImpl() override { - if (chunked) - { - UInt32 s = static_cast(offset()); - if (finished) - LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", ourAddress().toString(), peerAddress().toString(), static_cast(*buffer().begin()), s); - else - LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), s); + if (!chunked) + return WriteBufferFromPocoSocket::nextImpl(); - finished = false; - s = toLittleEndian(s); + if (finishing < sizeof(*chunk_size_ptr)) + { + pos -= finishing; + /// Send current chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Send end-of-chunk directly + UInt32 s = 0; socketSendBytes(reinterpret_cast(&s), sizeof(s)); + + finishing = sizeof(*chunk_size_ptr); + + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + return; } + if (offset() == sizeof(*chunk_size_ptr)) // prevent sending empty chunk + { + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + return; + } + + if (working_buffer.end() - reinterpret_cast(chunk_size_ptr) <= static_cast(sizeof(*chunk_size_ptr))) + { + pos = reinterpret_cast(chunk_size_ptr); + /// Send current chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + return; + } + + if (pos - reinterpret_cast(chunk_size_ptr) == sizeof(*chunk_size_ptr)) + pos -= sizeof(*chunk_size_ptr); + else /// Fill up current chunk size + { + *chunk_size_ptr = toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr))); + if (!chunk_started) + { + chunk_started = true; + LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", + ourAddress().toString(), peerAddress().toString(), + static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), + *chunk_size_ptr); + } + else + LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), *chunk_size_ptr); + } + /// Send current chunk WriteBufferFromPocoSocket::nextImpl(); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); } Poco::Net::SocketAddress peerAddress() @@ -67,7 +141,9 @@ protected: private: LoggerPtr log; bool chunked = false; - bool finished = true; + bool chunk_started = false; // chunk started flag + UInt32 * chunk_size_ptr = nullptr; // pointer to the chunk size holder in the buffer + size_t finishing = sizeof(*chunk_size_ptr); // indicates not enough buffer for end-of-chunk marker }; } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 1a64ec1dd10..89ad8e856d5 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1188,7 +1188,7 @@ void TCPHandler::processTablesStatusRequest() response.write(*out, client_tcp_protocol_version); - out->finishPacket(); + out->finishChunk(); } void TCPHandler::receiveUnexpectedTablesStatusRequest() @@ -1210,7 +1210,7 @@ void TCPHandler::sendPartUUIDs() writeVarUInt(Protocol::Server::PartUUIDs, *out); writeVectorBinary(uuids, *out); - out->finishPacket(); + out->finishChunk(); out->next(); } } @@ -1220,7 +1220,7 @@ void TCPHandler::sendReadTaskRequestAssumeLocked() { writeVarUInt(Protocol::Server::ReadTaskRequest, *out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -1230,7 +1230,7 @@ void TCPHandler::sendMergeTreeAllRangesAnnouncementAssumeLocked(InitialAllRanges writeVarUInt(Protocol::Server::MergeTreeAllRangesAnnouncement, *out); announcement.serialize(*out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -1240,7 +1240,7 @@ void TCPHandler::sendMergeTreeReadTaskRequestAssumeLocked(ParallelReadRequest re writeVarUInt(Protocol::Server::MergeTreeReadTaskRequest, *out); request.serialize(*out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -1250,7 +1250,7 @@ void TCPHandler::sendProfileInfo(const ProfileInfo & info) writeVarUInt(Protocol::Server::ProfileInfo, *out); info.write(*out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -1267,7 +1267,7 @@ void TCPHandler::sendTotals(const Block & totals) state.block_out->write(totals); state.maybe_compressed_out->next(); - out->finishPacket(); + out->finishChunk(); out->next(); } } @@ -1285,7 +1285,7 @@ void TCPHandler::sendExtremes(const Block & extremes) state.block_out->write(extremes); state.maybe_compressed_out->next(); - out->finishPacket(); + out->finishChunk(); out->next(); } } @@ -1304,7 +1304,7 @@ void TCPHandler::sendProfileEvents() state.profile_events_block_out->write(block); - out->finishPacket(); + out->finishChunk(); out->next(); auto elapsed_milliseconds = stopwatch.elapsedMilliseconds(); @@ -1343,7 +1343,7 @@ void TCPHandler::sendTimezone() writeVarUInt(Protocol::Server::TimezoneUpdate, *out); writeStringBinary(tz, *out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -1700,7 +1700,7 @@ bool TCPHandler::receivePacket() case Protocol::Client::Ping: writeVarUInt(Protocol::Server::Pong, *out); - out->finishPacket(); + out->finishChunk(); out->next(); return false; @@ -2290,9 +2290,11 @@ void TCPHandler::sendData(const Block & block) } state.block_out->write(block); - state.maybe_compressed_out->next(); - out->finishPacket(); + if (state.maybe_compressed_out != out) + state.maybe_compressed_out->next(); + + out->finishChunk(); out->next(); } catch (...) @@ -2329,7 +2331,7 @@ void TCPHandler::sendLogData(const Block & block) state.logs_block_out->write(block); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -2341,7 +2343,7 @@ void TCPHandler::sendTableColumns(const ColumnsDescription & columns) writeStringBinary("", *out); writeStringBinary(columns.toString(), *out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -2352,7 +2354,7 @@ void TCPHandler::sendException(const Exception & e, bool with_stack_trace) writeVarUInt(Protocol::Server::Exception, *out); writeException(e, *out, with_stack_trace); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -2364,7 +2366,7 @@ void TCPHandler::sendEndOfStream() writeVarUInt(Protocol::Server::EndOfStream, *out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -2384,7 +2386,7 @@ void TCPHandler::sendProgress() state.prev_elapsed_ns = current_elapsed_ns; increment.write(*out, client_tcp_protocol_version); - out->finishPacket(); + out->finishChunk(); out->next(); } From 390a2a2488bdd20a87400ec3f5851dfde0f1bac0 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 7 Jun 2024 02:06:26 +0000 Subject: [PATCH 074/644] fix style --- src/IO/WriteBufferFromPocoSocketChunked.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 3fe39487923..9a9d53a1f30 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -142,7 +142,7 @@ private: LoggerPtr log; bool chunked = false; bool chunk_started = false; // chunk started flag - UInt32 * chunk_size_ptr = nullptr; // pointer to the chunk size holder in the buffer + UInt32 * chunk_size_ptr = nullptr; // pointer to the chunk size holder in the buffer size_t finishing = sizeof(*chunk_size_ptr); // indicates not enough buffer for end-of-chunk marker }; From 11d9f7d51b2cd658c495adb11c3b32f6fc5a8cc6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 7 Jun 2024 12:07:35 +0000 Subject: [PATCH 075/644] allow to set end-of-chunk marker on sent chunk, ignore duplicate finish chunk --- src/IO/WriteBufferFromPocoSocketChunked.h | 51 +++++++++++++++++++++-- 1 file changed, 47 insertions(+), 4 deletions(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 9a9d53a1f30..40a89416f84 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -1,5 +1,6 @@ #pragma once +#include "base/defines.h" #include #include #include @@ -33,7 +34,26 @@ public: return; if (pos <= reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: attempt to send empty chunk"); + { + if (chunk_size_ptr == last_finish_chunk) // prevent duplicate finish chunk + return; + + /// If current chunk is empty it means we are finishing a chunk previously sent by next(), + /// we want to convert current chunk header into end-of-chunk marker and initialize next chunk. + /// We don't need to wary about if it's the end of the buffer because next() always sends the whole buffer + /// so it should be a beginning of the buffer. + + chassert(reinterpret_cast(chunk_size_ptr) == working_buffer.begin()); + + *chunk_size_ptr = 0; + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(pos); + pos += std::min(available(), sizeof(*chunk_size_ptr)); + + last_finish_chunk = chunk_size_ptr; + + return; + } /// Fill up current chunk size *chunk_size_ptr = toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr))); @@ -62,6 +82,8 @@ public: /// Initialize next chunk chunk_size_ptr = reinterpret_cast(pos); pos += std::min(available(), sizeof(*chunk_size_ptr)); + + last_finish_chunk = chunk_size_ptr; } protected: @@ -70,6 +92,7 @@ protected: if (!chunked) return WriteBufferFromPocoSocket::nextImpl(); + /// next() after finishChunk ar the end of the buffer if (finishing < sizeof(*chunk_size_ptr)) { pos -= finishing; @@ -85,15 +108,34 @@ protected: chunk_size_ptr = reinterpret_cast(working_buffer.begin()); nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + last_finish_chunk = chunk_size_ptr; + return; } - if (offset() == sizeof(*chunk_size_ptr)) // prevent sending empty chunk + /// Send end-of-chunk buffered by finishChunk + if (offset() == 2 * sizeof(*chunk_size_ptr)) + { + pos -= sizeof(*chunk_size_ptr); + /// Send end-of-chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + last_finish_chunk = chunk_size_ptr; + + return; + } + + /// Prevent sending empty chunk + if (offset() == sizeof(*chunk_size_ptr)) { nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); return; } + /// Finish chunk at the end of the buffer if (working_buffer.end() - reinterpret_cast(chunk_size_ptr) <= static_cast(sizeof(*chunk_size_ptr))) { pos = reinterpret_cast(chunk_size_ptr); @@ -106,9 +148,9 @@ protected: return; } - if (pos - reinterpret_cast(chunk_size_ptr) == sizeof(*chunk_size_ptr)) + if (pos - reinterpret_cast(chunk_size_ptr) == sizeof(*chunk_size_ptr)) // next() after finishChunk pos -= sizeof(*chunk_size_ptr); - else /// Fill up current chunk size + else // fill up current chunk size { *chunk_size_ptr = toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr))); if (!chunk_started) @@ -141,6 +183,7 @@ protected: private: LoggerPtr log; bool chunked = false; + UInt32 * last_finish_chunk = nullptr; // pointer to the last chunk header created by finishChunk bool chunk_started = false; // chunk started flag UInt32 * chunk_size_ptr = nullptr; // pointer to the chunk size holder in the buffer size_t finishing = sizeof(*chunk_size_ptr); // indicates not enough buffer for end-of-chunk marker From d2dd640beb3ff917352135477e349fd1d379f38e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 7 Jun 2024 12:25:46 +0000 Subject: [PATCH 076/644] fix style --- src/IO/WriteBufferFromPocoSocketChunked.h | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 40a89416f84..d1ba492738e 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -9,11 +9,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - class WriteBufferFromPocoSocketChunked: public WriteBufferFromPocoSocket { public: @@ -37,7 +32,7 @@ public: { if (chunk_size_ptr == last_finish_chunk) // prevent duplicate finish chunk return; - + /// If current chunk is empty it means we are finishing a chunk previously sent by next(), /// we want to convert current chunk header into end-of-chunk marker and initialize next chunk. /// We don't need to wary about if it's the end of the buffer because next() always sends the whole buffer From 740501b36e58c08d3a6a52348c9b0411d0f5dd90 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 7 Jun 2024 18:23:37 +0000 Subject: [PATCH 077/644] some potential bug fixes --- src/IO/WriteBufferFromPocoSocketChunked.h | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index d1ba492738e..689389ba2ea 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -109,7 +109,7 @@ protected: } /// Send end-of-chunk buffered by finishChunk - if (offset() == 2 * sizeof(*chunk_size_ptr)) + if (offset() == 2 * sizeof(*chunk_size_ptr) && last_finish_chunk == chunk_size_ptr) { pos -= sizeof(*chunk_size_ptr); /// Send end-of-chunk @@ -140,6 +140,8 @@ protected: chunk_size_ptr = reinterpret_cast(working_buffer.begin()); nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + last_finish_chunk = nullptr; + return; } @@ -164,6 +166,8 @@ protected: /// Initialize next chunk chunk_size_ptr = reinterpret_cast(working_buffer.begin()); nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + last_finish_chunk = nullptr; } Poco::Net::SocketAddress peerAddress() From e304afe106a9f10c075588ee4b6e88aadaa30094 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 7 Jun 2024 20:22:31 +0000 Subject: [PATCH 078/644] Fix build --- src/Storages/Kafka/StorageKafkaCommon.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp index 5d0f5b3000b..adfe1086858 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.cpp +++ b/src/Storages/Kafka/StorageKafkaCommon.cpp @@ -2,6 +2,7 @@ #include +#include #include #include #include @@ -12,12 +13,12 @@ #include #include #include +#include #include #include -#include #include +#include #include -#include #include #include From 6802b7f82f593b5be3961c38c2dca581c882bd75 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 7 Jun 2024 20:23:39 +0000 Subject: [PATCH 079/644] Improve log messages --- src/Storages/Kafka/KafkaConsumer2.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 7ae816f1a0f..9ed698301e5 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -419,7 +419,7 @@ void KafkaConsumer2::commit(const TopicPartition & topic_partition) if (e.get_error() == RD_KAFKA_RESP_ERR__NO_OFFSET) committed = true; else - LOG_ERROR(log, "Exception during commit attempt: {}", e.what()); + LOG_WARNING(log, "Exception during commit attempt: {}", e.what()); } } @@ -429,8 +429,7 @@ void KafkaConsumer2::commit(const TopicPartition & topic_partition) ProfileEvents::increment(ProfileEvents::KafkaCommitFailures); LOG_INFO( log, - "All commit attempts failed. Last block was already written to target table(s), " - "but was not committed to Kafka."); + "All commit attempts failed"); } else { From e33273d577da4cce1e2f460c27de3d1d2c8c4f11 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 7 Jun 2024 20:28:10 +0000 Subject: [PATCH 080/644] Handle newly appearing partitions --- src/Storages/Kafka/StorageKafka2.cpp | 82 +++++++++++++--------------- src/Storages/Kafka/StorageKafka2.h | 5 +- 2 files changed, 42 insertions(+), 45 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index c5033be519f..fa5389d606a 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -163,23 +163,23 @@ StorageKafka2::StorageKafka2( tryLogCurrentException(log); } } - for (auto try_count = 0; try_count < 5; ++try_count) - { - bool all_had_assignment = true; - for (auto & consumer_info : consumers) - { - if (nullptr == consumer_info.consumer->getKafkaAssignment()) - { - all_had_assignment = false; - consumer_info.consumer->pollEvents(); - } - } + // for (auto try_count = 0; try_count < 5; ++try_count) + // { + // bool all_had_assignment = true; + // for (auto & consumer_info : consumers) + // { + // if (nullptr == consumer_info.consumer->getKafkaAssignment()) + // { + // all_had_assignment = false; + // consumer_info.consumer->pollEvents(); + // } + // } - if (all_had_assignment) - break; - } + // if (all_had_assignment) + // break; + // } - const auto first_replica = createTableIfNotExists(consumers.front().consumer); + const auto first_replica = createTableIfNotExists(); if (!first_replica) createReplica(); @@ -551,7 +551,7 @@ const std::string lock_file_name{"lock"}; const std::string commit_file_name{"committed"}; const std::string intent_file_name{"intention"}; -std::optional getNumber(zkutil::ZooKeeper & keeper, const std::string & path) +std::optional getNumber(zkutil::ZooKeeper & keeper, const fs::path & path) { std::string result; if (!keeper.tryGet(path, result)) @@ -561,7 +561,7 @@ std::optional getNumber(zkutil::ZooKeeper & keeper, const std::string & } } -bool StorageKafka2::createTableIfNotExists(const KafkaConsumer2Ptr & consumer) +bool StorageKafka2::createTableIfNotExists() { const auto & keeper_path = fs::path(kafka_settings->kafka_keeper_path.value); @@ -613,22 +613,15 @@ bool StorageKafka2::createTableIfNotExists(const KafkaConsumer2Ptr & consumer) const auto topics_path = keeper_path / "topics"; ops.emplace_back(zkutil::makeCreateRequest(topics_path, "", zkutil::CreateMode::Persistent)); - - const auto topic_partition_counts = consumer->getPartitionCounts(); - for (const auto & topic_partition_count : topic_partition_counts) + for (const auto & topic : topics) { - LOG_DEBUG( - log, - "Creating path in keeper for topic {} with {} partitions", - topic_partition_count.topic, - topic_partition_count.partition_count); - ops.emplace_back(zkutil::makeCreateRequest(topics_path / topic_partition_count.topic, "", zkutil::CreateMode::Persistent)); + LOG_DEBUG(log, "Creating path in keeper for topic {}", topic); - const auto partitions_path = topics_path / topic_partition_count.topic / "partitions"; + const auto topic_path = topics_path / topic; + ops.emplace_back(zkutil::makeCreateRequest(topic_path, "", zkutil::CreateMode::Persistent)); + + const auto partitions_path = topic_path / "partitions"; ops.emplace_back(zkutil::makeCreateRequest(partitions_path, "", zkutil::CreateMode::Persistent)); - // TODO(antaljanosbenjamin): handle changing number of partitions - for (auto partition_id{0U}; partition_id < topic_partition_count.partition_count; ++partition_id) - ops.emplace_back(zkutil::makeCreateRequest(partitions_path / toString(partition_id), "", zkutil::CreateMode::Persistent)); } // Create the first replica @@ -799,18 +792,21 @@ void StorageKafka2::dropReplica() std::optional StorageKafka2::lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const TopicPartitions & topic_partitions) { - std::vector topic_partition_paths; + std::vector topic_partition_paths; topic_partition_paths.reserve(topic_partitions.size()); for (const auto & topic_partition : topic_partitions) topic_partition_paths.emplace_back(getTopicPartitionPath(topic_partition)); Coordination::Requests ops; + static constexpr auto ignore_if_exists = true; + for (const auto & topic_partition_path : topic_partition_paths) { - LOG_TRACE(log, "Creating locking ops for: {}", topic_partition_path + lock_file_name); - ops.push_back(zkutil::makeCreateRequest( - topic_partition_path + lock_file_name, kafka_settings->kafka_replica_name.value, zkutil::CreateMode::Ephemeral)); + const auto lock_file_path = String(topic_partition_path / lock_file_name); + LOG_TRACE(log, "Creating locking ops for: {}", lock_file_path); + ops.push_back(zkutil::makeCreateRequest(topic_partition_path, "", zkutil::CreateMode::Persistent, ignore_if_exists)); + ops.push_back(zkutil::makeCreateRequest(lock_file_path, kafka_settings->kafka_replica_name.value, zkutil::CreateMode::Ephemeral)); } Coordination::Responses responses; @@ -831,10 +827,10 @@ StorageKafka2::lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const Topi for (; tp_it != topic_partitions.end(); ++tp_it, ++path_it) { using zkutil::EphemeralNodeHolder; - LockedTopicPartitionInfo lock_info{.lock = EphemeralNodeHolder::existing(*path_it + lock_file_name, keeper_to_use)}; + LockedTopicPartitionInfo lock_info{.lock = EphemeralNodeHolder::existing(*path_it / lock_file_name, keeper_to_use)}; - lock_info.committed_offset = getNumber(keeper_to_use, *path_it + commit_file_name); - lock_info.intent_size = getNumber(keeper_to_use, *path_it + intent_file_name); + lock_info.committed_offset = getNumber(keeper_to_use, *path_it / commit_file_name); + lock_info.intent_size = getNumber(keeper_to_use, *path_it / intent_file_name); LOG_TRACE( log, @@ -855,9 +851,9 @@ void StorageKafka2::saveCommittedOffset( zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition) { const auto partition_prefix = getTopicPartitionPath(topic_partition); - keeper_to_use.createOrUpdate(partition_prefix + commit_file_name, toString(topic_partition.offset), zkutil::CreateMode::Persistent); + keeper_to_use.createOrUpdate(partition_prefix / commit_file_name, toString(topic_partition.offset), zkutil::CreateMode::Persistent); // This is best effort, if it fails we will try to remove in the next round - keeper_to_use.tryRemove(partition_prefix + intent_file_name, -1); + keeper_to_use.tryRemove(partition_prefix / intent_file_name, -1); LOG_TEST(log, "Saved offset {} for topic-partition [{}:{}]", topic_partition.offset, topic_partition.topic, topic_partition.partition_id); } @@ -871,7 +867,7 @@ void StorageKafka2::saveIntent(zkutil::ZooKeeper & keeper_to_use, const TopicPar topic_partition.partition_id, topic_partition.offset); keeper_to_use.createOrUpdate( - getTopicPartitionPath(topic_partition) + intent_file_name, toString(intent), zkutil::CreateMode::Persistent); + getTopicPartitionPath(topic_partition) / intent_file_name, toString(intent), zkutil::CreateMode::Persistent); } @@ -1325,10 +1321,10 @@ zkutil::ZooKeeperPtr StorageKafka2::getZooKeeper() } -std::string StorageKafka2::getTopicPartitionPath(const TopicPartition & topic_partition) +fs::path StorageKafka2::getTopicPartitionPath(const TopicPartition & topic_partition) { - return kafka_settings->kafka_keeper_path.value + "/topics/" + topic_partition.topic + "/partitions/" - + std::to_string(topic_partition.partition_id) + '/'; + return fs::path(kafka_settings->kafka_keeper_path.value) / "topics" / topic_partition.topic / "partitions" + / std::to_string(topic_partition.partition_id); } } diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index e8cfcac2689..86d09c584f2 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -15,6 +15,7 @@ #include #include +#include #include #include @@ -191,7 +192,7 @@ private: bool checkDependencies(const StorageID & table_id); // Returns true if this is the first replica - bool createTableIfNotExists(const KafkaConsumer2Ptr & consumer); + bool createTableIfNotExists(); // Returns true if all of the nodes were cleaned up bool removeTableNodesFromZooKeeper(const zkutil::EphemeralNodeHolder::Ptr & drop_lock); // Creates only the replica in ZooKeeper. Shouldn't be called on the first replica as it is created in createTableIfNotExists @@ -211,7 +212,7 @@ private: zkutil::ZooKeeperPtr getZooKeeper(); - std::string getTopicPartitionPath(const TopicPartition & topic_partition); + std::filesystem::path getTopicPartitionPath(const TopicPartition & topic_partition); static VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode); }; From 1eec201777942652aa9fed0b929250a9c70e8be6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 7 Jun 2024 22:39:17 +0000 Subject: [PATCH 081/644] Make most of the tests work --- tests/integration/test_storage_kafka/test.py | 2502 +++++++++--------- 1 file changed, 1228 insertions(+), 1274 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 96438b5efa1..5b7d7f65b9f 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -164,6 +164,18 @@ def kafka_topic( # Code to release resource, e.g.: kafka_delete_topic(admin_client, topic_name, max_retries) + +@contextmanager +def existing_kafka_topic(admin_client, topic_name, max_retries=50): + try: + yield None + finally: + kafka_delete_topic(admin_client, topic_name, max_retries) + + +def get_admin_client(kafka_cluster): + return KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) + def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=15): logging.debug( "kafka_produce server:{}:{} topic:{}".format( @@ -183,7 +195,7 @@ def kafka_producer_send_heartbeat_msg(max_retries=50): kafka_produce(kafka_cluster, "test_heartbeat_topic", ["test"], retries=max_retries) -def kafka_consume(kafka_cluster, topic, needDecode=True, timestamp=0): +def kafka_consume(kafka_cluster, topic, need_decode=True, timestamp=0): consumer = KafkaConsumer( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), auto_offset_reset="earliest", @@ -193,7 +205,7 @@ def kafka_consume(kafka_cluster, topic, needDecode=True, timestamp=0): if toppar.topic == topic: for message in messages: assert timestamp == 0 or message.timestamp / 1000 == timestamp - if needDecode: + if need_decode: yield message.value.decode() else: yield message.value @@ -218,7 +230,21 @@ def kafka_produce_protobuf_messages(kafka_cluster, topic, start_index, num_messa logging.debug(("Produced {} messages for topic {}".format(num_messages, topic))) -def kafka_produce_protobuf_messages_no_delimeters( + +def kafka_consume_with_retry(kafka_cluster, topic, expected_messages, need_decode=True, timestamp=0, retry_count=20, sleep_time=0.1): + messages = [] + try_count = 0 + while try_count < retry_count: + try_count += 1 + messages.extend(kafka_consume(kafka_cluster, topic, need_decode=need_decode, timestamp=timestamp)) + if len(messages) == expected_messages: + break + time.sleep(sleep_time) + if len(messages) != expected_messages: + raise Exception(f"Got only {len(messages)} messages") + return messages + +def kafka_produce_protobuf_messages_no_delimiters( kafka_cluster, topic, start_index, num_messages ): data = "" @@ -312,6 +338,8 @@ def create_settings_string(settings): def format_value(value): if isinstance(value, str): return f"'{value}'" + elif isinstance(value, bool): + return str(int(value)) return str(value) settings_string = "SETTINGS " @@ -371,12 +399,20 @@ SETTINGS allow_experimental_kafka_store_offsets_in_keeper=1""" logging.debug(f"Generated new create query: {query}") return query +def must_use_thread_per_consumer(generator): + if generator == generate_old_create_table_query: + return False + if generator == generate_new_create_table_query: + return True + raise Exception("Unexpected generator") + + def get_topic_postfix(generator): if generator == generate_old_create_table_query: return "old" if generator == generate_new_create_table_query: return "new" - raise "Unexpected generator" + raise Exception("Unexpected generator") # Tests @pytest.mark.parametrize('create_query_generator, do_direct_read', [(generate_old_create_table_query, True), (generate_new_create_table_query, False)]) @@ -410,7 +446,7 @@ def test_kafka_column_types(kafka_cluster, create_query_generator, do_direct_rea if do_direct_read: # check ALIAS - instance.query(create_query_generator("kafka", "a Int, b String Alias toString(a)", settings={"kafka_commit_on_select":1})) + instance.query(create_query_generator("kafka", "a Int, b String Alias toString(a)", settings={"kafka_commit_on_select": True})) messages = [] for i in range(5): messages.append(json.dumps({"a": i})) @@ -600,12 +636,7 @@ def test_kafka_json_as_string(kafka_cluster): @pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) def test_kafka_formats(kafka_cluster, create_query_generator): - schema_registry_client = CachedSchemaRegistryClient( - "http://localhost:{}".format(kafka_cluster.schema_registry_port) - ) - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) + schema_registry_client = CachedSchemaRegistryClient({"url":f"http://localhost:{kafka_cluster.schema_registry_port}"}) # data was dumped from clickhouse itself in a following manner # clickhouse-client --format=Native --query='SELECT toInt64(number) as id, toUInt16( intDiv( id, 65536 ) ) as blockNo, reinterpretAsString(19777) as val1, toFloat32(0.5) as val2, toUInt8(1) as val3 from numbers(100) ORDER BY id' | xxd -ps | tr -d '\n' | sed 's/\(..\)/\\x\1/g' @@ -1001,7 +1032,7 @@ def test_kafka_formats(kafka_cluster, create_query_generator): assert TSV(result) == TSV(expected), "Proper result for format: {}".format( format_name ) - kafka_delete_topic(admin_client, topic_name) + kafka_delete_topic(get_admin_client(kafka_cluster), topic_name) # Since everything is async and shaky when receiving messages from Kafka, @@ -1177,9 +1208,7 @@ def test_kafka_issue4116(kafka_cluster): def test_kafka_consumer_hang(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) + admin_client = get_admin_client(kafka_cluster) topic_name = "consumer_hang" kafka_create_topic(admin_client, topic_name, num_partitions=8) @@ -1259,9 +1288,7 @@ def test_kafka_consumer_hang(kafka_cluster): def test_kafka_consumer_hang2(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) + admin_client = get_admin_client(kafka_cluster) topic_name = "consumer_hang2" kafka_create_topic(admin_client, topic_name) @@ -1322,9 +1349,7 @@ def test_kafka_consumer_hang2(kafka_cluster): # sequential read from different consumers leads to breaking lot of kafka invariants # (first consumer will get all partitions initially, and may have problems in doing polls every 60 sec) def test_kafka_read_consumers_in_parallel(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) + admin_client = get_admin_client(kafka_cluster) topic_name = "read_consumers_in_parallel" kafka_create_topic(admin_client, topic_name, num_partitions=8) @@ -1444,9 +1469,7 @@ def test_kafka_tsv_with_delimiter(kafka_cluster): def test_kafka_select_empty(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) + admin_client = get_admin_client(kafka_cluster) topic_name = "empty" kafka_create_topic(admin_client, topic_name) @@ -1624,13 +1647,13 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): """ ) - kafka_produce_protobuf_messages_no_delimeters( + kafka_produce_protobuf_messages_no_delimiters( kafka_cluster, "pb_no_delimiter", 0, 20 ) - kafka_produce_protobuf_messages_no_delimeters( + kafka_produce_protobuf_messages_no_delimiters( kafka_cluster, "pb_no_delimiter", 20, 1 ) - kafka_produce_protobuf_messages_no_delimeters( + kafka_produce_protobuf_messages_no_delimiters( kafka_cluster, "pb_no_delimiter", 21, 29 ) @@ -1673,31 +1696,28 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): @pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) def test_kafka_materialized_view(kafka_cluster, create_query_generator): topic_name="mv" - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.kafka; + + {create_query_generator("kafka", "key UInt64, value UInt64", topic_list=topic_name, consumer_group="mv")}; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; + """ ) - with kafka_topic(admin_client, topic_name): - instance.query( - f""" - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - DROP TABLE IF EXISTS test.kafka; - - {create_query_generator("kafka", "key UInt64, value UInt64", topic_list=topic_name, consumer_group="mv")}; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka; - """ - ) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, topic_name, messages) + messages = [] + for i in range(50): + messages.append(json.dumps({"key": i, "value": i})) + kafka_produce(kafka_cluster, topic_name, messages) + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): result = instance.query_with_retry("SELECT * FROM test.view", check_callback=kafka_check_result) kafka_check_result(result, True) @@ -1711,21 +1731,18 @@ def test_kafka_materialized_view(kafka_cluster, create_query_generator): ) -# TODO(antaljanosbenjamin): fails with the new, because it doesn't store the offsets... -@pytest.mark.parametrize('create_query_generator, thread_per_consumer, log_line', [ - (generate_new_create_table_query,1,r"kafka.*Saved offset [0-9]+ for topic-partition \[recreate_kafka_table:[0-9]+"), - (generate_old_create_table_query,0,"kafka.*Committed offset [0-9]+.*recreate_kafka_table"), +@pytest.mark.parametrize('create_query_generator, log_line', [ + (generate_new_create_table_query, r"kafka.*Saved offset [0-9]+ for topic-partition \[recreate_kafka_table:[0-9]+"), + (generate_old_create_table_query, "kafka.*Committed offset [0-9]+.*recreate_kafka_table"), ]) -def test_kafka_recreate_kafka_table(kafka_cluster, create_query_generator, thread_per_consumer, log_line): +def test_kafka_recreate_kafka_table(kafka_cluster, create_query_generator, log_line): """ Checks that materialized view work properly after dropping and recreating the Kafka table. """ - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) topic_name = "recreate_kafka_table" + thread_per_consumer = must_use_thread_per_consumer(create_query_generator) - with kafka_topic(admin_client, topic_name, num_partitions=6): + with kafka_topic(get_admin_client(kafka_cluster), topic_name, num_partitions=6): create_query = create_query_generator( "kafka", "key UInt64, value UInt64", @@ -1831,9 +1848,7 @@ def test_librdkafka_compression(kafka_cluster, create_query_generator, log_line) expected = "\n".join(expected) - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) + admin_client = get_admin_client(kafka_cluster) for compression_type in supported_compression_types: logging.debug(("Check compression {}".format(compression_type))) @@ -1869,39 +1884,34 @@ def test_librdkafka_compression(kafka_cluster, create_query_generator, log_line) instance.query("DROP TABLE test.consumer SYNC") -# TODO(antaljanosbenjamin): It fails with the new if the topic is not created explicitly @pytest.mark.parametrize('create_query_generator', [generate_new_create_table_query, generate_old_create_table_query]) def test_kafka_materialized_view_with_subquery(kafka_cluster, create_query_generator): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - topic_name = "mysq" logging.debug(f"Using topic {topic_name}") - with kafka_topic(admin_client, topic_name): - create_query = create_query_generator("kafka", "key UInt64, value UInt64", topic_list=topic_name, consumer_group=topic_name) - instance.query( - f""" - DROP TABLE IF EXISTS test.kafka; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; + create_query = create_query_generator("kafka", "key UInt64, value UInt64", topic_list=topic_name, consumer_group=topic_name) + instance.query( + f""" + DROP TABLE IF EXISTS test.kafka; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; - {create_query}; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM (SELECT * FROM test.kafka); - """ - ) + {create_query}; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM (SELECT * FROM test.kafka); + """ + ) - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, topic_name, messages) + messages = [] + for i in range(50): + messages.append(json.dumps({"key": i, "value": i})) + kafka_produce(kafka_cluster, topic_name, messages) - result = instance.query_with_retry("SELECT * FROM test.view", check_callback=kafka_check_result) + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): + result = instance.query_with_retry("SELECT * FROM test.view", check_callback=kafka_check_result, retry_count=40, sleep_time=0.75) instance.query( """ @@ -1915,9 +1925,6 @@ def test_kafka_materialized_view_with_subquery(kafka_cluster, create_query_gener @pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) def test_kafka_many_materialized_views(kafka_cluster, create_query_generator): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) topic_name = f"mmv-{get_topic_postfix(create_query_generator)}" create_query = create_query_generator( "kafka", @@ -1925,32 +1932,33 @@ def test_kafka_many_materialized_views(kafka_cluster, create_query_generator): topic_list=topic_name, consumer_group=f"{topic_name}-group" ) - with kafka_topic(admin_client, topic_name): - instance.query( - f""" - DROP TABLE IF EXISTS test.view1; - DROP TABLE IF EXISTS test.view2; - DROP TABLE IF EXISTS test.consumer1; - DROP TABLE IF EXISTS test.consumer2; - {create_query}; - CREATE TABLE test.view1 (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE TABLE test.view2 (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer1 TO test.view1 AS - SELECT * FROM test.kafka; - CREATE MATERIALIZED VIEW test.consumer2 TO test.view2 AS - SELECT * FROM test.kafka; - """ - ) - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, topic_name, messages) + instance.query( + f""" + DROP TABLE IF EXISTS test.view1; + DROP TABLE IF EXISTS test.view2; + DROP TABLE IF EXISTS test.consumer1; + DROP TABLE IF EXISTS test.consumer2; + {create_query}; + CREATE TABLE test.view1 (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE TABLE test.view2 (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer1 TO test.view1 AS + SELECT * FROM test.kafka; + CREATE MATERIALIZED VIEW test.consumer2 TO test.view2 AS + SELECT * FROM test.kafka; + """ + ) + messages = [] + for i in range(50): + messages.append(json.dumps({"key": i, "value": i})) + kafka_produce(kafka_cluster, topic_name, messages) + + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): result1 = instance.query_with_retry("SELECT * FROM test.view1", check_callback=kafka_check_result) result2 = instance.query_with_retry("SELECT * FROM test.view2", check_callback=kafka_check_result) @@ -1966,116 +1974,113 @@ def test_kafka_many_materialized_views(kafka_cluster, create_query_generator): kafka_check_result(result1, True) kafka_check_result(result2, True) -# TODO(antaljanosbenjamin) -def test_kafka_flush_on_big_message(kafka_cluster): + +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_flush_on_big_message(kafka_cluster, create_query_generator): # Create batches of messages of size ~100Kb kafka_messages = 1000 batch_messages = 1000 + topic_name = "flush" + get_topic_postfix(create_query_generator) messages = [ json.dumps({"key": i, "value": "x" * 100}) * batch_messages for i in range(kafka_messages) ] - kafka_produce(kafka_cluster, "flush", messages) + kafka_produce(kafka_cluster, topic_name, messages) - instance.query( + admin_client = get_admin_client(kafka_cluster) + + with existing_kafka_topic(admin_client, topic_name): + create_query = create_query_generator( + "kafka", + "key UInt64, value String", + topic_list=topic_name, + consumer_group=topic_name, + settings={"kafka_max_block_size": 10} + ) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + {create_query}; + CREATE TABLE test.view (key UInt64, value String) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'flush', - kafka_group_name = 'flush', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 10; - CREATE TABLE test.view (key UInt64, value String) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka; - """ - ) + ) - client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - received = False - while not received: - try: - offsets = client.list_consumer_group_offsets("flush") - for topic, offset in list(offsets.items()): - if topic.topic == "flush" and offset.offset == kafka_messages: - received = True - break - except kafka.errors.GroupCoordinatorNotAvailableError: - continue + received = False + while not received: + try: + offsets = admin_client.list_consumer_group_offsets(topic_name) + for topic, offset in list(offsets.items()): + if topic.topic == topic_name and offset.offset == kafka_messages: + received = True + break + except kafka.errors.GroupCoordinatorNotAvailableError: + continue - while True: - result = instance.query("SELECT count() FROM test.view") - if int(result) == kafka_messages * batch_messages: - break + while True: + result = instance.query("SELECT count() FROM test.view") + if int(result) == kafka_messages * batch_messages: + break - instance.query( + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) + ) - assert ( - int(result) == kafka_messages * batch_messages - ), "ClickHouse lost some messages: {}".format(result) + assert ( + int(result) == kafka_messages * batch_messages + ), "ClickHouse lost some messages: {}".format(result) def test_kafka_virtual_columns(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) topic_config = { # default retention, since predefined timestamp_ms is used. "retention.ms": "-1", } - kafka_create_topic(admin_client, "virt1", config=topic_config) - - instance.query( - """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'virt1', - kafka_group_name = 'virt1', - kafka_commit_on_select = 1, - kafka_format = 'JSONEachRow'; - """ - ) - - messages = "" - for i in range(25): - messages += json.dumps({"key": i, "value": i}) + "\n" - kafka_produce(kafka_cluster, "virt1", [messages], 0) - - messages = "" - for i in range(25, 50): - messages += json.dumps({"key": i, "value": i}) + "\n" - kafka_produce(kafka_cluster, "virt1", [messages], 0) - - result = "" - while True: - result += instance.query( - """SELECT _key, key, _topic, value, _offset, _partition, _timestamp = 0 ? '0000-00-00 00:00:00' : toString(_timestamp) AS _timestamp FROM test.kafka""", - ignore_error=True, + with kafka_topic(get_admin_client(kafka_cluster), "virt1", config=topic_config): + instance.query( + """ + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'virt1', + kafka_group_name = 'virt1', + kafka_commit_on_select = 1, + kafka_format = 'JSONEachRow'; + """ ) - if kafka_check_result(result, False, "test_kafka_virtual1.reference"): - break - kafka_check_result(result, True, "test_kafka_virtual1.reference") + messages = "" + for i in range(25): + messages += json.dumps({"key": i, "value": i}) + "\n" + kafka_produce(kafka_cluster, "virt1", [messages], 0) + + messages = "" + for i in range(25, 50): + messages += json.dumps({"key": i, "value": i}) + "\n" + kafka_produce(kafka_cluster, "virt1", [messages], 0) + + result = "" + while True: + result += instance.query( + """SELECT _key, key, _topic, value, _offset, _partition, _timestamp = 0 ? '0000-00-00 00:00:00' : toString(_timestamp) AS _timestamp FROM test.kafka""", + ignore_error=True, + ) + if kafka_check_result(result, False, "test_kafka_virtual1.reference"): + break + + kafka_check_result(result, True, "test_kafka_virtual1.reference") @pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) def test_kafka_virtual_columns_with_materialized_view(kafka_cluster, create_query_generator): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) topic_config = { # default retention, since predefined timestamp_ms is used. "retention.ms": "-1", @@ -2089,7 +2094,7 @@ def test_kafka_virtual_columns_with_materialized_view(kafka_cluster, create_quer topic_list=topic_name, consumer_group=f"{topic_name}-group" ) - with kafka_topic(admin_client, topic_name, config=topic_config): + with kafka_topic(get_admin_client(kafka_cluster), topic_name, config=topic_config): instance.query( f""" @@ -2153,73 +2158,62 @@ def test_kafka_insert(kafka_cluster, create_query_generator): ) ) + message_count = 50 values = [] - for i in range(50): + for i in range(message_count): values.append("({i}, {i})".format(i=i)) values = ",".join(values) insert_with_retry(instance, values) - messages = [] - try_count = 0 - while True and try_count < 5: - try_count += 1 - messages.extend(kafka_consume(kafka_cluster, topic_name)) - if len(messages) == 50: - break - time.sleep(0.1) - + messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count) result = "\n".join(messages) kafka_check_result(result, True) @pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) def test_kafka_produce_consume(kafka_cluster, create_query_generator): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - topic_name = "insert2" + get_topic_postfix(create_query_generator) - with kafka_topic(admin_client, topic_name): - create_query = create_query_generator( - "kafka", - "key UInt64, value UInt64", - topic_list=topic_name, - consumer_group=topic_name, - format="TSV" - ) - instance.query( - f""" - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - {create_query}; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka; - """ - ) + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format="TSV" + ) + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + {create_query}; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; + """ + ) - messages_num = 10000 + messages_num = 10000 - def insert(): - values = [] - for i in range(messages_num): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) + def insert(): + values = [] + for i in range(messages_num): + values.append("({i}, {i})".format(i=i)) + values = ",".join(values) - insert_with_retry(instance, values) + insert_with_retry(instance, values) - threads = [] - threads_num = 16 - for _ in range(threads_num): - threads.append(threading.Thread(target=insert)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() + threads = [] + threads_num = 16 + for _ in range(threads_num): + threads.append(threading.Thread(target=insert)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): expected_row_count = messages_num * threads_num result = instance.query_with_retry( "SELECT count() FROM test.view", @@ -2242,19 +2236,21 @@ def test_kafka_produce_consume(kafka_cluster, create_query_generator): ), "ClickHouse lost some messages: {}".format(result) -def test_kafka_commit_on_block_write(kafka_cluster): +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_commit_on_block_write(kafka_cluster, create_query_generator): + topic_name="block" + get_topic_postfix(create_query_generator) + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + settings={"kafka_max_block_size": 100}, + ) instance.query( - """ + f""" DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'block', - kafka_group_name = 'block', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 100, - kafka_row_delimiter = '\\n'; + {create_query}; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; @@ -2265,45 +2261,37 @@ def test_kafka_commit_on_block_write(kafka_cluster): cancel = threading.Event() + # We need to pass i as a reference. Simple integers are passed by value. + # Making an array is probably the easiest way to "force pass by reference". i = [0] - def produce(): + def produce(i): while not cancel.is_set(): messages = [] for _ in range(101): messages.append(json.dumps({"key": i[0], "value": i[0]})) i[0] += 1 - kafka_produce(kafka_cluster, "block", messages) + kafka_produce(kafka_cluster, topic_name, messages) - kafka_thread = threading.Thread(target=produce) + kafka_thread = threading.Thread(target=produce, args=[i]) kafka_thread.start() - while int(instance.query("SELECT count() FROM test.view")) == 0: - time.sleep(1) + instance.query_with_retry( + "SELECT count() FROM test.view", + sleep_time=1, + check_callback=lambda res: int(res) >= 100) cancel.set() - instance.query( - """ - DROP TABLE test.kafka; - """ - ) + instance.query("DROP TABLE test.kafka SYNC") - instance.query( - """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'block', - kafka_group_name = 'block', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 100, - kafka_row_delimiter = '\\n'; - """ - ) + instance.query(create_query) + kafka_thread.join() - while int(instance.query("SELECT uniqExact(key) FROM test.view")) < i[0]: - time.sleep(1) + instance.query_with_retry( + "SELECT uniqExact(key) FROM test.view", + sleep_time=1, + check_callback=lambda res: int(res) >= i[0]) result = int(instance.query("SELECT count() == uniqExact(key) FROM test.view")) @@ -2318,19 +2306,18 @@ def test_kafka_commit_on_block_write(kafka_cluster): assert result == 1, "Messages from kafka get duplicated!" -@pytest.mark.parametrize('create_query_generator, thread_per_consumer, log_line', [ - (generate_old_create_table_query,0,"kafka.*Committed offset 2.*virt2_[01]"), - (generate_new_create_table_query,1,r"kafka.*Saved offset 2[0-9]* for topic-partition \[virt2_[01]:[0-9]+"), +@pytest.mark.parametrize('create_query_generator, log_line', [ + (generate_old_create_table_query, "kafka.*Committed offset 2.*virt2_[01]"), + (generate_new_create_table_query, r"kafka.*Saved offset 2[0-9]* for topic-partition \[virt2_[01]:[0-9]+"), ]) -def test_kafka_virtual_columns2(kafka_cluster, create_query_generator, thread_per_consumer, log_line): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) +def test_kafka_virtual_columns2(kafka_cluster, create_query_generator, log_line): + admin_client = get_admin_client(kafka_cluster) topic_config = { # default retention, since predefined timestamp_ms is used. "retention.ms": "-1", } + thread_per_consumer = must_use_thread_per_consumer(create_query_generator) topic_name_0 = "virt2_0" topic_name_1 = "virt2_1" consumer_group = "virt2"+get_topic_postfix(create_query_generator) @@ -2342,7 +2329,7 @@ def test_kafka_virtual_columns2(kafka_cluster, create_query_generator, thread_pe topic_list=f"{topic_name_0},{topic_name_1}", consumer_group=consumer_group, settings={ - "kafka_num_consumers":2, + "kafka_num_consumers": 2, "kafka_thread_per_consumer": thread_per_consumer, } ) @@ -2544,24 +2531,18 @@ def test_kafka_producer_consumer_separate_settings(kafka_cluster): assert property_in_log in kafka_producer_applyed_properties -# TODO(antaljanosbenjamin) @pytest.mark.parametrize('create_query_generator, log_line', [ - #(generate_new_create_table_query,"Saved offset 5"), + (generate_new_create_table_query,"Saved offset 5"), (generate_old_create_table_query, "Committed offset 5"), ]) def test_kafka_produce_key_timestamp(kafka_cluster, create_query_generator, log_line): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - topic_name = "insert3" topic_config = { # default retention, since predefined timestamp_ms is used. "retention.ms": "-1", } - with kafka_topic(admin_client, topic_name, config=topic_config): - + with kafka_topic(get_admin_client(kafka_cluster), topic_name, config=topic_config): writer_create_query = create_query_generator( "kafka_writer", "key UInt64, value UInt64, _key String, _timestamp DateTime('UTC')", @@ -2617,11 +2598,12 @@ def test_kafka_produce_key_timestamp(kafka_cluster, create_query_generator, log_ 5 5 k5 1577836805 k5 insert3 0 4 1577836805 """ - result = instance.query_with_retry("SELECT * FROM test.view ORDER BY value", ignore_error=True, check_callback=lambda res: TSV(res) == TSV(expected)) - - # logging.debug(result) - - + result = instance.query_with_retry( + "SELECT * FROM test.view ORDER BY value", + ignore_error=True, + retry_count=5, + sleep_time=1, + check_callback=lambda res: TSV(res) == TSV(expected)) assert TSV(result) == TSV(expected) @@ -2655,15 +2637,8 @@ def test_kafka_insert_avro(kafka_cluster, create_query_generator): "INSERT INTO test.kafka select number*10 as key, number*100 as value, 1636505534 as _timestamp from numbers(4) SETTINGS output_format_avro_rows_in_file = 2, output_format_avro_codec = 'deflate'" ) - messages = [] - while True: - messages.extend( - kafka_consume( - kafka_cluster, topic_name, needDecode=False, timestamp=1636505534 - ) - ) - if len(messages) == 2: - break + message_count = 2 + messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count, need_decode=False, timestamp=1636505534) result = "" for a_message in messages: @@ -2679,67 +2654,69 @@ def test_kafka_insert_avro(kafka_cluster, create_query_generator): assert result == expected_result -# TODO(antaljanosbenjamin) -def test_kafka_produce_consume_avro(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - - topic_name = "insert_avro" - kafka_create_topic(admin_client, topic_name) - - num_rows = 75 - - instance.query( - """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.kafka; - DROP TABLE IF EXISTS test.kafka_writer; - - CREATE TABLE test.kafka_writer (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'avro', - kafka_group_name = 'avro', - kafka_format = 'Avro'; - - - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'avro', - kafka_group_name = 'avro', - kafka_format = 'Avro'; - - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.kafka; - """ - ) - - instance.query( - "INSERT INTO test.kafka_writer select number*10 as key, number*100 as value from numbers({num_rows}) SETTINGS output_format_avro_rows_in_file = 7".format( - num_rows=num_rows - ) - ) - - instance.wait_for_log_line( - "Committed offset {offset}".format(offset=math.ceil(num_rows / 7)) - ) - - expected_num_rows = instance.query( - "SELECT COUNT(1) FROM test.view", ignore_error=True - ) - assert int(expected_num_rows) == num_rows - - expected_max_key = instance.query( - "SELECT max(key) FROM test.view", ignore_error=True - ) - assert int(expected_max_key) == (num_rows - 1) * 10 - - kafka_delete_topic(admin_client, topic_name) - - @pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_produce_consume_avro(kafka_cluster, create_query_generator): + topic_name = "insert_avro" + get_topic_postfix(create_query_generator) + with kafka_topic(get_admin_client(kafka_cluster), topic_name): + + num_rows = 75 + + writer_create_query = create_query_generator( + "kafka_writer", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format="Avro", + ) + + reader_create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format="Avro", + ) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.kafka; + DROP TABLE IF EXISTS test.kafka_writer; + + {writer_create_query}; + {reader_create_query}; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.kafka; + """ + ) + + instance.query( + "INSERT INTO test.kafka_writer select number*10 as key, number*100 as value from numbers({num_rows}) SETTINGS output_format_avro_rows_in_file = 7".format( + num_rows=num_rows + ) + ) + + instance.wait_for_log_line( + "Committed offset {offset}".format(offset=math.ceil(num_rows / 7)) + ) + + expected_num_rows = instance.query( + "SELECT COUNT(1) FROM test.view", ignore_error=True + ) + assert int(expected_num_rows) == num_rows + + expected_max_key = instance.query( + "SELECT max(key) FROM test.view", ignore_error=True + ) + assert int(expected_max_key) == (num_rows - 1) * 10 + + +@pytest.mark.parametrize('create_query_generator', [ + generate_old_create_table_query, + # TODO(antaljanosbenjamin): Something is off with timing + # generate_new_create_table_query +]) def test_kafka_flush_by_time(kafka_cluster, create_query_generator): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) @@ -2753,7 +2730,7 @@ def test_kafka_flush_by_time(kafka_cluster, create_query_generator): topic_list=topic_name, consumer_group=topic_name, settings={ - "kafka_max_block_size":100, + "kafka_max_block_size": 100, } ) instance.query( @@ -2787,9 +2764,10 @@ def test_kafka_flush_by_time(kafka_cluster, create_query_generator): """ ) + # By default the flush timeout should be 7.5 seconds => 18 seconds should be enough for 2 flushes, but not for 3 time.sleep(18) - result = instance.query("SELECT uniqExact(ts) = 2, count() >= 15 FROM test.view") + result = instance.query("SELECT uniqExact(ts), count() >= 15 FROM test.view") cancel.set() kafka_thread.join() @@ -2801,14 +2779,11 @@ def test_kafka_flush_by_time(kafka_cluster, create_query_generator): """ ) - assert TSV(result) == TSV("1 1") + assert TSV(result) == TSV("2 1") @pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) def test_kafka_flush_by_block_size(kafka_cluster, create_query_generator): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) topic_name = "flush_by_block_size" + get_topic_postfix(create_query_generator) cancel = threading.Event() @@ -2820,7 +2795,7 @@ def test_kafka_flush_by_block_size(kafka_cluster, create_query_generator): kafka_thread = threading.Thread(target=produce) - with kafka_topic(admin_client, topic_name): + with kafka_topic(get_admin_client(kafka_cluster), topic_name): kafka_thread.start() create_query = create_query_generator( @@ -3105,65 +3080,79 @@ def test_kafka_rebalance(kafka_cluster, create_query_generator, log_line): assert result == 1, "Messages from kafka get duplicated!" -def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): - messages = [json.dumps({"key": j + 1, "value": "x" * 300}) for j in range(22)] - kafka_produce(kafka_cluster, "no_holes_when_write_suffix_failed", messages) +# TODO(antaljanosbenjamin): find another way to make insertion fail +@pytest.mark.parametrize('create_query_generator', [ + generate_old_create_table_query, + # generate_new_create_table_query, +]) +def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster, create_query_generator): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + topic_name = "no_holes_when_write_suffix_failed" + get_topic_postfix(create_query_generator) - instance.query( + with kafka_topic(admin_client, topic_name): + messages = [json.dumps({"key": j + 1, "value": "x" * 300}) for j in range(22)] + kafka_produce(kafka_cluster, topic_name, messages) + + create_query = create_query_generator( + "kafka", + "key UInt64, value String", + topic_list=topic_name, + consumer_group=topic_name, + settings={ + "kafka_max_block_size": 20, + "kafka_flush_interval_ms": 2000, + } + ) + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + + {create_query}; + + CREATE TABLE test.view (key UInt64, value String) + ENGINE = ReplicatedMergeTree('/clickhouse/kafkatest/tables/{topic_name}', 'node1') + ORDER BY key; """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; + ) - CREATE TABLE test.kafka (key UInt64, value String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'no_holes_when_write_suffix_failed', - kafka_group_name = 'no_holes_when_write_suffix_failed', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 20, - kafka_flush_interval_ms = 2000; + # init PartitionManager (it starts container) earlier + pm = PartitionManager() - CREATE TABLE test.view (key UInt64, value String) - ENGINE = ReplicatedMergeTree('/clickhouse/kafkatest/tables/no_holes_when_write_suffix_failed', 'node1') - ORDER BY key; - """ - ) - - # init PartitionManager (it starts container) earlier - pm = PartitionManager() - - instance.query( + instance.query( + """ + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka + WHERE NOT sleepEachRow(0.25); """ - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka - WHERE NOT sleepEachRow(0.25); - """ - ) + ) - instance.wait_for_log_line("Polled batch of 20 messages") - # the tricky part here is that disconnect should happen after write prefix, but before write suffix - # we have 0.25 (sleepEachRow) * 20 ( Rows ) = 5 sec window after "Polled batch of 20 messages" - # while materialized view is working to inject zookeeper failure - pm.drop_instance_zk_connections(instance) - instance.wait_for_log_line( - "Error.*(Connection loss|Coordination::Exception).*while pushing to view" - ) - pm.heal_all() - instance.wait_for_log_line("Committed offset 22") + instance.wait_for_log_line("Polled batch of 20 messages") + # the tricky part here is that disconnect should happen after write prefix, but before write suffix + # we have 0.25 (sleepEachRow) * 20 ( Rows ) = 5 sec window after "Polled batch of 20 messages" + # while materialized view is working to inject zookeeper failure + pm.drop_instance_zk_connections(instance) + instance.wait_for_log_line( + "Error.*(Connection loss|Coordination::Exception).*while pushing to view" + ) + pm.heal_all() + instance.wait_for_log_line("Committed offset 22") - result = instance.query("SELECT count(), uniqExact(key), max(key) FROM test.view") - logging.debug(result) + result = instance.query("SELECT count(), uniqExact(key), max(key) FROM test.view") + logging.debug(result) - # kafka_cluster.open_bash_shell('instance') + # kafka_cluster.open_bash_shell('instance') - instance.query( + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) + ) - assert TSV(result) == TSV("22\t22\t22") + assert TSV(result) == TSV("22\t22\t22") def test_exception_from_destructor(kafka_cluster): @@ -3209,12 +3198,25 @@ def test_exception_from_destructor(kafka_cluster): assert TSV(instance.query("SELECT 1")) == TSV("1") -def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_commits_of_unprocessed_messages_on_drop(kafka_cluster, create_query_generator): + topic_name = "commits_of_unprocessed_messages_on_drop" + get_topic_postfix(create_query_generator) messages = [json.dumps({"key": j + 1, "value": j + 1}) for j in range(1)] - kafka_produce(kafka_cluster, "commits_of_unprocessed_messages_on_drop", messages) + kafka_produce(kafka_cluster, topic_name, messages) + + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=f"{topic_name}_test_group", + settings={ + "kafka_max_block_size": 1000, + "kafka_flush_interval_ms": 1000, + } + ) instance.query( - """ + f""" DROP TABLE IF EXISTS test.destination SYNC; CREATE TABLE test.destination ( key UInt64, @@ -3229,14 +3231,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): ENGINE = MergeTree() ORDER BY key; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'commits_of_unprocessed_messages_on_drop', - kafka_group_name = 'commits_of_unprocessed_messages_on_drop_test_group', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 1000, - kafka_flush_interval_ms = 1000; + {create_query}; CREATE MATERIALIZED VIEW test.kafka_consumer TO test.destination AS SELECT @@ -3265,7 +3260,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): messages.append(json.dumps({"key": i[0], "value": i[0]})) i[0] += 1 kafka_produce( - kafka_cluster, "commits_of_unprocessed_messages_on_drop", messages + kafka_cluster, topic_name, messages ) time.sleep(0.5) @@ -3279,17 +3274,18 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): """ ) + new_create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=f"{topic_name}_test_group", + settings={ + "kafka_max_block_size": 10000, + "kafka_flush_interval_ms": 1000, + } + ) instance.query( - """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'commits_of_unprocessed_messages_on_drop', - kafka_group_name = 'commits_of_unprocessed_messages_on_drop_test_group', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 10000, - kafka_flush_interval_ms = 1000; - """ + new_create_query ) cancel.set() @@ -3314,20 +3310,26 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): assert TSV(result) == TSV("{0}\t{0}\t{0}".format(i[0] - 1)), "Missing data!" -def test_bad_reschedule(kafka_cluster): - messages = [json.dumps({"key": j + 1, "value": j + 1}) for j in range(20000)] - kafka_produce(kafka_cluster, "test_bad_reschedule", messages) +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_bad_reschedule(kafka_cluster, create_query_generator): + topic_name = "test_bad_reschedule" + get_topic_postfix(create_query_generator) + messages = [json.dumps({"key": j + 1, "value": j + 1}) for j in range(20000)] + kafka_produce(kafka_cluster, topic_name, messages) + + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + settings={ + "kafka_max_block_size": 1000, + "kafka_flush_interval_ms": 1000, + } + ) instance.query( - """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'test_bad_reschedule', - kafka_group_name = 'test_bad_reschedule', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 1000, - kafka_flush_interval_ms = 1000; + f""" + {create_query}; CREATE MATERIALIZED VIEW test.destination Engine=Log AS SELECT @@ -3425,21 +3427,23 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): # if we came to partition end we will repeat polling until reaching kafka_max_block_size or flush_interval -# that behavior is a bit quesionable - we can just take a bigger pauses between polls instead - +# that behavior is a bit questionable - we can just take a bigger pauses between polls instead - # to do more job in a single pass, and give more rest for a thread. # But in cases of some peaky loads in kafka topic the current contract sounds more predictable and # easier to understand, so let's keep it as is for now. # also we can came to eof because we drained librdkafka internal queue too fast -def test_premature_flush_on_eof(kafka_cluster): +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_premature_flush_on_eof(kafka_cluster, create_query_generator): + topic_name = "premature_flush_on_eof" + get_topic_postfix(create_query_generator) + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + ) instance.query( - """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'premature_flush_on_eof', - kafka_group_name = 'premature_flush_on_eof', - kafka_format = 'JSONEachRow'; - SELECT * FROM test.kafka LIMIT 1; + f""" + {create_query}; CREATE TABLE test.destination ( key UInt64, value UInt64, @@ -3455,13 +3459,13 @@ def test_premature_flush_on_eof(kafka_cluster): """ ) - # messages created here will be consumed immedeately after MV creation + # messages created here will be consumed immediately after MV creation # reaching topic EOF. - # But we should not do flush immedeately after reaching EOF, because + # But we should not do flush immediately after reaching EOF, because # next poll can return more data, and we should respect kafka_flush_interval_ms # and try to form bigger block - messages = [json.dumps({"key": j + 1, "value": j + 1}) for j in range(1)] - kafka_produce(kafka_cluster, "premature_flush_on_eof", messages) + messages = [json.dumps({"key": 1, "value": 1})] + kafka_produce(kafka_cluster, topic_name, messages) instance.query( """ @@ -3485,7 +3489,7 @@ def test_premature_flush_on_eof(kafka_cluster): instance.wait_for_log_line("Stalled") # produce more messages after delay - kafka_produce(kafka_cluster, "premature_flush_on_eof", messages) + kafka_produce(kafka_cluster, topic_name, messages) # data was not flushed yet (it will be flushed 7.5 sec after creating MV) assert int(instance.query("SELECT count() FROM test.destination")) == 0 @@ -3506,58 +3510,75 @@ def test_premature_flush_on_eof(kafka_cluster): ) -def test_kafka_unavailable(kafka_cluster): - messages = [json.dumps({"key": j + 1, "value": j + 1}) for j in range(20000)] - kafka_produce(kafka_cluster, "test_bad_reschedule", messages) +@pytest.mark.parametrize('create_query_generator, do_direct_read', [ + (generate_old_create_table_query, True), + (generate_new_create_table_query, False) +]) +def test_kafka_unavailable(kafka_cluster, create_query_generator, do_direct_read): + number_of_messages=20000 + topic_name = "test_bad_reschedule" + get_topic_postfix(create_query_generator) + messages = [json.dumps({"key": j + 1, "value": j + 1}) for j in range(number_of_messages)] + kafka_produce(kafka_cluster, topic_name, messages) - kafka_cluster.pause_container("kafka1") + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): + kafka_cluster.pause_container("kafka1") - instance.query( + create_query = create_query_generator( + "test_bad_reschedule", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + settings={"kafka_max_block_size": 1000} + ) + instance.query( + f""" + {create_query}; + + CREATE MATERIALIZED VIEW test.destination_unavailable Engine=Log AS + SELECT + key, + now() as consume_ts, + value, + _topic, + _key, + _offset, + _partition, + _timestamp + FROM test.test_bad_reschedule; """ - CREATE TABLE test.test_bad_reschedule (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'test_bad_reschedule', - kafka_group_name = 'test_bad_reschedule', - kafka_format = 'JSONEachRow', - kafka_commit_on_select = 1, - kafka_max_block_size = 1000; + ) - CREATE MATERIALIZED VIEW test.destination_unavailable Engine=Log AS - SELECT - key, - now() as consume_ts, - value, - _topic, - _key, - _offset, - _partition, - _timestamp - FROM test.test_bad_reschedule; - """ - ) + if do_direct_read: + instance.query("SELECT * FROM test.test_bad_reschedule") + instance.query("SELECT count() FROM test.destination_unavailable") - instance.query("SELECT * FROM test.test_bad_reschedule") - instance.query("SELECT count() FROM test.destination_unavailable") + # enough to trigger issue + time.sleep(30) + kafka_cluster.unpause_container("kafka1") - # enough to trigger issue - time.sleep(30) - kafka_cluster.unpause_container("kafka1") + result = instance.query_with_retry( + "SELECT count() FROM test.destination_unavailable", + sleep_time=1, + check_callback=lambda res: int(res) == number_of_messages) - while ( - int(instance.query("SELECT count() FROM test.destination_unavailable")) < 20000 - ): - print("Waiting for consume") - time.sleep(1) + assert int(result) == number_of_messages -def test_kafka_issue14202(kafka_cluster): +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_issue14202(kafka_cluster, create_query_generator): """ INSERT INTO Kafka Engine from an empty SELECT sub query was leading to failure """ + topic_name = "issue14202" + get_topic_postfix(create_query_generator) + create_query = create_query_generator( + "kafka_q", + "t UInt64, some_string String", + topic_list=topic_name, + consumer_group=topic_name, + ) instance.query( - """ + f""" CREATE TABLE test.empty_table ( dt Date, some_string String @@ -3566,12 +3587,7 @@ def test_kafka_issue14202(kafka_cluster): PARTITION BY toYYYYMM(dt) ORDER BY some_string; - CREATE TABLE test.kafka_q (t UInt64, `some_string` String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'issue14202', - kafka_group_name = 'issue14202', - kafka_format = 'JSONEachRow'; + {create_query}; """ ) @@ -3622,20 +3638,25 @@ def random_string(size=8): return "".join(random.choices(string.ascii_uppercase + string.digits, k=size)) -def test_kafka_engine_put_errors_to_stream(kafka_cluster): +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_engine_put_errors_to_stream(kafka_cluster, create_query_generator): + topic_name = "kafka_engine_put_errors_to_stream" + get_topic_postfix(create_query_generator) + create_query = create_query_generator( + "kafka", + "i Int64, s String", + topic_list=topic_name, + consumer_group=topic_name, + settings={ + "kafka_max_block_size": 128, + "kafka_handle_error_mode": "stream", + } + ) instance.query( - """ + f""" DROP TABLE IF EXISTS test.kafka; DROP TABLE IF EXISTS test.kafka_data; DROP TABLE IF EXISTS test.kafka_errors; - CREATE TABLE test.kafka (i Int64, s String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'kafka_engine_put_errors_to_stream', - kafka_group_name = 'kafka_engine_put_errors_to_stream', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 128, - kafka_handle_error_mode = 'stream'; + {create_query}; CREATE MATERIALIZED VIEW test.kafka_data (i Int64, s String) ENGINE = MergeTree ORDER BY i @@ -3663,19 +3684,20 @@ def test_kafka_engine_put_errors_to_stream(kafka_cluster): json.dumps({"i": "n_" + random_string(4), "s": random_string(8)}) ) - kafka_produce(kafka_cluster, "kafka_engine_put_errors_to_stream", messages) - instance.wait_for_log_line("Committed offset 128") + kafka_produce(kafka_cluster, topic_name, messages) + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): + instance.wait_for_log_line("Committed offset 128") - assert TSV(instance.query("SELECT count() FROM test.kafka_data")) == TSV("64") - assert TSV(instance.query("SELECT count() FROM test.kafka_errors")) == TSV("64") + assert TSV(instance.query("SELECT count() FROM test.kafka_data")) == TSV("64") + assert TSV(instance.query("SELECT count() FROM test.kafka_errors")) == TSV("64") - instance.query( + instance.query( + """ + DROP TABLE test.kafka; + DROP TABLE test.kafka_data; + DROP TABLE test.kafka_errors; """ - DROP TABLE test.kafka; - DROP TABLE test.kafka_data; - DROP TABLE test.kafka_errors; - """ - ) + ) def gen_normal_json(): @@ -3704,21 +3726,27 @@ def gen_message_with_jsons(jsons=10, malformed=0): return s.getvalue() -def test_kafka_engine_put_errors_to_stream_with_random_malformed_json(kafka_cluster): +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_engine_put_errors_to_stream_with_random_malformed_json(kafka_cluster, create_query_generator): + topic_name = "kafka_engine_put_errors_to_stream_with_random_malformed_json" + get_topic_postfix(create_query_generator) + create_query = create_query_generator( + "kafka", + "i Int64, s String", + topic_list=topic_name, + consumer_group=topic_name, + settings={ + "kafka_max_block_size": 100, + "kafka_poll_max_batch_size": 1, + "kafka_handle_error_mode": "stream", + } + ) + instance.query( - """ + f""" DROP TABLE IF EXISTS test.kafka; DROP TABLE IF EXISTS test.kafka_data; DROP TABLE IF EXISTS test.kafka_errors; - CREATE TABLE test.kafka (i Int64, s String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'kafka_engine_put_errors_to_stream_with_random_malformed_json', - kafka_group_name = 'kafka_engine_put_errors_to_stream_with_random_malformed_json', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 100, - kafka_poll_max_batch_size = 1, - kafka_handle_error_mode = 'stream'; + {create_query}; CREATE MATERIALIZED VIEW test.kafka_data (i Int64, s String) ENGINE = MergeTree ORDER BY i @@ -3743,28 +3771,25 @@ def test_kafka_engine_put_errors_to_stream_with_random_malformed_json(kafka_clus else: messages.append(gen_message_with_jsons(10, 0)) - kafka_produce( - kafka_cluster, - "kafka_engine_put_errors_to_stream_with_random_malformed_json", - messages, - ) + kafka_produce(kafka_cluster, topic_name, messages) + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): + instance.wait_for_log_line("Committed offset 128") + # 64 good messages, each containing 10 rows + assert TSV(instance.query("SELECT count() FROM test.kafka_data")) == TSV("640") + # 64 bad messages, each containing some broken row + assert TSV(instance.query("SELECT count() FROM test.kafka_errors")) == TSV("64") - instance.wait_for_log_line("Committed offset 128") - # 64 good messages, each containing 10 rows - assert TSV(instance.query("SELECT count() FROM test.kafka_data")) == TSV("640") - # 64 bad messages, each containing some broken row - assert TSV(instance.query("SELECT count() FROM test.kafka_errors")) == TSV("64") - - instance.query( + instance.query( + """ + DROP TABLE test.kafka; + DROP TABLE test.kafka_data; + DROP TABLE test.kafka_errors; """ - DROP TABLE test.kafka; - DROP TABLE test.kafka_data; - DROP TABLE test.kafka_errors; - """ - ) + ) -def test_kafka_formats_with_broken_message(kafka_cluster): +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator): # data was dumped from clickhouse itself in a following manner # clickhouse-client --format=Native --query='SELECT toInt64(number) as id, toUInt16( intDiv( id, 65536 ) ) as blockNo, reinterpretAsString(19777) as val1, toFloat32(0.5) as val2, toUInt8(1) as val3 from numbers(100) ORDER BY id' | xxd -ps | tr -d '\n' | sed 's/\(..\)/\\x\1/g' admin_client = KafkaAdminClient( @@ -3782,7 +3807,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message '{"id":"0","blockNo":"BAD","val1":"AM","val2":0.5,"val3":1}', ], - "expected": """{"raw_message":"{\\"id\\":\\"0\\",\\"blockNo\\":\\"BAD\\",\\"val1\\":\\"AM\\",\\"val2\\":0.5,\\"val3\\":1}","error":"Cannot parse input: expected '\\"' before: 'BAD\\",\\"val1\\":\\"AM\\",\\"val2\\":0.5,\\"val3\\":1}': (while reading the value of key blockNo)"}""", + "expected": { + "raw_message":"{\"id\":\"0\",\"blockNo\":\"BAD\",\"val1\":\"AM\",\"val2\":0.5,\"val3\":1}", + "error":"Cannot parse input: expected '\"' before: 'BAD\",\"val1\":\"AM\",\"val2\":0.5,\"val3\":1}': (while reading the value of key blockNo)" + }, "supports_empty_value": True, "printable": True, }, @@ -3795,7 +3823,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message '["0", "BAD", "AM", 0.5, 1]', ], - "expected": """{"raw_message":"[\\"0\\", \\"BAD\\", \\"AM\\", 0.5, 1]","error":"Cannot parse input: expected '\\"' before: 'BAD\\", \\"AM\\", 0.5, 1]': (while reading the value of key blockNo)"}""", + "expected": { + "raw_message":"[\"0\", \"BAD\", \"AM\", 0.5, 1]", + "error":"Cannot parse input: expected '\"' before: 'BAD\", \"AM\", 0.5, 1]': (while reading the value of key blockNo)" + }, "supports_empty_value": True, "printable": True, }, @@ -3807,7 +3838,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message '["0", "BAD", "AM", 0.5, 1]', ], - "expected": """{"raw_message":"[\\"0\\", \\"BAD\\", \\"AM\\", 0.5, 1]","error":"Cannot parse JSON string: expected opening quote"}""", + "expected": { + "raw_message":"[\"0\", \"BAD\", \"AM\", 0.5, 1]", + "error":"Cannot parse JSON string: expected opening quote" + }, "printable": True, }, "TSKV": { @@ -3818,7 +3852,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message "id=0\tblockNo=BAD\tval1=AM\tval2=0.5\tval3=1\n", ], - "expected": '{"raw_message":"id=0\\tblockNo=BAD\\tval1=AM\\tval2=0.5\\tval3=1\\n","error":"Found garbage after field in TSKV format: blockNo: (at row 1)\\n"}', + "expected": { + "raw_message":"id=0\tblockNo=BAD\tval1=AM\tval2=0.5\tval3=1\n", + "error":"Found garbage after field in TSKV format: blockNo: (at row 1)\n" + }, "printable": True, }, "CSV": { @@ -3829,7 +3866,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message '0,"BAD","AM",0.5,1\n', ], - "expected": """{"raw_message":"0,\\"BAD\\",\\"AM\\",0.5,1\\n","error":"Cannot parse input: expected '\\"' before: 'BAD\\",\\"AM\\",0.5,1\\\\n'"}""", + "expected": { + "raw_message":"0,\"BAD\",\"AM\",0.5,1\n", + "error":"Cannot parse input: expected '\"' before: 'BAD\",\"AM\",0.5,1\\n'" + }, "printable": True, "supports_empty_value": True, }, @@ -3841,7 +3881,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message "0\tBAD\tAM\t0.5\t1\n", ], - "expected": """{"raw_message":"0\\tBAD\\tAM\\t0.5\\t1\\n","error":"Cannot parse input: expected '\\\\t' before: 'BAD\\\\tAM\\\\t0.5\\\\t1\\\\n'"}""", + "expected": { + "raw_message":"0\tBAD\tAM\t0.5\t1\n", + "error":"Cannot parse input: expected '\\t' before: 'BAD\\tAM\\t0.5\\t1\\n'" + }, "supports_empty_value": True, "printable": True, }, @@ -3853,7 +3896,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message '"id","blockNo","val1","val2","val3"\n0,"BAD","AM",0.5,1\n', ], - "expected": """{"raw_message":"\\"id\\",\\"blockNo\\",\\"val1\\",\\"val2\\",\\"val3\\"\\n0,\\"BAD\\",\\"AM\\",0.5,1\\n","error":"Cannot parse input: expected '\\"' before: 'BAD\\",\\"AM\\",0.5,1\\\\n'"}""", + "expected": { + "raw_message":"\"id\",\"blockNo\",\"val1\",\"val2\",\"val3\"\n0,\"BAD\",\"AM\",0.5,1\n", + "error":"Cannot parse input: expected '\"' before: 'BAD\",\"AM\",0.5,1\\n'" + }, "printable": True, }, "Values": { @@ -3864,7 +3910,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message "(0,'BAD','AM',0.5,1)", ], - "expected": r"""{"raw_message":"(0,'BAD','AM',0.5,1)","error":"Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero\/NULL instead of throwing exception"}""", + "expected": { + "raw_message":"(0,'BAD','AM',0.5,1)", + "error":"Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero/NULL instead of throwing exception" + }, "supports_empty_value": True, "printable": True, }, @@ -3876,7 +3925,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message "id\tblockNo\tval1\tval2\tval3\n0\tBAD\tAM\t0.5\t1\n", ], - "expected": """{"raw_message":"id\\tblockNo\\tval1\\tval2\\tval3\\n0\\tBAD\\tAM\\t0.5\\t1\\n","error":"Cannot parse input: expected '\\\\t' before: 'BAD\\\\tAM\\\\t0.5\\\\t1\\\\n"}""", + "expected": { + "raw_message":"id\tblockNo\tval1\tval2\tval3\n0\tBAD\tAM\t0.5\t1\n", + "error":"Cannot parse input: expected '\\t' before: 'BAD\\tAM\\t0.5\\t1\\n" + }, "supports_empty_value": True, "printable": True, }, @@ -3888,7 +3940,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message "id\tblockNo\tval1\tval2\tval3\nInt64\tUInt16\tString\tFloat32\tUInt8\n0\tBAD\tAM\t0.5\t1\n", ], - "expected": """{"raw_message":"id\\tblockNo\\tval1\\tval2\\tval3\\nInt64\\tUInt16\\tString\\tFloat32\\tUInt8\\n0\\tBAD\\tAM\\t0.5\\t1\\n","error":"Cannot parse input: expected '\\\\t' before: 'BAD\\\\tAM\\\\t0.5\\\\t1\\\\n'"}""", + "expected": { + "raw_message":"id\tblockNo\tval1\tval2\tval3\nInt64\tUInt16\tString\tFloat32\tUInt8\n0\tBAD\tAM\t0.5\t1\n", + "error":"Cannot parse input: expected '\\t' before: 'BAD\\tAM\\t0.5\\t1\\n'" + }, "printable": True, }, "Native": { @@ -3899,7 +3954,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message b"\x05\x01\x02\x69\x64\x05\x49\x6e\x74\x36\x34\x00\x00\x00\x00\x00\x00\x00\x00\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x06\x53\x74\x72\x69\x6e\x67\x03\x42\x41\x44\x04\x76\x61\x6c\x31\x06\x53\x74\x72\x69\x6e\x67\x02\x41\x4d\x04\x76\x61\x6c\x32\x07\x46\x6c\x6f\x61\x74\x33\x32\x00\x00\x00\x3f\x04\x76\x61\x6c\x33\x05\x55\x49\x6e\x74\x38\x01", ], - "expected": """{"raw_message":"050102696405496E743634000000000000000007626C6F636B4E6F06537472696E67034241440476616C3106537472696E6702414D0476616C3207466C6F617433320000003F0476616C330555496E743801","error":"Cannot convert: String to UInt16"}""", + "expected": { + "raw_message":"050102696405496E743634000000000000000007626C6F636B4E6F06537472696E67034241440476616C3106537472696E6702414D0476616C3207466C6F617433320000003F0476616C330555496E743801", + "error":"Cannot convert: String to UInt16" + }, "printable": False, }, "RowBinary": { @@ -3910,7 +3968,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message b"\x00\x00\x00\x00\x00\x00\x00\x00\x03\x42\x41\x44\x02\x41\x4d\x00\x00\x00\x3f\x01", ], - "expected": '{"raw_message":"00000000000000000342414402414D0000003F01","error":"Cannot read all data. Bytes read: 9. Bytes expected: 65.: (at row 1)\\n"}', + "expected": { + "raw_message":"00000000000000000342414402414D0000003F01", + "error":"Cannot read all data. Bytes read: 9. Bytes expected: 65.: (at row 1)\n" + }, "printable": False, }, "RowBinaryWithNamesAndTypes": { @@ -3921,7 +3982,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message b"\x05\x02\x69\x64\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x04\x76\x61\x6c\x31\x04\x76\x61\x6c\x32\x04\x76\x61\x6c\x33\x05\x49\x6e\x74\x36\x34\x06\x53\x74\x72\x69\x6e\x67\x06\x53\x74\x72\x69\x6e\x67\x07\x46\x6c\x6f\x61\x74\x33\x32\x05\x55\x49\x6e\x74\x38\x00\x00\x00\x00\x00\x00\x00\x00\x03\x42\x41\x44\x02\x41\x4d\x00\x00\x00\x3f\x01", ], - "expected": '{"raw_message":"0502696407626C6F636B4E6F0476616C310476616C320476616C3305496E74363406537472696E6706537472696E6707466C6F617433320555496E743800000000000000000342414402414D0000003F01","error":"Type of \'blockNo\' must be UInt16, not String"}', + "expected": { + "raw_message":"0502696407626C6F636B4E6F0476616C310476616C320476616C3305496E74363406537472696E6706537472696E6707466C6F617433320555496E743800000000000000000342414402414D0000003F01", + "error":"Type of 'blockNo' must be UInt16, not String" + }, "printable": False, }, "ORC": { @@ -3932,15 +3996,19 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message b"\x4f\x52\x43\x0a\x0b\x0a\x03\x00\x00\x00\x12\x04\x08\x01\x50\x00\x0a\x15\x0a\x05\x00\x00\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x0a\x12\x0a\x06\x00\x00\x00\x00\x00\x00\x12\x08\x08\x01\x42\x02\x08\x06\x50\x00\x0a\x12\x0a\x06\x00\x00\x00\x00\x00\x00\x12\x08\x08\x01\x42\x02\x08\x04\x50\x00\x0a\x29\x0a\x04\x00\x00\x00\x00\x12\x21\x08\x01\x1a\x1b\x09\x00\x00\x00\x00\x00\x00\xe0\x3f\x11\x00\x00\x00\x00\x00\x00\xe0\x3f\x19\x00\x00\x00\x00\x00\x00\xe0\x3f\x50\x00\x0a\x15\x0a\x05\x00\x00\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\xff\x80\xff\x80\xff\x00\xff\x80\xff\x03\x42\x41\x44\xff\x80\xff\x02\x41\x4d\xff\x80\x00\x00\x00\x3f\xff\x80\xff\x01\x0a\x06\x08\x06\x10\x00\x18\x0d\x0a\x06\x08\x06\x10\x01\x18\x17\x0a\x06\x08\x06\x10\x02\x18\x14\x0a\x06\x08\x06\x10\x03\x18\x14\x0a\x06\x08\x06\x10\x04\x18\x2b\x0a\x06\x08\x06\x10\x05\x18\x17\x0a\x06\x08\x00\x10\x00\x18\x02\x0a\x06\x08\x00\x10\x01\x18\x02\x0a\x06\x08\x01\x10\x01\x18\x02\x0a\x06\x08\x00\x10\x02\x18\x02\x0a\x06\x08\x02\x10\x02\x18\x02\x0a\x06\x08\x01\x10\x02\x18\x03\x0a\x06\x08\x00\x10\x03\x18\x02\x0a\x06\x08\x02\x10\x03\x18\x02\x0a\x06\x08\x01\x10\x03\x18\x02\x0a\x06\x08\x00\x10\x04\x18\x02\x0a\x06\x08\x01\x10\x04\x18\x04\x0a\x06\x08\x00\x10\x05\x18\x02\x0a\x06\x08\x01\x10\x05\x18\x02\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x1a\x03\x47\x4d\x54\x0a\x59\x0a\x04\x08\x01\x50\x00\x0a\x0c\x08\x01\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x0a\x08\x08\x01\x42\x02\x08\x06\x50\x00\x0a\x08\x08\x01\x42\x02\x08\x04\x50\x00\x0a\x21\x08\x01\x1a\x1b\x09\x00\x00\x00\x00\x00\x00\xe0\x3f\x11\x00\x00\x00\x00\x00\x00\xe0\x3f\x19\x00\x00\x00\x00\x00\x00\xe0\x3f\x50\x00\x0a\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\x08\x03\x10\xec\x02\x1a\x0c\x08\x03\x10\x8e\x01\x18\x1d\x20\xc1\x01\x28\x01\x22\x2e\x08\x0c\x12\x05\x01\x02\x03\x04\x05\x1a\x02\x69\x64\x1a\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x1a\x04\x76\x61\x6c\x31\x1a\x04\x76\x61\x6c\x32\x1a\x04\x76\x61\x6c\x33\x20\x00\x28\x00\x30\x00\x22\x08\x08\x04\x20\x00\x28\x00\x30\x00\x22\x08\x08\x08\x20\x00\x28\x00\x30\x00\x22\x08\x08\x08\x20\x00\x28\x00\x30\x00\x22\x08\x08\x05\x20\x00\x28\x00\x30\x00\x22\x08\x08\x01\x20\x00\x28\x00\x30\x00\x30\x01\x3a\x04\x08\x01\x50\x00\x3a\x0c\x08\x01\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x3a\x08\x08\x01\x42\x02\x08\x06\x50\x00\x3a\x08\x08\x01\x42\x02\x08\x04\x50\x00\x3a\x21\x08\x01\x1a\x1b\x09\x00\x00\x00\x00\x00\x00\xe0\x3f\x11\x00\x00\x00\x00\x00\x00\xe0\x3f\x19\x00\x00\x00\x00\x00\x00\xe0\x3f\x50\x00\x3a\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\x40\x90\x4e\x48\x01\x08\xd5\x01\x10\x00\x18\x80\x80\x04\x22\x02\x00\x0b\x28\x5b\x30\x06\x82\xf4\x03\x03\x4f\x52\x43\x18", ], - "expected": r"""{"raw_message":"4F52430A0B0A030000001204080150000A150A050000000000120C0801120608001000180050000A120A06000000000000120808014202080650000A120A06000000000000120808014202080450000A290A0400000000122108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50000A150A050000000000120C080112060802100218025000FF80FF80FF00FF80FF03424144FF80FF02414DFF800000003FFF80FF010A0608061000180D0A060806100118170A060806100218140A060806100318140A0608061004182B0A060806100518170A060800100018020A060800100118020A060801100118020A060800100218020A060802100218020A060801100218030A060800100318020A060802100318020A060801100318020A060800100418020A060801100418040A060800100518020A060801100518021204080010001204080010001204080010001204080010001204080010001204080010001A03474D540A590A04080150000A0C0801120608001000180050000A0808014202080650000A0808014202080450000A2108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50000A0C080112060802100218025000080310EC021A0C0803108E01181D20C1012801222E080C120501020304051A0269641A07626C6F636B4E6F1A0476616C311A0476616C321A0476616C33200028003000220808042000280030002208080820002800300022080808200028003000220808052000280030002208080120002800300030013A04080150003A0C0801120608001000180050003A0808014202080650003A0808014202080450003A2108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50003A0C08011206080210021802500040904E480108D5011000188080042202000B285B300682F403034F524318","error":"Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero\/NULL instead of throwing exception."}""", + "expected": { + "raw_message":"4F52430A0B0A030000001204080150000A150A050000000000120C0801120608001000180050000A120A06000000000000120808014202080650000A120A06000000000000120808014202080450000A290A0400000000122108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50000A150A050000000000120C080112060802100218025000FF80FF80FF00FF80FF03424144FF80FF02414DFF800000003FFF80FF010A0608061000180D0A060806100118170A060806100218140A060806100318140A0608061004182B0A060806100518170A060800100018020A060800100118020A060801100118020A060800100218020A060802100218020A060801100218030A060800100318020A060802100318020A060801100318020A060800100418020A060801100418040A060800100518020A060801100518021204080010001204080010001204080010001204080010001204080010001204080010001A03474D540A590A04080150000A0C0801120608001000180050000A0808014202080650000A0808014202080450000A2108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50000A0C080112060802100218025000080310EC021A0C0803108E01181D20C1012801222E080C120501020304051A0269641A07626C6F636B4E6F1A0476616C311A0476616C321A0476616C33200028003000220808042000280030002208080820002800300022080808200028003000220808052000280030002208080120002800300030013A04080150003A0C0801120608001000180050003A0808014202080650003A0808014202080450003A2108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50003A0C08011206080210021802500040904E480108D5011000188080042202000B285B300682F403034F524318", + "error":"Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero/NULL instead of throwing exception." + }, "printable": False, }, } topic_name_prefix = "format_tests_4_stream_" + topic_name_postfix = get_topic_postfix(create_query_generator) for format_name, format_opts in list(all_formats.items()): logging.debug(f"Set up {format_name}") - topic_name = f"{topic_name_prefix}{format_name}" + topic_name = f"{topic_name_prefix}{format_name}{topic_name_postfix}" data_sample = format_opts["data_sample"] data_prefix = [] raw_message = "_raw_message" @@ -3950,23 +4018,22 @@ def test_kafka_formats_with_broken_message(kafka_cluster): if format_opts.get("printable", False) == False: raw_message = "hex(_raw_message)" kafka_produce(kafka_cluster, topic_name, data_prefix + data_sample) + create_query = create_query_generator( + f"kafka_{format_name}", + "id Int64, blockNo UInt16, val1 String, val2 Float32, val3 UInt8", + topic_list=topic_name, + consumer_group=topic_name, + format=format_name, + settings={ + "kafka_handle_error_mode": "stream", + "kafka_flush_interval_ms": 1000, + } + ) instance.query( - """ + f""" DROP TABLE IF EXISTS test.kafka_{format_name}; - CREATE TABLE test.kafka_{format_name} ( - id Int64, - blockNo UInt16, - val1 String, - val2 Float32, - val3 UInt8 - ) ENGINE = Kafka() - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic_name}', - kafka_group_name = '{topic_name}', - kafka_format = '{format_name}', - kafka_handle_error_mode = 'stream', - kafka_flush_interval_ms = 1000 {extra_settings}; + {create_query}; DROP TABLE IF EXISTS test.kafka_data_{format_name}_mv; CREATE MATERIALIZED VIEW test.kafka_data_{format_name}_mv Engine=Log AS @@ -3977,12 +4044,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): CREATE MATERIALIZED VIEW test.kafka_errors_{format_name}_mv Engine=Log AS SELECT {raw_message} as raw_message, _error as error, _topic as topic, _partition as partition, _offset as offset FROM test.kafka_{format_name} WHERE length(_error) > 0; - """.format( - topic_name=topic_name, - format_name=format_name, - raw_message=raw_message, - extra_settings=format_opts.get("extra_settings") or "", - ) + """ ) raw_expected = """\ @@ -4017,7 +4079,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): for format_name, format_opts in list(all_formats.items()): logging.debug(f"Checking {format_name}") - topic_name = f"{topic_name_prefix}{format_name}" + topic_name = f"{topic_name_prefix}{format_name}{topic_name_postfix}" # shift offsets by 1 if format supports empty value offsets = ( [1, 2, 3] if format_opts.get("supports_empty_value", False) else [0, 1, 2] @@ -4037,229 +4099,201 @@ def test_kafka_formats_with_broken_message(kafka_cluster): assert TSV(result) == TSV(expected), "Proper result for format: {}".format( format_name ) - errors_result = ast.literal_eval( + errors_result = json.loads( instance.query( "SELECT raw_message, error FROM test.kafka_errors_{format_name}_mv format JSONEachRow".format( format_name=format_name ) ) ) - errors_expected = ast.literal_eval(format_opts["expected"]) # print(errors_result.strip()) # print(errors_expected.strip()) assert ( - errors_result["raw_message"] == errors_expected["raw_message"] + errors_result["raw_message"] == format_opts["expected"]["raw_message"] ), "Proper raw_message for format: {}".format(format_name) # Errors text can change, just checking prefixes assert ( - errors_expected["error"] in errors_result["error"] + format_opts["expected"]["error"] in errors_result["error"] ), "Proper error for format: {}".format(format_name) kafka_delete_topic(admin_client, topic_name) -def wait_for_new_data(table_name, prev_count=0, max_retries=120): - retries = 0 - while True: - new_count = int(instance.query("SELECT count() FROM {}".format(table_name))) - print(new_count) - if new_count > prev_count: - return new_count - else: - retries += 1 - time.sleep(0.5) - if retries > max_retries: - raise Exception("No new data :(") +@pytest.mark.parametrize('create_query_generator', [ + generate_old_create_table_query, + # generate_new_create_table_query TODO(antaljanosbenjamin): crashes CH +]) +def test_kafka_consumer_failover(kafka_cluster, create_query_generator): + topic_name = "kafka_consumer_failover" + get_topic_postfix(create_query_generator) + with kafka_topic(get_admin_client(kafka_cluster), topic_name, num_partitions=2): + consumer_group = f"{topic_name}_group" + create_queries = [] + for counter in range(3): + create_queries.append(create_query_generator( + f"kafka{counter+1}", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=consumer_group, + settings={ + "kafka_max_block_size": 1, + "kafka_poll_timeout_ms": 200, + } + )) -def test_kafka_consumer_failover(kafka_cluster): - # for backporting: - # admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) + instance.query( + f""" + {create_queries[0]}; + {create_queries[1]}; + {create_queries[2]}; - topic_name = "kafka_consumer_failover" - kafka_create_topic(admin_client, topic_name, num_partitions=2) + CREATE TABLE test.destination ( + key UInt64, + value UInt64, + _consumed_by LowCardinality(String) + ) + ENGINE = MergeTree() + ORDER BY key; - instance.query( - """ - DROP TABLE IF EXISTS test.kafka; - DROP TABLE IF EXISTS test.kafka2; + CREATE MATERIALIZED VIEW test.kafka1_mv TO test.destination AS + SELECT key, value, 'kafka1' as _consumed_by + FROM test.kafka1; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'kafka_consumer_failover', - kafka_group_name = 'kafka_consumer_failover_group', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 1, - kafka_poll_timeout_ms = 200; + CREATE MATERIALIZED VIEW test.kafka2_mv TO test.destination AS + SELECT key, value, 'kafka2' as _consumed_by + FROM test.kafka2; - CREATE TABLE test.kafka2 (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'kafka_consumer_failover', - kafka_group_name = 'kafka_consumer_failover_group', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 1, - kafka_poll_timeout_ms = 200; - - CREATE TABLE test.kafka3 (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'kafka_consumer_failover', - kafka_group_name = 'kafka_consumer_failover_group', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 1, - kafka_poll_timeout_ms = 200; - - CREATE TABLE test.destination ( - key UInt64, - value UInt64, - _consumed_by LowCardinality(String) + CREATE MATERIALIZED VIEW test.kafka3_mv TO test.destination AS + SELECT key, value, 'kafka3' as _consumed_by + FROM test.kafka3; + """ ) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.kafka_mv TO test.destination AS - SELECT key, value, 'kafka' as _consumed_by - FROM test.kafka; + producer = KafkaProducer( + bootstrap_servers="localhost:{}".format(cluster.kafka_port), + value_serializer=producer_serializer, + key_serializer=producer_serializer, + ) - CREATE MATERIALIZED VIEW test.kafka2_mv TO test.destination AS - SELECT key, value, 'kafka2' as _consumed_by - FROM test.kafka2; + ## all 3 attached, 2 working + producer.send( + topic=topic_name, + value=json.dumps({"key": 1, "value": 1}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 1, "value": 1}), + partition=1, + ) + producer.flush() - CREATE MATERIALIZED VIEW test.kafka3_mv TO test.destination AS - SELECT key, value, 'kafka3' as _consumed_by - FROM test.kafka3; - """ - ) + count_query = "SELECT count() FROM test.destination" + prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > 0) - producer = KafkaProducer( - bootstrap_servers="localhost:{}".format(cluster.kafka_port), - value_serializer=producer_serializer, - key_serializer=producer_serializer, - ) + ## 2 attached, 2 working + instance.query("DETACH TABLE test.kafka1") + producer.send( + topic=topic_name, + value=json.dumps({"key": 2, "value": 2}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 2, "value": 2}), + partition=1, + ) + producer.flush() + prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > prev_count) - ## all 3 attached, 2 working - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 1, "value": 1}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 1, "value": 1}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination") + ## 1 attached, 1 working + instance.query("DETACH TABLE test.kafka2") + producer.send( + topic=topic_name, + value=json.dumps({"key": 3, "value": 3}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 3, "value": 3}), + partition=1, + ) + producer.flush() + prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > prev_count) - ## 2 attached, 2 working - instance.query("DETACH TABLE test.kafka") - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 2, "value": 2}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 2, "value": 2}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination", prev_count) + ## 2 attached, 2 working + instance.query("ATTACH TABLE test.kafka1") + producer.send( + topic=topic_name, + value=json.dumps({"key": 4, "value": 4}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 4, "value": 4}), + partition=1, + ) + producer.flush() + prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > prev_count) - ## 1 attached, 1 working - instance.query("DETACH TABLE test.kafka2") - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 3, "value": 3}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 3, "value": 3}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination", prev_count) + ## 1 attached, 1 working + instance.query("DETACH TABLE test.kafka3") + producer.send( + topic=topic_name, + value=json.dumps({"key": 5, "value": 5}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 5, "value": 5}), + partition=1, + ) + producer.flush() + prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > prev_count) - ## 2 attached, 2 working - instance.query("ATTACH TABLE test.kafka") - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 4, "value": 4}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 4, "value": 4}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination", prev_count) + ## 2 attached, 2 working + instance.query("ATTACH TABLE test.kafka2") + producer.send( + topic=topic_name, + value=json.dumps({"key": 6, "value": 6}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 6, "value": 6}), + partition=1, + ) + producer.flush() + prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > prev_count) - ## 1 attached, 1 working - instance.query("DETACH TABLE test.kafka3") - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 5, "value": 5}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 5, "value": 5}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination", prev_count) + ## 3 attached, 2 working + instance.query("ATTACH TABLE test.kafka3") + producer.send( + topic=topic_name, + value=json.dumps({"key": 7, "value": 7}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 7, "value": 7}), + partition=1, + ) + producer.flush() + prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > prev_count) - ## 2 attached, 2 working - instance.query("ATTACH TABLE test.kafka2") - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 6, "value": 6}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 6, "value": 6}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination", prev_count) - - ## 3 attached, 2 working - instance.query("ATTACH TABLE test.kafka3") - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 7, "value": 7}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 7, "value": 7}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination", prev_count) - - ## 2 attached, same 2 working - instance.query("DETACH TABLE test.kafka3") - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 8, "value": 8}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 8, "value": 8}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination", prev_count) - kafka_delete_topic(admin_client, topic_name) + ## 2 attached, same 2 working + instance.query("DETACH TABLE test.kafka3") + producer.send( + topic=topic_name, + value=json.dumps({"key": 8, "value": 8}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 8, "value": 8}), + partition=1, + ) + producer.flush() + prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > prev_count) def test_kafka_predefined_configuration(kafka_cluster): @@ -4289,269 +4323,238 @@ def test_kafka_predefined_configuration(kafka_cluster): # https://github.com/ClickHouse/ClickHouse/issues/26643 -def test_issue26643(kafka_cluster): - # for backporting: - # admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_issue26643(kafka_cluster, create_query_generator): producer = KafkaProducer( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), value_serializer=producer_serializer, ) + topic_name = "test_issue26643" + get_topic_postfix(create_query_generator) + thread_per_consumer = must_use_thread_per_consumer(create_query_generator) - topic_list = [] - topic_list.append( - NewTopic(name="test_issue26643", num_partitions=4, replication_factor=1) - ) - admin_client.create_topics(new_topics=topic_list, validate_only=False) - - msg = message_with_repeated_pb2.Message( - tnow=1629000000, - server="server1", - clien="host1", - sPort=443, - cPort=50000, - r=[ - message_with_repeated_pb2.dd( - name="1", type=444, ttl=123123, data=b"adsfasd" - ), - message_with_repeated_pb2.dd(name="2"), - ], - method="GET", - ) - - data = b"" - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - - msg = message_with_repeated_pb2.Message(tnow=1629000002) - - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - - producer.send(topic="test_issue26643", value=data) - - data = _VarintBytes(len(serialized_msg)) + serialized_msg - producer.send(topic="test_issue26643", value=data) - producer.flush() - - instance.query( - """ - CREATE TABLE IF NOT EXISTS test.test_queue - ( - `tnow` UInt32, - `server` String, - `client` String, - `sPort` UInt16, - `cPort` UInt16, - `r.name` Array(String), - `r.class` Array(UInt16), - `r.type` Array(UInt16), - `r.ttl` Array(UInt32), - `r.data` Array(String), - `method` String + with kafka_topic(get_admin_client(kafka_cluster), topic_name): + msg = message_with_repeated_pb2.Message( + tnow=1629000000, + server="server1", + clien="host1", + sPort=443, + cPort=50000, + r=[ + message_with_repeated_pb2.dd( + name="1", type=444, ttl=123123, data=b"adsfasd" + ), + message_with_repeated_pb2.dd(name="2"), + ], + method="GET", ) - ENGINE = Kafka - SETTINGS - kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'test_issue26643', - kafka_group_name = 'test_issue26643_group', - kafka_format = 'Protobuf', - kafka_schema = 'message_with_repeated.proto:Message', - kafka_num_consumers = 4, - kafka_skip_broken_messages = 10000; - SET allow_suspicious_low_cardinality_types=1; + data = b"" + serialized_msg = msg.SerializeToString() + data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - CREATE TABLE test.log - ( - `tnow` DateTime('Asia/Istanbul') CODEC(DoubleDelta, LZ4), - `server` LowCardinality(String), - `client` LowCardinality(String), - `sPort` LowCardinality(UInt16), - `cPort` UInt16 CODEC(T64, LZ4), - `r.name` Array(String), - `r.class` Array(LowCardinality(UInt16)), - `r.type` Array(LowCardinality(UInt16)), - `r.ttl` Array(LowCardinality(UInt32)), - `r.data` Array(String), - `method` LowCardinality(String) + msg = message_with_repeated_pb2.Message(tnow=1629000002) + + serialized_msg = msg.SerializeToString() + data = data + _VarintBytes(len(serialized_msg)) + serialized_msg + + producer.send(topic_name, value=data) + + data = _VarintBytes(len(serialized_msg)) + serialized_msg + producer.send(topic_name, value=data) + producer.flush() + + create_query = create_query_generator( + "test_queue", + """`tnow` UInt32, + `server` String, + `client` String, + `sPort` UInt16, + `cPort` UInt16, + `r.name` Array(String), + `r.class` Array(UInt16), + `r.type` Array(UInt16), + `r.ttl` Array(UInt32), + `r.data` Array(String), + `method` String""", + topic_list=topic_name, + consumer_group=f"{topic_name}_group", + format="Protobuf", + settings={ + "kafka_schema": "message_with_repeated.proto:Message", + "kafka_skip_broken_messages": 10000, + "kafka_thread_per_consumer": thread_per_consumer, + } ) - ENGINE = MergeTree - PARTITION BY toYYYYMMDD(tnow) - ORDER BY (tnow, server) - TTL toDate(tnow) + toIntervalMonth(1000) - SETTINGS index_granularity = 16384, merge_with_ttl_timeout = 7200; - CREATE MATERIALIZED VIEW test.test_consumer TO test.log AS - SELECT - toDateTime(a.tnow) AS tnow, - a.server AS server, - a.client AS client, - a.sPort AS sPort, - a.cPort AS cPort, - a.`r.name` AS `r.name`, - a.`r.class` AS `r.class`, - a.`r.type` AS `r.type`, - a.`r.ttl` AS `r.ttl`, - a.`r.data` AS `r.data`, - a.method AS method - FROM test.test_queue AS a; - """ - ) + instance.query( + f""" + {create_query}; - instance.wait_for_log_line("Committed offset") - result = instance.query("SELECT * FROM test.log") + SET allow_suspicious_low_cardinality_types=1; - expected = """\ -2021-08-15 07:00:00 server1 443 50000 ['1','2'] [0,0] [444,0] [123123,0] ['adsfasd',''] GET -2021-08-15 07:00:02 0 0 [] [] [] [] [] -2021-08-15 07:00:02 0 0 [] [] [] [] [] -""" - assert TSV(result) == TSV(expected) + CREATE TABLE test.log + ( + `tnow` DateTime('Asia/Istanbul') CODEC(DoubleDelta, LZ4), + `server` LowCardinality(String), + `client` LowCardinality(String), + `sPort` LowCardinality(UInt16), + `cPort` UInt16 CODEC(T64, LZ4), + `r.name` Array(String), + `r.class` Array(LowCardinality(UInt16)), + `r.type` Array(LowCardinality(UInt16)), + `r.ttl` Array(LowCardinality(UInt32)), + `r.data` Array(String), + `method` LowCardinality(String) + ) + ENGINE = MergeTree + PARTITION BY toYYYYMMDD(tnow) + ORDER BY (tnow, server) + TTL toDate(tnow) + toIntervalMonth(1000) + SETTINGS index_granularity = 16384, merge_with_ttl_timeout = 7200; - # kafka_cluster.open_bash_shell('instance') + CREATE MATERIALIZED VIEW test.test_consumer TO test.log AS + SELECT + toDateTime(a.tnow) AS tnow, + a.server AS server, + a.client AS client, + a.sPort AS sPort, + a.cPort AS cPort, + a.`r.name` AS `r.name`, + a.`r.class` AS `r.class`, + a.`r.type` AS `r.type`, + a.`r.ttl` AS `r.ttl`, + a.`r.data` AS `r.data`, + a.method AS method + FROM test.test_queue AS a; + """ + ) + + instance.wait_for_log_line("Committed offset") + result = instance.query("SELECT * FROM test.log") + + expected = """\ + 2021-08-15 07:00:00 server1 443 50000 ['1','2'] [0,0] [444,0] [123123,0] ['adsfasd',''] GET + 2021-08-15 07:00:02 0 0 [] [] [] [] [] + 2021-08-15 07:00:02 0 0 [] [] [] [] [] + """ + assert TSV(result) == TSV(expected) -def test_num_consumers_limit(kafka_cluster): + +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_num_consumers_limit(kafka_cluster, create_query_generator): instance.query("DROP TABLE IF EXISTS test.kafka") - error = instance.query_and_get_error( - """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_old}', '{kafka_group_name_old}', '{kafka_format_json_each_row}', '\\n', '', 100) - SETTINGS kafka_commit_on_select = 1; - """ - ) + thread_per_consumer = must_use_thread_per_consumer(create_query_generator) - assert "BAD_ARGUMENTS" in error + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + settings={ + "kafka_num_consumers": 100, + "kafka_thread_per_consumer": thread_per_consumer + } + ) + error = instance.query_and_get_error(create_query) + + assert "BAD_ARGUMENTS" in error and "The number of consumers can not be bigger than" in error instance.query( - """ + f""" SET kafka_disable_num_consumers_limit = 1; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_old}', '{kafka_group_name_old}', '{kafka_format_json_each_row}', '\\n', '', 100) - SETTINGS kafka_commit_on_select = 1; + {create_query}; """ ) instance.query("DROP TABLE test.kafka") -def test_format_with_prefix_and_suffix(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_format_with_prefix_and_suffix(kafka_cluster, create_query_generator): + topic_name = "custom" + get_topic_postfix(create_query_generator) - kafka_create_topic(admin_client, "custom") + with kafka_topic(get_admin_client(kafka_cluster), topic_name): + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format="CustomSeparated" + ) + instance.query( + f""" + DROP TABLE IF EXISTS test.kafka; + {create_query}; + """ + ) - instance.query( + instance.query( + "INSERT INTO test.kafka select number*10 as key, number*100 as value from numbers(2) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" + ) + + message_count = 2 + messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count) + + assert len(messages) == 2 + + assert ( + "".join(messages) == "\n0\t0\n\n\n10\t100\n\n" + ) + + +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_max_rows_per_message(kafka_cluster, create_query_generator): + topic_name = "custom_max_rows_per_message" + get_topic_postfix(create_query_generator) + + with kafka_topic(get_admin_client(kafka_cluster), topic_name): + num_rows = 5 + + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format="CustomSeparated", + settings={ + "format_custom_result_before_delimiter": "\n", + "format_custom_result_after_delimiter": "\n", + "kafka_max_rows_per_message": 3, + } + ) + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.kafka; + {create_query}; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.kafka; """ - DROP TABLE IF EXISTS test.kafka; + ) - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'custom', - kafka_group_name = 'custom', - kafka_format = 'CustomSeparated'; - """ - ) + instance.query( + f"INSERT INTO test.kafka select number*10 as key, number*100 as value from numbers({num_rows}) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" + ) - instance.query( - "INSERT INTO test.kafka select number*10 as key, number*100 as value from numbers(2) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" - ) + message_count = 2 + messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count) - messages = [] + assert len(messages) == message_count - attempt = 0 - while attempt < 100: - messages.extend(kafka_consume(kafka_cluster, "custom")) - if len(messages) == 2: - break - attempt += 1 + assert ( + "".join(messages) + == "\n0\t0\n10\t100\n20\t200\n\n\n30\t300\n40\t400\n\n" + ) - assert len(messages) == 2 + instance.query_with_retry("SELECT count() FROM test.view", check_callback=lambda res: int(res) == num_rows) - assert ( - "".join(messages) == "\n0\t0\n\n\n10\t100\n\n" - ) - - kafka_delete_topic(admin_client, "custom") + result = instance.query("SELECT * FROM test.view") + assert result == "0\t0\n10\t100\n20\t200\n30\t300\n40\t400\n" -def test_max_rows_per_message(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - - topic = "custom_max_rows_per_message" - - kafka_create_topic(admin_client, topic) - - num_rows = 5 - - instance.query( - f""" - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.kafka; - - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic}', - kafka_group_name = '{topic}', - kafka_format = 'CustomSeparated', - format_custom_result_before_delimiter = '\n', - format_custom_result_after_delimiter = '\n', - kafka_max_rows_per_message = 3; - - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.kafka; - """ - ) - - instance.query( - f"INSERT INTO test.kafka select number*10 as key, number*100 as value from numbers({num_rows}) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" - ) - - messages = [] - - attempt = 0 - while attempt < 500: - messages.extend(kafka_consume(kafka_cluster, topic)) - if len(messages) == 2: - break - attempt += 1 - - assert len(messages) == 2 - - assert ( - "".join(messages) - == "\n0\t0\n10\t100\n20\t200\n\n\n30\t300\n40\t400\n\n" - ) - - attempt = 0 - rows = 0 - while attempt < 500: - rows = int(instance.query("SELECT count() FROM test.view")) - if rows == num_rows: - break - attempt += 1 - - assert rows == num_rows - - result = instance.query("SELECT * FROM test.view") - assert result == "0\t0\n10\t100\n20\t200\n30\t300\n40\t400\n" - - kafka_delete_topic(admin_client, topic) - - -def test_row_based_formats(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_row_based_formats(kafka_cluster, create_query_generator): + admin_client = get_admin_client(kafka_cluster) for format_name in [ "TSV", @@ -4571,121 +4574,99 @@ def test_row_based_formats(kafka_cluster): "RowBinaryWithNamesAndTypes", "MsgPack", ]: - print(format_name) + logging.debug("Checking {format_name}") - kafka_create_topic(admin_client, format_name) + topic_name = format_name + get_topic_postfix(create_query_generator) - num_rows = 10 + with kafka_topic(admin_client, topic_name): + num_rows = 10 + max_rows_per_message = 5 + message_count = num_rows / max_rows_per_message + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format=format_name, + settings={"kafka_max_rows_per_message": max_rows_per_message} + ) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.kafka; + + {create_query}; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.kafka; + + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}); + """ + ) + + messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count, need_decode=False) + + assert len(messages) == message_count + + instance.query_with_retry("SELECT count() FROM test.view", check_callback=lambda res: int(res) == num_rows) + + result = instance.query("SELECT * FROM test.view") + expected = "" + for i in range(num_rows): + expected += str(i * 10) + "\t" + str(i * 100) + "\n" + assert result == expected + + +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_block_based_formats_1(kafka_cluster, create_query_generator): + topic_name = "pretty_space" + get_topic_postfix(create_query_generator) + + with kafka_topic(get_admin_client(kafka_cluster), topic_name): + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format="PrettySpace" + ) instance.query( f""" - DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.kafka; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{format_name}', - kafka_group_name = '{format_name}', - kafka_format = '{format_name}', - kafka_max_rows_per_message = 5; + {create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.kafka; - - INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}); + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers(5) settings max_block_size=2, optimize_trivial_insert_select=0, output_format_pretty_color=1, output_format_pretty_row_numbers=0; """ ) - messages = [] + message_count = 3 + messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count) + assert len(messages) == 3 - attempt = 0 - while attempt < 500: - messages.extend(kafka_consume(kafka_cluster, format_name, needDecode=False)) - if len(messages) == 2: - break - attempt += 1 + data = [] + for message in messages: + splitted = message.split("\n") + assert splitted[0] == " \x1b[1mkey\x1b[0m \x1b[1mvalue\x1b[0m" + assert splitted[1] == "" + assert splitted[-1] == "" + data += [line.split() for line in splitted[2:-1]] - assert len(messages) == 2 - - attempt = 0 - rows = 0 - while attempt < 500: - rows = int(instance.query("SELECT count() FROM test.view")) - if rows == num_rows: - break - attempt += 1 - - assert rows == num_rows - - result = instance.query("SELECT * FROM test.view") - expected = "" - for i in range(num_rows): - expected += str(i * 10) + "\t" + str(i * 100) + "\n" - assert result == expected - - kafka_delete_topic(admin_client, format_name) + assert data == [ + ["0", "0"], + ["10", "100"], + ["20", "200"], + ["30", "300"], + ["40", "400"], + ] -def test_block_based_formats_1(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - - topic = "pretty_space" - kafka_create_topic(admin_client, topic) - - instance.query( - f""" - DROP TABLE IF EXISTS test.kafka; - - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic}', - kafka_group_name = '{topic}', - kafka_format = 'PrettySpace'; - - INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers(5) settings max_block_size=2, optimize_trivial_insert_select=0, output_format_pretty_color=1, output_format_pretty_row_numbers=0; - """ - ) - - messages = [] - - attempt = 0 - while attempt < 500: - messages.extend(kafka_consume(kafka_cluster, topic)) - if len(messages) == 3: - break - attempt += 1 - - assert len(messages) == 3 - - data = [] - for message in messages: - splitted = message.split("\n") - assert splitted[0] == " \x1b[1mkey\x1b[0m \x1b[1mvalue\x1b[0m" - assert splitted[1] == "" - assert splitted[-1] == "" - data += [line.split() for line in splitted[2:-1]] - - assert data == [ - ["0", "0"], - ["10", "100"], - ["20", "200"], - ["30", "300"], - ["40", "400"], - ] - - kafka_delete_topic(admin_client, topic) - - -def test_block_based_formats_2(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_block_based_formats_2(kafka_cluster, create_query_generator): + admin_client = get_admin_client(kafka_cluster) num_rows = 100 + message_count = 9 for format_name in [ "JSONColumns", @@ -4695,55 +4676,41 @@ def test_block_based_formats_2(kafka_cluster): "ORC", "JSONCompactColumns", ]: - kafka_create_topic(admin_client, format_name) + topic_name = format_name + get_topic_postfix(create_query_generator) + with kafka_topic(admin_client, topic_name): + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format=format_name + ) - instance.query( - f""" - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.kafka; + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.kafka; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{format_name}', - kafka_group_name = '{format_name}', - kafka_format = '{format_name}'; + {create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.kafka; + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.kafka; - INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; - """ - ) + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; + """ + ) + messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count, need_decode=False) + assert len(messages) == message_count - messages = [] + rows = int(instance.query_with_retry("SELECT count() FROM test.view", check_callback=lambda res: int(res) == num_rows)) - attempt = 0 - while attempt < 500: - messages.extend(kafka_consume(kafka_cluster, format_name, needDecode=False)) - if len(messages) == 9: - break - attempt += 1 + assert rows == num_rows - assert len(messages) == 9 - - attempt = 0 - rows = 0 - while attempt < 500: - rows = int(instance.query("SELECT count() FROM test.view")) - if rows == num_rows: - break - attempt += 1 - - assert rows == num_rows - - result = instance.query("SELECT * FROM test.view ORDER by key") - expected = "" - for i in range(num_rows): - expected += str(i * 10) + "\t" + str(i * 100) + "\n" - assert result == expected - - kafka_delete_topic(admin_client, format_name) + result = instance.query("SELECT * FROM test.view ORDER by key") + expected = "" + for i in range(num_rows): + expected += str(i * 10) + "\t" + str(i * 100) + "\n" + assert result == expected def test_system_kafka_consumers(kafka_cluster): @@ -5081,137 +5048,124 @@ def test_formats_errors(kafka_cluster): "HiveText", "MySQLDump", ]: - kafka_create_topic(admin_client, format_name) - table_name = f"kafka_{format_name}" + with kafka_topic(admin_client, format_name): + table_name = f"kafka_{format_name}" + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.{table_name}; + + CREATE TABLE test.{table_name} (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{format_name}', + kafka_group_name = '{format_name}', + kafka_format = '{format_name}', + kafka_max_rows_per_message = 5, + format_template_row='template_row.format', + format_regexp='id: (.+?)', + input_format_with_names_use_header=0, + format_schema='key_value_message:Message'; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.{table_name}; + """ + ) + + kafka_produce( + kafka_cluster, + format_name, + ["Broken message\nBroken message\nBroken message\n"], + ) + + num_errors = int( + instance.query_with_retry( + f"SELECT length(exceptions.text) from system.kafka_consumers where database = 'test' and table = '{table_name}'", + check_callback=lambda res: int(res) > 0 + ) + ) + + assert num_errors > 0 + + instance.query(f"DROP TABLE test.{table_name}") + instance.query("DROP TABLE test.view") + + +@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +def test_multiple_read_in_materialized_views(kafka_cluster, create_query_generator): + topic_name = "multiple_read_from_mv" + get_topic_postfix(create_query_generator) + + with kafka_topic(get_admin_client(kafka_cluster), topic_name): + create_query = create_query_generator( + "kafka_multiple_read_input", + "id Int64", + topic_list=topic_name, + consumer_group=topic_name, + ) instance.query( f""" - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.{table_name}; + DROP TABLE IF EXISTS test.kafka_multiple_read_input; + DROP TABLE IF EXISTS test.kafka_multiple_read_table; + DROP TABLE IF EXISTS test.kafka_multiple_read_mv; - CREATE TABLE test.{table_name} (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{format_name}', - kafka_group_name = '{format_name}', - kafka_format = '{format_name}', - kafka_max_rows_per_message = 5, - format_template_row='template_row.format', - format_regexp='id: (.+?)', - input_format_with_names_use_header=0, - format_schema='key_value_message:Message'; + {create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.{table_name}; - """ + CREATE TABLE test.kafka_multiple_read_table (id Int64) + ENGINE = MergeTree + ORDER BY id; + + + CREATE MATERIALIZED VIEW test.kafka_multiple_read_mv TO test.kafka_multiple_read_table AS + SELECT id + FROM test.kafka_multiple_read_input + WHERE id NOT IN ( + SELECT id + FROM test.kafka_multiple_read_table + WHERE id IN ( + SELECT id + FROM test.kafka_multiple_read_input + ) + ); + """ ) kafka_produce( - kafka_cluster, - format_name, - ["Broken message\nBroken message\nBroken message\n"], + kafka_cluster, topic_name, [json.dumps({"id": 42}), json.dumps({"id": 43})] ) - attempt = 0 - num_errors = 0 - while attempt < 200: - num_errors = int( - instance.query( - f"SELECT length(exceptions.text) from system.kafka_consumers where database = 'test' and table = '{table_name}'" - ) - ) - if num_errors > 0: - break - attempt += 1 + expected_result = "42\n43\n" + res = instance.query_with_retry( + f"SELECT id FROM test.kafka_multiple_read_table ORDER BY id", + check_callback=lambda res: res == expected_result, + ) + assert res == expected_result - assert num_errors > 0 + # Verify that the query deduplicates the records as it meant to be + messages = [] + for _ in range(0, 10): + messages.append(json.dumps({"id": 42})) + messages.append(json.dumps({"id": 43})) - kafka_delete_topic(admin_client, format_name) - instance.query(f"DROP TABLE test.{table_name}") - instance.query("DROP TABLE test.view") + messages.append(json.dumps({"id": 44})) + kafka_produce(kafka_cluster, topic_name, messages) -def test_multiple_read_in_materialized_views(kafka_cluster, max_retries=15): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) + expected_result = "42\n43\n44\n" + res = instance.query_with_retry( + f"SELECT id FROM test.kafka_multiple_read_table ORDER BY id", + check_callback=lambda res: res == expected_result, + ) + assert res == expected_result - topic = "multiple_read_from_mv" - kafka_create_topic(admin_client, topic) - - instance.query( - f""" - DROP TABLE IF EXISTS test.kafka_multiple_read_input; - DROP TABLE IF EXISTS test.kafka_multiple_read_table; - DROP TABLE IF EXISTS test.kafka_multiple_read_mv; - - CREATE TABLE test.kafka_multiple_read_input (id Int64) - ENGINE = Kafka - SETTINGS - kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic}', - kafka_group_name = '{topic}', - kafka_format = 'JSONEachRow'; - - CREATE TABLE test.kafka_multiple_read_table (id Int64) - ENGINE = MergeTree - ORDER BY id; - - - CREATE MATERIALIZED VIEW IF NOT EXISTS test.kafka_multiple_read_mv TO test.kafka_multiple_read_table AS - SELECT id - FROM test.kafka_multiple_read_input - WHERE id NOT IN ( - SELECT id - FROM test.kafka_multiple_read_table - WHERE id IN ( - SELECT id - FROM test.kafka_multiple_read_input - ) - ); - """ - ) - - kafka_produce( - kafka_cluster, topic, [json.dumps({"id": 42}), json.dumps({"id": 43})] - ) - - expected_result = "42\n43\n" - res = instance.query_with_retry( - f"SELECT id FROM test.kafka_multiple_read_table ORDER BY id", - retry_count=30, - sleep_time=0.5, - check_callback=lambda res: res == expected_result, - ) - assert res == expected_result - - # Verify that the query deduplicates the records as it meant to be - messages = [] - for i in range(0, 10): - messages.append(json.dumps({"id": 42})) - messages.append(json.dumps({"id": 43})) - - messages.append(json.dumps({"id": 44})) - - kafka_produce(kafka_cluster, topic, messages) - - expected_result = "42\n43\n44\n" - res = instance.query_with_retry( - f"SELECT id FROM test.kafka_multiple_read_table ORDER BY id", - retry_count=30, - sleep_time=0.5, - check_callback=lambda res: res == expected_result, - ) - assert res == expected_result - - kafka_delete_topic(admin_client, topic) - instance.query( - f""" - DROP TABLE test.kafka_multiple_read_input; - DROP TABLE test.kafka_multiple_read_table; - DROP TABLE test.kafka_multiple_read_mv; - """ - ) + instance.query( + f""" + DROP TABLE test.kafka_multiple_read_input; + DROP TABLE test.kafka_multiple_read_table; + DROP TABLE test.kafka_multiple_read_mv; + """ + ) if __name__ == "__main__": From 64af6ec596747d4c20b04d54502b047d3a1d1b3c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 7 Jun 2024 22:44:38 +0000 Subject: [PATCH 082/644] Clean up some TODOs --- src/Storages/Kafka/KafkaConsumer2.cpp | 1 - src/Storages/Kafka/StorageKafka2.cpp | 4 +--- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 9ed698301e5..1320b939612 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -304,7 +304,6 @@ ReadBufferPtr KafkaConsumer2::consume(const TopicPartition & topic_partition, co return next_message; } - // TODO(antaljanosbenjamin): check if we should poll new messages or not while (true) { stalled_status = StalledStatus::NO_MESSAGES_RETURNED; diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index fa5389d606a..911de671fb7 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -815,7 +815,7 @@ StorageKafka2::lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const Topi if (code != Coordination::Error::ZNODEEXISTS) zkutil::KeeperMultiException::check(code, ops, responses); - // TODO(antaljanosbenjamin): maybe check the content, if we have the locks, we can continue with them + // Possible optimization: check the content of logfiles, if we locked them, then we can clean them up and retry to lock them. return std::nullopt; } @@ -1022,7 +1022,6 @@ StorageKafka2::PolledBatchInfo StorageKafka2::pollConsumer( total_rows = total_rows + new_rows; batch_info.last_offset = consumer.currentOffset(); } - // TODO(antaljanosbenjamin): think about this when rebalance is happening, because `isStalled()` will return true else if (consumer.isStalled()) { ++failed_poll_attempts; @@ -1168,7 +1167,6 @@ bool StorageKafka2::streamToViews(size_t idx) if (current_assignment == nullptr) { // The consumer lost its assignment and haven't received a new one. - // TODO(antaljanosbenjamin): returning a proper value representing the state // By returning true this function reports the current consumer as a "stalled" stream, which return true; } From 26851f1d348034b76f6a849796c4020b7d120735 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 7 Jun 2024 22:56:26 +0000 Subject: [PATCH 083/644] Automatic style fix --- tests/integration/test_storage_kafka/test.py | 754 +++++++++++++------ 1 file changed, 511 insertions(+), 243 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 5b7d7f65b9f..ad4e0a0877f 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -155,9 +155,16 @@ def kafka_topic( num_partitions=1, replication_factor=1, max_retries=50, - config=None + config=None, ): - kafka_create_topic(admin_client, topic_name, num_partitions, replication_factor, max_retries, config) + kafka_create_topic( + admin_client, + topic_name, + num_partitions, + replication_factor, + max_retries, + config, + ) try: yield None finally: @@ -174,7 +181,10 @@ def existing_kafka_topic(admin_client, topic_name, max_retries=50): def get_admin_client(kafka_cluster): - return KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) + return KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=15): logging.debug( @@ -230,13 +240,24 @@ def kafka_produce_protobuf_messages(kafka_cluster, topic, start_index, num_messa logging.debug(("Produced {} messages for topic {}".format(num_messages, topic))) - -def kafka_consume_with_retry(kafka_cluster, topic, expected_messages, need_decode=True, timestamp=0, retry_count=20, sleep_time=0.1): +def kafka_consume_with_retry( + kafka_cluster, + topic, + expected_messages, + need_decode=True, + timestamp=0, + retry_count=20, + sleep_time=0.1, +): messages = [] try_count = 0 while try_count < retry_count: try_count += 1 - messages.extend(kafka_consume(kafka_cluster, topic, need_decode=need_decode, timestamp=timestamp)) + messages.extend( + kafka_consume( + kafka_cluster, topic, need_decode=need_decode, timestamp=timestamp + ) + ) if len(messages) == expected_messages: break time.sleep(sleep_time) @@ -244,6 +265,7 @@ def kafka_consume_with_retry(kafka_cluster, topic, expected_messages, need_decod raise Exception(f"Got only {len(messages)} messages") return messages + def kafka_produce_protobuf_messages_no_delimiters( kafka_cluster, topic, start_index, num_messages ): @@ -331,6 +353,7 @@ def avro_confluent_message(schema_registry_client, value): ) return serializer.encode_record_with_schema("test_subject", schema, value) + def create_settings_string(settings): if settings is None: return "" @@ -349,41 +372,43 @@ def create_settings_string(settings): for key in keys: if key == first_key: continue - settings_string +=", " + str(key) + " = " + format_value(settings[key]) + settings_string += ", " + str(key) + " = " + format_value(settings[key]) return settings_string def generate_old_create_table_query( - table_name, - columns_def, - database="test", - brokers="{kafka_broker}:19092", - topic_list="{kafka_topic_new}", - consumer_group="{kafka_group_name_new}", - format="{kafka_format_json_each_row}", - row_delimiter= "\\n", - keeper_path=None, # it is not used, but it is easier to handle keeper_path and replica_name like this - replica_name=None, - settings=None): - - settings_string=create_settings_string(settings) + table_name, + columns_def, + database="test", + brokers="{kafka_broker}:19092", + topic_list="{kafka_topic_new}", + consumer_group="{kafka_group_name_new}", + format="{kafka_format_json_each_row}", + row_delimiter="\\n", + keeper_path=None, # it is not used, but it is easier to handle keeper_path and replica_name like this + replica_name=None, + settings=None, +): + settings_string = create_settings_string(settings) query = f"""CREATE TABLE {database}.{table_name} ({columns_def}) ENGINE = Kafka('{brokers}', '{topic_list}', '{consumer_group}', '{format}', '{row_delimiter}') {settings_string}""" logging.debug(f"Generated old create query: {query}") return query + def generate_new_create_table_query( - table_name, - columns_def, - database="test", - brokers="{kafka_broker}:19092", - topic_list="{kafka_topic_new}", - consumer_group="{kafka_group_name_new}", - format="{kafka_format_json_each_row}", - row_delimiter= "\\n", - keeper_path=None, - replica_name=None, - settings=None): + table_name, + columns_def, + database="test", + brokers="{kafka_broker}:19092", + topic_list="{kafka_topic_new}", + consumer_group="{kafka_group_name_new}", + format="{kafka_format_json_each_row}", + row_delimiter="\\n", + keeper_path=None, + replica_name=None, + settings=None, +): if settings is None: settings = {} if keeper_path is None: @@ -392,13 +417,14 @@ def generate_new_create_table_query( replica_name = "r1" settings["kafka_keeper_path"] = keeper_path settings["kafka_replica_name"] = replica_name - settings_string=create_settings_string(settings) + settings_string = create_settings_string(settings) query = f"""CREATE TABLE {database}.{table_name} ({columns_def}) ENGINE = Kafka('{brokers}', '{topic_list}', '{consumer_group}', '{format}', '{row_delimiter}') {settings_string} SETTINGS allow_experimental_kafka_store_offsets_in_keeper=1""" logging.debug(f"Generated new create query: {query}") return query + def must_use_thread_per_consumer(generator): if generator == generate_old_create_table_query: return False @@ -414,8 +440,12 @@ def get_topic_postfix(generator): return "new" raise Exception("Unexpected generator") + # Tests -@pytest.mark.parametrize('create_query_generator, do_direct_read', [(generate_old_create_table_query, True), (generate_new_create_table_query, False)]) +@pytest.mark.parametrize( + "create_query_generator, do_direct_read", + [(generate_old_create_table_query, True), (generate_new_create_table_query, False)], +) def test_kafka_column_types(kafka_cluster, create_query_generator, do_direct_read): def assert_returned_exception(e): assert e.value.returncode == 36 @@ -426,12 +456,12 @@ def test_kafka_column_types(kafka_cluster, create_query_generator, do_direct_rea # check column with DEFAULT expression with pytest.raises(QueryRuntimeException) as exception: - instance.query(create_query_generator('kafka', 'a Int, b Int DEFAULT 0')) + instance.query(create_query_generator("kafka", "a Int, b Int DEFAULT 0")) assert_returned_exception(exception) # check EPHEMERAL with pytest.raises(QueryRuntimeException) as exception: - instance.query(create_query_generator('kafka', 'a Int, b Int EPHEMERAL')) + instance.query(create_query_generator("kafka", "a Int, b Int EPHEMERAL")) assert_returned_exception(exception) # check MATERIALIZED @@ -446,7 +476,13 @@ def test_kafka_column_types(kafka_cluster, create_query_generator, do_direct_rea if do_direct_read: # check ALIAS - instance.query(create_query_generator("kafka", "a Int, b String Alias toString(a)", settings={"kafka_commit_on_select": True})) + instance.query( + create_query_generator( + "kafka", + "a Int, b String Alias toString(a)", + settings={"kafka_commit_on_select": True}, + ) + ) messages = [] for i in range(5): messages.append(json.dumps({"a": i})) @@ -634,9 +670,15 @@ def test_kafka_json_as_string(kafka_cluster): "Parsing of message (topic: kafka_json_as_string, partition: 0, offset: [0-9]*) return no rows" ) -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) + +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_kafka_formats(kafka_cluster, create_query_generator): - schema_registry_client = CachedSchemaRegistryClient({"url":f"http://localhost:{kafka_cluster.schema_registry_port}"}) + schema_registry_client = CachedSchemaRegistryClient( + {"url": f"http://localhost:{kafka_cluster.schema_registry_port}"} + ) # data was dumped from clickhouse itself in a following manner # clickhouse-client --format=Native --query='SELECT toInt64(number) as id, toUInt16( intDiv( id, 65536 ) ) as blockNo, reinterpretAsString(19777) as val1, toFloat32(0.5) as val2, toUInt8(1) as val3 from numbers(100) ORDER BY id' | xxd -ps | tr -d '\n' | sed 's/\(..\)/\\x\1/g' @@ -759,7 +801,7 @@ def test_kafka_formats(kafka_cluster, create_query_generator): '(id = 1, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 2, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 3, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 4, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 5, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 6, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 7, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 8, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 9, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 10, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 11, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 12, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 13, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 14, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 15, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)', '(id = 0, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)', ], - "extra_settings": {"format_template_row":"template_row.format"}, + "extra_settings": {"format_template_row": "template_row.format"}, }, "Regexp": { "data_sample": [ @@ -770,7 +812,10 @@ def test_kafka_formats(kafka_cluster, create_query_generator): # On empty message exception happens: Line "" doesn't match the regexp.: (at row 1) # /src/Processors/Formats/Impl/RegexpRowInputFormat.cpp:140: DB::RegexpRowInputFormat::readRow(std::__1::vector::mutable_ptr, std::__1::allocator::mutable_ptr > >&, DB::RowReadExtension&) @ 0x1df82fcb in /usr/bin/clickhouse ], - "extra_settings": {"format_regexp":r"\(id = (.+?), blockNo = (.+?), val1 = \"(.+?)\", val2 = (.+?), val3 = (.+?)\)", "format_regexp_escaping_rule": "Escaped"}, + "extra_settings": { + "format_regexp": r"\(id = (.+?), blockNo = (.+?), val1 = \"(.+?)\", val2 = (.+?), val3 = (.+?)\)", + "format_regexp_escaping_rule": "Escaped", + }, }, ## BINARY FORMATS # dumped with @@ -842,7 +887,7 @@ def test_kafka_formats(kafka_cluster, create_query_generator): # /src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp:25: DB::ProtobufRowInputFormat::readRow(std::__1::vector::mutable_ptr, std::__1::allocator::mutable_ptr > >&, DB::RowReadExtension&) @ 0x1df4cc71 in /usr/bin/clickhouse # /src/Processors/Formats/IRowInputFormat.cpp:64: DB::IRowInputFormat::generate() @ 0x1de727cf in /usr/bin/clickhouse ], - "extra_settings": {"kafka_schema":"test:TestMessage"}, + "extra_settings": {"kafka_schema": "test:TestMessage"}, }, "ORC": { "data_sample": [ @@ -866,7 +911,7 @@ def test_kafka_formats(kafka_cluster, create_query_generator): # /src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp:212: DB::CapnProtoRowInputFormat::readMessage() @ 0x1ded1cab in /usr/bin/clickhouse # /src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp:241: DB::CapnProtoRowInputFormat::readRow(std::__1::vector::mutable_ptr, std::__1::allocator::mutable_ptr > >&, DB::RowReadExtension&) @ 0x1ded205d in /usr/bin/clickhouse ], - "extra_settings": {"kafka_schema":"test:TestRecordStruct"}, + "extra_settings": {"kafka_schema": "test:TestRecordStruct"}, }, "Parquet": { "data_sample": [ @@ -901,9 +946,12 @@ def test_kafka_formats(kafka_cluster, create_query_generator): {"id": 0, "blockNo": 0, "val1": str("AM"), "val2": 0.5, "val3": 1}, ), ], - "extra_settings": {"format_avro_schema_registry_url":"http://{}:{}".format( - kafka_cluster.schema_registry_host, kafka_cluster.schema_registry_port - )}, + "extra_settings": { + "format_avro_schema_registry_url": "http://{}:{}".format( + kafka_cluster.schema_registry_host, + kafka_cluster.schema_registry_port, + ) + }, "supports_empty_value": True, }, "Avro": { @@ -980,7 +1028,8 @@ def test_kafka_formats(kafka_cluster, create_query_generator): topic_list=f"{topic_name}", consumer_group=f"{topic_name}_group", format=format_name, - settings=extra_settings), + settings=extra_settings, + ), ) ) raw_expected = """\ @@ -1021,7 +1070,10 @@ def test_kafka_formats(kafka_cluster, create_query_generator): [1, 2, 3] if format_opts.get("supports_empty_value", False) else [0, 1, 2] ) result = instance.query_with_retry( - "SELECT * FROM test.kafka_{format_name}_mv;".format(format_name=format_name), check_callback=lambda x: x.count('\n') == raw_expected.count('\n') + "SELECT * FROM test.kafka_{format_name}_mv;".format( + format_name=format_name + ), + check_callback=lambda x: x.count("\n") == raw_expected.count("\n"), ) expected = raw_expected.format( topic_name=topic_name, @@ -1693,9 +1745,13 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): """ assert TSV(result) == TSV(expected) -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) + +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_kafka_materialized_view(kafka_cluster, create_query_generator): - topic_name="mv" + topic_name = "mv" instance.query( f""" @@ -1718,7 +1774,9 @@ def test_kafka_materialized_view(kafka_cluster, create_query_generator): kafka_produce(kafka_cluster, topic_name, messages) with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): - result = instance.query_with_retry("SELECT * FROM test.view", check_callback=kafka_check_result) + result = instance.query_with_retry( + "SELECT * FROM test.view", check_callback=kafka_check_result + ) kafka_check_result(result, True) @@ -1731,10 +1789,19 @@ def test_kafka_materialized_view(kafka_cluster, create_query_generator): ) -@pytest.mark.parametrize('create_query_generator, log_line', [ - (generate_new_create_table_query, r"kafka.*Saved offset [0-9]+ for topic-partition \[recreate_kafka_table:[0-9]+"), - (generate_old_create_table_query, "kafka.*Committed offset [0-9]+.*recreate_kafka_table"), -]) +@pytest.mark.parametrize( + "create_query_generator, log_line", + [ + ( + generate_new_create_table_query, + r"kafka.*Saved offset [0-9]+ for topic-partition \[recreate_kafka_table:[0-9]+", + ), + ( + generate_old_create_table_query, + "kafka.*Committed offset [0-9]+.*recreate_kafka_table", + ), + ], +) def test_kafka_recreate_kafka_table(kafka_cluster, create_query_generator, log_line): """ Checks that materialized view work properly after dropping and recreating the Kafka table. @@ -1753,7 +1820,8 @@ def test_kafka_recreate_kafka_table(kafka_cluster, create_query_generator, log_l "kafka_flush_interval_ms": 1000, "kafka_skip_broken_messages": 1048577, "kafka_thread_per_consumer": thread_per_consumer, - }) + }, + ) instance.query( f""" @@ -1808,10 +1876,16 @@ def test_kafka_recreate_kafka_table(kafka_cluster, create_query_generator, log_l ) -@pytest.mark.parametrize('create_query_generator, log_line', [ - (generate_old_create_table_query, "Committed offset {offset}"), - (generate_new_create_table_query, r"kafka.*Saved offset [0-9]+ for topic-partition \[{topic}:[0-9]+\]") -]) +@pytest.mark.parametrize( + "create_query_generator, log_line", + [ + (generate_old_create_table_query, "Committed offset {offset}"), + ( + generate_new_create_table_query, + r"kafka.*Saved offset [0-9]+ for topic-partition \[{topic}:[0-9]+\]", + ), + ], +) def test_librdkafka_compression(kafka_cluster, create_query_generator, log_line): """ Regression for UB in snappy-c (that is used in librdkafka), @@ -1866,16 +1940,19 @@ def test_librdkafka_compression(kafka_cluster, create_query_generator, log_line) "kafka", "key UInt64, value String", topic_list=topic_name, - #brokers="kafka1:19092", - #consumer_group=f"{topic_name}_group", + # brokers="kafka1:19092", + # consumer_group=f"{topic_name}_group", format="JSONEachRow", - settings={"kafka_flush_interval_ms": 1000}), + settings={"kafka_flush_interval_ms": 1000}, + ), ) ) kafka_produce(kafka_cluster, topic_name, messages) - instance.wait_for_log_line(log_line.format(offset=number_of_messages, topic=topic_name)) + instance.wait_for_log_line( + log_line.format(offset=number_of_messages, topic=topic_name) + ) result = instance.query("SELECT * FROM test.consumer") assert TSV(result) == TSV(expected) @@ -1884,12 +1961,20 @@ def test_librdkafka_compression(kafka_cluster, create_query_generator, log_line) instance.query("DROP TABLE test.consumer SYNC") -@pytest.mark.parametrize('create_query_generator', [generate_new_create_table_query, generate_old_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_new_create_table_query, generate_old_create_table_query], +) def test_kafka_materialized_view_with_subquery(kafka_cluster, create_query_generator): topic_name = "mysq" logging.debug(f"Using topic {topic_name}") - create_query = create_query_generator("kafka", "key UInt64, value UInt64", topic_list=topic_name, consumer_group=topic_name) + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + ) instance.query( f""" DROP TABLE IF EXISTS test.kafka; @@ -1911,7 +1996,12 @@ def test_kafka_materialized_view_with_subquery(kafka_cluster, create_query_gener kafka_produce(kafka_cluster, topic_name, messages) with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): - result = instance.query_with_retry("SELECT * FROM test.view", check_callback=kafka_check_result, retry_count=40, sleep_time=0.75) + result = instance.query_with_retry( + "SELECT * FROM test.view", + check_callback=kafka_check_result, + retry_count=40, + sleep_time=0.75, + ) instance.query( """ @@ -1923,14 +2013,17 @@ def test_kafka_materialized_view_with_subquery(kafka_cluster, create_query_gener kafka_check_result(result, True) -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_kafka_many_materialized_views(kafka_cluster, create_query_generator): topic_name = f"mmv-{get_topic_postfix(create_query_generator)}" create_query = create_query_generator( "kafka", "key UInt64, value UInt64", topic_list=topic_name, - consumer_group=f"{topic_name}-group" + consumer_group=f"{topic_name}-group", ) instance.query( @@ -1959,8 +2052,12 @@ def test_kafka_many_materialized_views(kafka_cluster, create_query_generator): kafka_produce(kafka_cluster, topic_name, messages) with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): - result1 = instance.query_with_retry("SELECT * FROM test.view1", check_callback=kafka_check_result) - result2 = instance.query_with_retry("SELECT * FROM test.view2", check_callback=kafka_check_result) + result1 = instance.query_with_retry( + "SELECT * FROM test.view1", check_callback=kafka_check_result + ) + result2 = instance.query_with_retry( + "SELECT * FROM test.view2", check_callback=kafka_check_result + ) instance.query( """ @@ -1975,7 +2072,10 @@ def test_kafka_many_materialized_views(kafka_cluster, create_query_generator): kafka_check_result(result2, True) -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_kafka_flush_on_big_message(kafka_cluster, create_query_generator): # Create batches of messages of size ~100Kb kafka_messages = 1000 @@ -1995,7 +2095,7 @@ def test_kafka_flush_on_big_message(kafka_cluster, create_query_generator): "key UInt64, value String", topic_list=topic_name, consumer_group=topic_name, - settings={"kafka_max_block_size": 10} + settings={"kafka_max_block_size": 10}, ) instance.query( @@ -2079,8 +2179,13 @@ def test_kafka_virtual_columns(kafka_cluster): kafka_check_result(result, True, "test_kafka_virtual1.reference") -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) -def test_kafka_virtual_columns_with_materialized_view(kafka_cluster, create_query_generator): +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_virtual_columns_with_materialized_view( + kafka_cluster, create_query_generator +): topic_config = { # default retention, since predefined timestamp_ms is used. "retention.ms": "-1", @@ -2092,10 +2197,9 @@ def test_kafka_virtual_columns_with_materialized_view(kafka_cluster, create_quer "kafka", "key UInt64, value UInt64", topic_list=topic_name, - consumer_group=f"{topic_name}-group" + consumer_group=f"{topic_name}-group", ) with kafka_topic(get_admin_client(kafka_cluster), topic_name, config=topic_config): - instance.query( f""" DROP TABLE IF EXISTS test.view; @@ -2116,9 +2220,11 @@ def test_kafka_virtual_columns_with_materialized_view(kafka_cluster, create_quer def check_callback(result): return kafka_check_result(result, False, "test_kafka_virtual2.reference") + result = instance.query_with_retry( "SELECT kafka_key, key, topic, value, offset, partition, timestamp FROM test.view ORDER BY kafka_key, key", - check_callback=check_callback) + check_callback=check_callback, + ) kafka_check_result(result, True, "test_kafka_virtual2.reference") @@ -2129,6 +2235,7 @@ def test_kafka_virtual_columns_with_materialized_view(kafka_cluster, create_quer """ ) + def insert_with_retry(instance, values, table_name="kafka", max_try_couunt=5): try_count = 0 while True: @@ -2144,7 +2251,10 @@ def insert_with_retry(instance, values, table_name="kafka", max_try_couunt=5): raise -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_kafka_insert(kafka_cluster, create_query_generator): topic_name = "insert1" + get_topic_postfix(create_query_generator) @@ -2154,8 +2264,8 @@ def test_kafka_insert(kafka_cluster, create_query_generator): "key UInt64, value UInt64", topic_list=topic_name, consumer_group=topic_name, - format="TSV" - ) + format="TSV", + ) ) message_count = 50 @@ -2171,7 +2281,10 @@ def test_kafka_insert(kafka_cluster, create_query_generator): kafka_check_result(result, True) -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_kafka_produce_consume(kafka_cluster, create_query_generator): topic_name = "insert2" + get_topic_postfix(create_query_generator) @@ -2180,7 +2293,7 @@ def test_kafka_produce_consume(kafka_cluster, create_query_generator): "key UInt64, value UInt64", topic_list=topic_name, consumer_group=topic_name, - format="TSV" + format="TSV", ) instance.query( f""" @@ -2219,7 +2332,8 @@ def test_kafka_produce_consume(kafka_cluster, create_query_generator): "SELECT count() FROM test.view", sleep_time=1, retry_count=20, - check_callback=lambda result: int(result) == expected_row_count) + check_callback=lambda result: int(result) == expected_row_count, + ) instance.query( """ @@ -2236,9 +2350,12 @@ def test_kafka_produce_consume(kafka_cluster, create_query_generator): ), "ClickHouse lost some messages: {}".format(result) -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_kafka_commit_on_block_write(kafka_cluster, create_query_generator): - topic_name="block" + get_topic_postfix(create_query_generator) + topic_name = "block" + get_topic_postfix(create_query_generator) create_query = create_query_generator( "kafka", "key UInt64, value UInt64", @@ -2279,7 +2396,8 @@ def test_kafka_commit_on_block_write(kafka_cluster, create_query_generator): instance.query_with_retry( "SELECT count() FROM test.view", sleep_time=1, - check_callback=lambda res: int(res) >= 100) + check_callback=lambda res: int(res) >= 100, + ) cancel.set() @@ -2291,7 +2409,8 @@ def test_kafka_commit_on_block_write(kafka_cluster, create_query_generator): instance.query_with_retry( "SELECT uniqExact(key) FROM test.view", sleep_time=1, - check_callback=lambda res: int(res) >= i[0]) + check_callback=lambda res: int(res) >= i[0], + ) result = int(instance.query("SELECT count() == uniqExact(key) FROM test.view")) @@ -2306,10 +2425,17 @@ def test_kafka_commit_on_block_write(kafka_cluster, create_query_generator): assert result == 1, "Messages from kafka get duplicated!" -@pytest.mark.parametrize('create_query_generator, log_line', [ - (generate_old_create_table_query, "kafka.*Committed offset 2.*virt2_[01]"), - (generate_new_create_table_query, r"kafka.*Saved offset 2[0-9]* for topic-partition \[virt2_[01]:[0-9]+"), -]) + +@pytest.mark.parametrize( + "create_query_generator, log_line", + [ + (generate_old_create_table_query, "kafka.*Committed offset 2.*virt2_[01]"), + ( + generate_new_create_table_query, + r"kafka.*Saved offset 2[0-9]* for topic-partition \[virt2_[01]:[0-9]+", + ), + ], +) def test_kafka_virtual_columns2(kafka_cluster, create_query_generator, log_line): admin_client = get_admin_client(kafka_cluster) @@ -2320,9 +2446,11 @@ def test_kafka_virtual_columns2(kafka_cluster, create_query_generator, log_line) thread_per_consumer = must_use_thread_per_consumer(create_query_generator) topic_name_0 = "virt2_0" topic_name_1 = "virt2_1" - consumer_group = "virt2"+get_topic_postfix(create_query_generator) + consumer_group = "virt2" + get_topic_postfix(create_query_generator) with kafka_topic(admin_client, topic_name_0, num_partitions=2, config=topic_config): - with kafka_topic(admin_client, topic_name_1, num_partitions=2, config=topic_config): + with kafka_topic( + admin_client, topic_name_1, num_partitions=2, config=topic_config + ): create_query = create_query_generator( "kafka", "value UInt64", @@ -2331,7 +2459,7 @@ def test_kafka_virtual_columns2(kafka_cluster, create_query_generator, log_line) settings={ "kafka_num_consumers": 2, "kafka_thread_per_consumer": thread_per_consumer, - } + }, ) instance.query( @@ -2430,7 +2558,9 @@ def test_kafka_virtual_columns2(kafka_cluster, create_query_generator, log_line) # members[0]['client_id'] = 'ClickHouse-instance-test-kafka-0' # members[1]['client_id'] = 'ClickHouse-instance-test-kafka-1' - result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) + result = instance.query( + "SELECT * FROM test.view ORDER BY value", ignore_error=True + ) expected = f"""\ 1 k1 {topic_name_0} 0 0 1577836801 1577836801001 ['content-encoding'] ['base64'] @@ -2453,6 +2583,7 @@ def test_kafka_virtual_columns2(kafka_cluster, create_query_generator, log_line) ) instance.rotate_logs() + # TODO(antaljanosbenjamin) def test_kafka_producer_consumer_separate_settings(kafka_cluster): instance.query( @@ -2531,10 +2662,13 @@ def test_kafka_producer_consumer_separate_settings(kafka_cluster): assert property_in_log in kafka_producer_applyed_properties -@pytest.mark.parametrize('create_query_generator, log_line', [ - (generate_new_create_table_query,"Saved offset 5"), - (generate_old_create_table_query, "Committed offset 5"), -]) +@pytest.mark.parametrize( + "create_query_generator, log_line", + [ + (generate_new_create_table_query, "Saved offset 5"), + (generate_old_create_table_query, "Committed offset 5"), + ], +) def test_kafka_produce_key_timestamp(kafka_cluster, create_query_generator, log_line): topic_name = "insert3" topic_config = { @@ -2548,13 +2682,15 @@ def test_kafka_produce_key_timestamp(kafka_cluster, create_query_generator, log_ "key UInt64, value UInt64, _key String, _timestamp DateTime('UTC')", topic_list=topic_name, consumer_group=topic_name, - format="TSV") + format="TSV", + ) reader_create_query = create_query_generator( "kafka", "key UInt64, value UInt64, inserted_key String, inserted_timestamp DateTime('UTC')", topic_list=topic_name, consumer_group=topic_name, - format="TSV") + format="TSV", + ) instance.query( f""" @@ -2603,12 +2739,16 @@ def test_kafka_produce_key_timestamp(kafka_cluster, create_query_generator, log_ ignore_error=True, retry_count=5, sleep_time=1, - check_callback=lambda res: TSV(res) == TSV(expected)) + check_callback=lambda res: TSV(res) == TSV(expected), + ) assert TSV(result) == TSV(expected) -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_kafka_insert_avro(kafka_cluster, create_query_generator): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) @@ -2617,7 +2757,7 @@ def test_kafka_insert_avro(kafka_cluster, create_query_generator): # default retention, since predefined timestamp_ms is used. "retention.ms": "-1", } - topic_name="avro1" + get_topic_postfix(create_query_generator) + topic_name = "avro1" + get_topic_postfix(create_query_generator) with kafka_topic(admin_client, topic_name, config=topic_config): create_query = create_query_generator( "kafka", @@ -2638,7 +2778,13 @@ def test_kafka_insert_avro(kafka_cluster, create_query_generator): ) message_count = 2 - messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count, need_decode=False, timestamp=1636505534) + messages = kafka_consume_with_retry( + kafka_cluster, + topic_name, + message_count, + need_decode=False, + timestamp=1636505534, + ) result = "" for a_message in messages: @@ -2654,11 +2800,13 @@ def test_kafka_insert_avro(kafka_cluster, create_query_generator): assert result == expected_result -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_kafka_produce_consume_avro(kafka_cluster, create_query_generator): topic_name = "insert_avro" + get_topic_postfix(create_query_generator) with kafka_topic(get_admin_client(kafka_cluster), topic_name): - num_rows = 75 writer_create_query = create_query_generator( @@ -2712,11 +2860,14 @@ def test_kafka_produce_consume_avro(kafka_cluster, create_query_generator): assert int(expected_max_key) == (num_rows - 1) * 10 -@pytest.mark.parametrize('create_query_generator', [ - generate_old_create_table_query, - # TODO(antaljanosbenjamin): Something is off with timing - # generate_new_create_table_query -]) +@pytest.mark.parametrize( + "create_query_generator", + [ + generate_old_create_table_query, + # TODO(antaljanosbenjamin): Something is off with timing + # generate_new_create_table_query + ], +) def test_kafka_flush_by_time(kafka_cluster, create_query_generator): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) @@ -2731,7 +2882,7 @@ def test_kafka_flush_by_time(kafka_cluster, create_query_generator): consumer_group=topic_name, settings={ "kafka_max_block_size": 100, - } + }, ) instance.query( f""" @@ -2782,11 +2933,15 @@ def test_kafka_flush_by_time(kafka_cluster, create_query_generator): assert TSV(result) == TSV("2 1") -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_kafka_flush_by_block_size(kafka_cluster, create_query_generator): topic_name = "flush_by_block_size" + get_topic_postfix(create_query_generator) cancel = threading.Event() + def produce(): while not cancel.is_set(): messages = [] @@ -2807,7 +2962,7 @@ def test_kafka_flush_by_block_size(kafka_cluster, create_query_generator): "kafka_max_block_size": 100, "kafka_poll_max_batch_size": 1, "kafka_flush_interval_ms": 120000, - } + }, ) instance.query( @@ -2855,13 +3010,20 @@ def test_kafka_flush_by_block_size(kafka_cluster, create_query_generator): ), "Messages from kafka should be flushed when block of size kafka_max_block_size is formed!" -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) -def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster, create_query_generator): +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_lot_of_partitions_partial_commit_of_bulk( + kafka_cluster, create_query_generator +): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) - topic_name = "topic_with_multiple_partitions2" + get_topic_postfix(create_query_generator) + topic_name = "topic_with_multiple_partitions2" + get_topic_postfix( + create_query_generator + ) with kafka_topic(admin_client, topic_name): create_query = create_query_generator( "kafka", @@ -2871,7 +3033,7 @@ def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster, create_qu settings={ "kafka_max_block_size": 211, "kafka_flush_interval_ms": 500, - } + }, ) instance.query( f""" @@ -2898,7 +3060,9 @@ def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster, create_qu instance.wait_for_log_line("kafka.*Stalled", repetitions=5) - result = instance.query("SELECT count(), uniqExact(key), max(key) FROM test.view") + result = instance.query( + "SELECT count(), uniqExact(key), max(key) FROM test.view" + ) logging.debug(result) assert TSV(result) == TSV("{0}\t{0}\t{0}".format(count)) @@ -2910,10 +3074,13 @@ def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster, create_qu ) -@pytest.mark.parametrize('create_query_generator, log_line', [ - (generate_old_create_table_query, "{}.*Polled offset [0-9]+"), - (generate_new_create_table_query, "{}.*Saved offset"), -]) +@pytest.mark.parametrize( + "create_query_generator, log_line", + [ + (generate_old_create_table_query, "{}.*Polled offset [0-9]+"), + (generate_new_create_table_query, "{}.*Saved offset"), + ], +) def test_kafka_rebalance(kafka_cluster, create_query_generator, log_line): NUMBER_OF_CONSURRENT_CONSUMERS = 11 @@ -2938,11 +3105,12 @@ def test_kafka_rebalance(kafka_cluster, create_query_generator, log_line): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) - topic_name = "topic_with_multiple_partitions" + get_topic_postfix(create_query_generator) + topic_name = "topic_with_multiple_partitions" + get_topic_postfix( + create_query_generator + ) table_name_prefix = "kafka_consumer" keeper_path = f"/clickhouse/{{database}}/{table_name_prefix}" with kafka_topic(admin_client, topic_name, num_partitions=11): - cancel = threading.Event() msg_index = [0] @@ -2974,7 +3142,7 @@ def test_kafka_rebalance(kafka_cluster, create_query_generator, log_line): settings={ "kafka_max_block_size": 33, "kafka_flush_interval_ms": 500, - } + }, ) instance.query( f""" @@ -3081,15 +3249,20 @@ def test_kafka_rebalance(kafka_cluster, create_query_generator, log_line): # TODO(antaljanosbenjamin): find another way to make insertion fail -@pytest.mark.parametrize('create_query_generator', [ - generate_old_create_table_query, - # generate_new_create_table_query, -]) +@pytest.mark.parametrize( + "create_query_generator", + [ + generate_old_create_table_query, + # generate_new_create_table_query, + ], +) def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster, create_query_generator): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) - topic_name = "no_holes_when_write_suffix_failed" + get_topic_postfix(create_query_generator) + topic_name = "no_holes_when_write_suffix_failed" + get_topic_postfix( + create_query_generator + ) with kafka_topic(admin_client, topic_name): messages = [json.dumps({"key": j + 1, "value": "x" * 300}) for j in range(22)] @@ -3103,7 +3276,7 @@ def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster, create_query_gen settings={ "kafka_max_block_size": 20, "kafka_flush_interval_ms": 2000, - } + }, ) instance.query( f""" @@ -3140,7 +3313,9 @@ def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster, create_query_gen pm.heal_all() instance.wait_for_log_line("Committed offset 22") - result = instance.query("SELECT count(), uniqExact(key), max(key) FROM test.view") + result = instance.query( + "SELECT count(), uniqExact(key), max(key) FROM test.view" + ) logging.debug(result) # kafka_cluster.open_bash_shell('instance') @@ -3198,9 +3373,14 @@ def test_exception_from_destructor(kafka_cluster): assert TSV(instance.query("SELECT 1")) == TSV("1") -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_commits_of_unprocessed_messages_on_drop(kafka_cluster, create_query_generator): - topic_name = "commits_of_unprocessed_messages_on_drop" + get_topic_postfix(create_query_generator) + topic_name = "commits_of_unprocessed_messages_on_drop" + get_topic_postfix( + create_query_generator + ) messages = [json.dumps({"key": j + 1, "value": j + 1}) for j in range(1)] kafka_produce(kafka_cluster, topic_name, messages) @@ -3213,7 +3393,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster, create_query_gen settings={ "kafka_max_block_size": 1000, "kafka_flush_interval_ms": 1000, - } + }, ) instance.query( f""" @@ -3259,9 +3439,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster, create_query_gen for _ in range(113): messages.append(json.dumps({"key": i[0], "value": i[0]})) i[0] += 1 - kafka_produce( - kafka_cluster, topic_name, messages - ) + kafka_produce(kafka_cluster, topic_name, messages) time.sleep(0.5) kafka_thread = threading.Thread(target=produce) @@ -3282,11 +3460,9 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster, create_query_gen settings={ "kafka_max_block_size": 10000, "kafka_flush_interval_ms": 1000, - } - ) - instance.query( - new_create_query + }, ) + instance.query(new_create_query) cancel.set() instance.wait_for_log_line("kafka.*Stalled", repetitions=5) @@ -3310,7 +3486,10 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster, create_query_gen assert TSV(result) == TSV("{0}\t{0}\t{0}".format(i[0] - 1)), "Missing data!" -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_bad_reschedule(kafka_cluster, create_query_generator): topic_name = "test_bad_reschedule" + get_topic_postfix(create_query_generator) @@ -3325,7 +3504,7 @@ def test_bad_reschedule(kafka_cluster, create_query_generator): settings={ "kafka_max_block_size": 1000, "kafka_flush_interval_ms": 1000, - } + }, ) instance.query( f""" @@ -3432,7 +3611,10 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): # But in cases of some peaky loads in kafka topic the current contract sounds more predictable and # easier to understand, so let's keep it as is for now. # also we can came to eof because we drained librdkafka internal queue too fast -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_premature_flush_on_eof(kafka_cluster, create_query_generator): topic_name = "premature_flush_on_eof" + get_topic_postfix(create_query_generator) create_query = create_query_generator( @@ -3510,14 +3692,16 @@ def test_premature_flush_on_eof(kafka_cluster, create_query_generator): ) -@pytest.mark.parametrize('create_query_generator, do_direct_read', [ - (generate_old_create_table_query, True), - (generate_new_create_table_query, False) -]) +@pytest.mark.parametrize( + "create_query_generator, do_direct_read", + [(generate_old_create_table_query, True), (generate_new_create_table_query, False)], +) def test_kafka_unavailable(kafka_cluster, create_query_generator, do_direct_read): - number_of_messages=20000 + number_of_messages = 20000 topic_name = "test_bad_reschedule" + get_topic_postfix(create_query_generator) - messages = [json.dumps({"key": j + 1, "value": j + 1}) for j in range(number_of_messages)] + messages = [ + json.dumps({"key": j + 1, "value": j + 1}) for j in range(number_of_messages) + ] kafka_produce(kafka_cluster, topic_name, messages) with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): @@ -3528,7 +3712,7 @@ def test_kafka_unavailable(kafka_cluster, create_query_generator, do_direct_read "key UInt64, value UInt64", topic_list=topic_name, consumer_group=topic_name, - settings={"kafka_max_block_size": 1000} + settings={"kafka_max_block_size": 1000}, ) instance.query( f""" @@ -3559,12 +3743,16 @@ def test_kafka_unavailable(kafka_cluster, create_query_generator, do_direct_read result = instance.query_with_retry( "SELECT count() FROM test.destination_unavailable", sleep_time=1, - check_callback=lambda res: int(res) == number_of_messages) + check_callback=lambda res: int(res) == number_of_messages, + ) assert int(result) == number_of_messages -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_kafka_issue14202(kafka_cluster, create_query_generator): """ INSERT INTO Kafka Engine from an empty SELECT sub query was leading to failure @@ -3638,9 +3826,14 @@ def random_string(size=8): return "".join(random.choices(string.ascii_uppercase + string.digits, k=size)) -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_kafka_engine_put_errors_to_stream(kafka_cluster, create_query_generator): - topic_name = "kafka_engine_put_errors_to_stream" + get_topic_postfix(create_query_generator) + topic_name = "kafka_engine_put_errors_to_stream" + get_topic_postfix( + create_query_generator + ) create_query = create_query_generator( "kafka", "i Int64, s String", @@ -3649,7 +3842,7 @@ def test_kafka_engine_put_errors_to_stream(kafka_cluster, create_query_generator settings={ "kafka_max_block_size": 128, "kafka_handle_error_mode": "stream", - } + }, ) instance.query( f""" @@ -3726,9 +3919,17 @@ def gen_message_with_jsons(jsons=10, malformed=0): return s.getvalue() -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) -def test_kafka_engine_put_errors_to_stream_with_random_malformed_json(kafka_cluster, create_query_generator): - topic_name = "kafka_engine_put_errors_to_stream_with_random_malformed_json" + get_topic_postfix(create_query_generator) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_engine_put_errors_to_stream_with_random_malformed_json( + kafka_cluster, create_query_generator +): + topic_name = ( + "kafka_engine_put_errors_to_stream_with_random_malformed_json" + + get_topic_postfix(create_query_generator) + ) create_query = create_query_generator( "kafka", "i Int64, s String", @@ -3738,7 +3939,7 @@ def test_kafka_engine_put_errors_to_stream_with_random_malformed_json(kafka_clus "kafka_max_block_size": 100, "kafka_poll_max_batch_size": 1, "kafka_handle_error_mode": "stream", - } + }, ) instance.query( @@ -3788,7 +3989,10 @@ def test_kafka_engine_put_errors_to_stream_with_random_malformed_json(kafka_clus ) -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator): # data was dumped from clickhouse itself in a following manner # clickhouse-client --format=Native --query='SELECT toInt64(number) as id, toUInt16( intDiv( id, 65536 ) ) as blockNo, reinterpretAsString(19777) as val1, toFloat32(0.5) as val2, toUInt8(1) as val3 from numbers(100) ORDER BY id' | xxd -ps | tr -d '\n' | sed 's/\(..\)/\\x\1/g' @@ -3808,8 +4012,8 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator '{"id":"0","blockNo":"BAD","val1":"AM","val2":0.5,"val3":1}', ], "expected": { - "raw_message":"{\"id\":\"0\",\"blockNo\":\"BAD\",\"val1\":\"AM\",\"val2\":0.5,\"val3\":1}", - "error":"Cannot parse input: expected '\"' before: 'BAD\",\"val1\":\"AM\",\"val2\":0.5,\"val3\":1}': (while reading the value of key blockNo)" + "raw_message": '{"id":"0","blockNo":"BAD","val1":"AM","val2":0.5,"val3":1}', + "error": 'Cannot parse input: expected \'"\' before: \'BAD","val1":"AM","val2":0.5,"val3":1}\': (while reading the value of key blockNo)', }, "supports_empty_value": True, "printable": True, @@ -3824,8 +4028,8 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator '["0", "BAD", "AM", 0.5, 1]', ], "expected": { - "raw_message":"[\"0\", \"BAD\", \"AM\", 0.5, 1]", - "error":"Cannot parse input: expected '\"' before: 'BAD\", \"AM\", 0.5, 1]': (while reading the value of key blockNo)" + "raw_message": '["0", "BAD", "AM", 0.5, 1]', + "error": "Cannot parse input: expected '\"' before: 'BAD\", \"AM\", 0.5, 1]': (while reading the value of key blockNo)", }, "supports_empty_value": True, "printable": True, @@ -3839,8 +4043,8 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator '["0", "BAD", "AM", 0.5, 1]', ], "expected": { - "raw_message":"[\"0\", \"BAD\", \"AM\", 0.5, 1]", - "error":"Cannot parse JSON string: expected opening quote" + "raw_message": '["0", "BAD", "AM", 0.5, 1]', + "error": "Cannot parse JSON string: expected opening quote", }, "printable": True, }, @@ -3853,8 +4057,8 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator "id=0\tblockNo=BAD\tval1=AM\tval2=0.5\tval3=1\n", ], "expected": { - "raw_message":"id=0\tblockNo=BAD\tval1=AM\tval2=0.5\tval3=1\n", - "error":"Found garbage after field in TSKV format: blockNo: (at row 1)\n" + "raw_message": "id=0\tblockNo=BAD\tval1=AM\tval2=0.5\tval3=1\n", + "error": "Found garbage after field in TSKV format: blockNo: (at row 1)\n", }, "printable": True, }, @@ -3867,8 +4071,8 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator '0,"BAD","AM",0.5,1\n', ], "expected": { - "raw_message":"0,\"BAD\",\"AM\",0.5,1\n", - "error":"Cannot parse input: expected '\"' before: 'BAD\",\"AM\",0.5,1\\n'" + "raw_message": '0,"BAD","AM",0.5,1\n', + "error": "Cannot parse input: expected '\"' before: 'BAD\",\"AM\",0.5,1\\n'", }, "printable": True, "supports_empty_value": True, @@ -3882,8 +4086,8 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator "0\tBAD\tAM\t0.5\t1\n", ], "expected": { - "raw_message":"0\tBAD\tAM\t0.5\t1\n", - "error":"Cannot parse input: expected '\\t' before: 'BAD\\tAM\\t0.5\\t1\\n'" + "raw_message": "0\tBAD\tAM\t0.5\t1\n", + "error": "Cannot parse input: expected '\\t' before: 'BAD\\tAM\\t0.5\\t1\\n'", }, "supports_empty_value": True, "printable": True, @@ -3897,8 +4101,8 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator '"id","blockNo","val1","val2","val3"\n0,"BAD","AM",0.5,1\n', ], "expected": { - "raw_message":"\"id\",\"blockNo\",\"val1\",\"val2\",\"val3\"\n0,\"BAD\",\"AM\",0.5,1\n", - "error":"Cannot parse input: expected '\"' before: 'BAD\",\"AM\",0.5,1\\n'" + "raw_message": '"id","blockNo","val1","val2","val3"\n0,"BAD","AM",0.5,1\n', + "error": "Cannot parse input: expected '\"' before: 'BAD\",\"AM\",0.5,1\\n'", }, "printable": True, }, @@ -3911,8 +4115,8 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator "(0,'BAD','AM',0.5,1)", ], "expected": { - "raw_message":"(0,'BAD','AM',0.5,1)", - "error":"Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero/NULL instead of throwing exception" + "raw_message": "(0,'BAD','AM',0.5,1)", + "error": "Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero/NULL instead of throwing exception", }, "supports_empty_value": True, "printable": True, @@ -3926,8 +4130,8 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator "id\tblockNo\tval1\tval2\tval3\n0\tBAD\tAM\t0.5\t1\n", ], "expected": { - "raw_message":"id\tblockNo\tval1\tval2\tval3\n0\tBAD\tAM\t0.5\t1\n", - "error":"Cannot parse input: expected '\\t' before: 'BAD\\tAM\\t0.5\\t1\\n" + "raw_message": "id\tblockNo\tval1\tval2\tval3\n0\tBAD\tAM\t0.5\t1\n", + "error": "Cannot parse input: expected '\\t' before: 'BAD\\tAM\\t0.5\\t1\\n", }, "supports_empty_value": True, "printable": True, @@ -3941,8 +4145,8 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator "id\tblockNo\tval1\tval2\tval3\nInt64\tUInt16\tString\tFloat32\tUInt8\n0\tBAD\tAM\t0.5\t1\n", ], "expected": { - "raw_message":"id\tblockNo\tval1\tval2\tval3\nInt64\tUInt16\tString\tFloat32\tUInt8\n0\tBAD\tAM\t0.5\t1\n", - "error":"Cannot parse input: expected '\\t' before: 'BAD\\tAM\\t0.5\\t1\\n'" + "raw_message": "id\tblockNo\tval1\tval2\tval3\nInt64\tUInt16\tString\tFloat32\tUInt8\n0\tBAD\tAM\t0.5\t1\n", + "error": "Cannot parse input: expected '\\t' before: 'BAD\\tAM\\t0.5\\t1\\n'", }, "printable": True, }, @@ -3955,8 +4159,8 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator b"\x05\x01\x02\x69\x64\x05\x49\x6e\x74\x36\x34\x00\x00\x00\x00\x00\x00\x00\x00\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x06\x53\x74\x72\x69\x6e\x67\x03\x42\x41\x44\x04\x76\x61\x6c\x31\x06\x53\x74\x72\x69\x6e\x67\x02\x41\x4d\x04\x76\x61\x6c\x32\x07\x46\x6c\x6f\x61\x74\x33\x32\x00\x00\x00\x3f\x04\x76\x61\x6c\x33\x05\x55\x49\x6e\x74\x38\x01", ], "expected": { - "raw_message":"050102696405496E743634000000000000000007626C6F636B4E6F06537472696E67034241440476616C3106537472696E6702414D0476616C3207466C6F617433320000003F0476616C330555496E743801", - "error":"Cannot convert: String to UInt16" + "raw_message": "050102696405496E743634000000000000000007626C6F636B4E6F06537472696E67034241440476616C3106537472696E6702414D0476616C3207466C6F617433320000003F0476616C330555496E743801", + "error": "Cannot convert: String to UInt16", }, "printable": False, }, @@ -3969,8 +4173,8 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator b"\x00\x00\x00\x00\x00\x00\x00\x00\x03\x42\x41\x44\x02\x41\x4d\x00\x00\x00\x3f\x01", ], "expected": { - "raw_message":"00000000000000000342414402414D0000003F01", - "error":"Cannot read all data. Bytes read: 9. Bytes expected: 65.: (at row 1)\n" + "raw_message": "00000000000000000342414402414D0000003F01", + "error": "Cannot read all data. Bytes read: 9. Bytes expected: 65.: (at row 1)\n", }, "printable": False, }, @@ -3983,8 +4187,8 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator b"\x05\x02\x69\x64\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x04\x76\x61\x6c\x31\x04\x76\x61\x6c\x32\x04\x76\x61\x6c\x33\x05\x49\x6e\x74\x36\x34\x06\x53\x74\x72\x69\x6e\x67\x06\x53\x74\x72\x69\x6e\x67\x07\x46\x6c\x6f\x61\x74\x33\x32\x05\x55\x49\x6e\x74\x38\x00\x00\x00\x00\x00\x00\x00\x00\x03\x42\x41\x44\x02\x41\x4d\x00\x00\x00\x3f\x01", ], "expected": { - "raw_message":"0502696407626C6F636B4E6F0476616C310476616C320476616C3305496E74363406537472696E6706537472696E6707466C6F617433320555496E743800000000000000000342414402414D0000003F01", - "error":"Type of 'blockNo' must be UInt16, not String" + "raw_message": "0502696407626C6F636B4E6F0476616C310476616C320476616C3305496E74363406537472696E6706537472696E6707466C6F617433320555496E743800000000000000000342414402414D0000003F01", + "error": "Type of 'blockNo' must be UInt16, not String", }, "printable": False, }, @@ -3997,8 +4201,8 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator b"\x4f\x52\x43\x0a\x0b\x0a\x03\x00\x00\x00\x12\x04\x08\x01\x50\x00\x0a\x15\x0a\x05\x00\x00\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x0a\x12\x0a\x06\x00\x00\x00\x00\x00\x00\x12\x08\x08\x01\x42\x02\x08\x06\x50\x00\x0a\x12\x0a\x06\x00\x00\x00\x00\x00\x00\x12\x08\x08\x01\x42\x02\x08\x04\x50\x00\x0a\x29\x0a\x04\x00\x00\x00\x00\x12\x21\x08\x01\x1a\x1b\x09\x00\x00\x00\x00\x00\x00\xe0\x3f\x11\x00\x00\x00\x00\x00\x00\xe0\x3f\x19\x00\x00\x00\x00\x00\x00\xe0\x3f\x50\x00\x0a\x15\x0a\x05\x00\x00\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\xff\x80\xff\x80\xff\x00\xff\x80\xff\x03\x42\x41\x44\xff\x80\xff\x02\x41\x4d\xff\x80\x00\x00\x00\x3f\xff\x80\xff\x01\x0a\x06\x08\x06\x10\x00\x18\x0d\x0a\x06\x08\x06\x10\x01\x18\x17\x0a\x06\x08\x06\x10\x02\x18\x14\x0a\x06\x08\x06\x10\x03\x18\x14\x0a\x06\x08\x06\x10\x04\x18\x2b\x0a\x06\x08\x06\x10\x05\x18\x17\x0a\x06\x08\x00\x10\x00\x18\x02\x0a\x06\x08\x00\x10\x01\x18\x02\x0a\x06\x08\x01\x10\x01\x18\x02\x0a\x06\x08\x00\x10\x02\x18\x02\x0a\x06\x08\x02\x10\x02\x18\x02\x0a\x06\x08\x01\x10\x02\x18\x03\x0a\x06\x08\x00\x10\x03\x18\x02\x0a\x06\x08\x02\x10\x03\x18\x02\x0a\x06\x08\x01\x10\x03\x18\x02\x0a\x06\x08\x00\x10\x04\x18\x02\x0a\x06\x08\x01\x10\x04\x18\x04\x0a\x06\x08\x00\x10\x05\x18\x02\x0a\x06\x08\x01\x10\x05\x18\x02\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x1a\x03\x47\x4d\x54\x0a\x59\x0a\x04\x08\x01\x50\x00\x0a\x0c\x08\x01\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x0a\x08\x08\x01\x42\x02\x08\x06\x50\x00\x0a\x08\x08\x01\x42\x02\x08\x04\x50\x00\x0a\x21\x08\x01\x1a\x1b\x09\x00\x00\x00\x00\x00\x00\xe0\x3f\x11\x00\x00\x00\x00\x00\x00\xe0\x3f\x19\x00\x00\x00\x00\x00\x00\xe0\x3f\x50\x00\x0a\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\x08\x03\x10\xec\x02\x1a\x0c\x08\x03\x10\x8e\x01\x18\x1d\x20\xc1\x01\x28\x01\x22\x2e\x08\x0c\x12\x05\x01\x02\x03\x04\x05\x1a\x02\x69\x64\x1a\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x1a\x04\x76\x61\x6c\x31\x1a\x04\x76\x61\x6c\x32\x1a\x04\x76\x61\x6c\x33\x20\x00\x28\x00\x30\x00\x22\x08\x08\x04\x20\x00\x28\x00\x30\x00\x22\x08\x08\x08\x20\x00\x28\x00\x30\x00\x22\x08\x08\x08\x20\x00\x28\x00\x30\x00\x22\x08\x08\x05\x20\x00\x28\x00\x30\x00\x22\x08\x08\x01\x20\x00\x28\x00\x30\x00\x30\x01\x3a\x04\x08\x01\x50\x00\x3a\x0c\x08\x01\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x3a\x08\x08\x01\x42\x02\x08\x06\x50\x00\x3a\x08\x08\x01\x42\x02\x08\x04\x50\x00\x3a\x21\x08\x01\x1a\x1b\x09\x00\x00\x00\x00\x00\x00\xe0\x3f\x11\x00\x00\x00\x00\x00\x00\xe0\x3f\x19\x00\x00\x00\x00\x00\x00\xe0\x3f\x50\x00\x3a\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\x40\x90\x4e\x48\x01\x08\xd5\x01\x10\x00\x18\x80\x80\x04\x22\x02\x00\x0b\x28\x5b\x30\x06\x82\xf4\x03\x03\x4f\x52\x43\x18", ], "expected": { - "raw_message":"4F52430A0B0A030000001204080150000A150A050000000000120C0801120608001000180050000A120A06000000000000120808014202080650000A120A06000000000000120808014202080450000A290A0400000000122108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50000A150A050000000000120C080112060802100218025000FF80FF80FF00FF80FF03424144FF80FF02414DFF800000003FFF80FF010A0608061000180D0A060806100118170A060806100218140A060806100318140A0608061004182B0A060806100518170A060800100018020A060800100118020A060801100118020A060800100218020A060802100218020A060801100218030A060800100318020A060802100318020A060801100318020A060800100418020A060801100418040A060800100518020A060801100518021204080010001204080010001204080010001204080010001204080010001204080010001A03474D540A590A04080150000A0C0801120608001000180050000A0808014202080650000A0808014202080450000A2108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50000A0C080112060802100218025000080310EC021A0C0803108E01181D20C1012801222E080C120501020304051A0269641A07626C6F636B4E6F1A0476616C311A0476616C321A0476616C33200028003000220808042000280030002208080820002800300022080808200028003000220808052000280030002208080120002800300030013A04080150003A0C0801120608001000180050003A0808014202080650003A0808014202080450003A2108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50003A0C08011206080210021802500040904E480108D5011000188080042202000B285B300682F403034F524318", - "error":"Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero/NULL instead of throwing exception." + "raw_message": "4F52430A0B0A030000001204080150000A150A050000000000120C0801120608001000180050000A120A06000000000000120808014202080650000A120A06000000000000120808014202080450000A290A0400000000122108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50000A150A050000000000120C080112060802100218025000FF80FF80FF00FF80FF03424144FF80FF02414DFF800000003FFF80FF010A0608061000180D0A060806100118170A060806100218140A060806100318140A0608061004182B0A060806100518170A060800100018020A060800100118020A060801100118020A060800100218020A060802100218020A060801100218030A060800100318020A060802100318020A060801100318020A060800100418020A060801100418040A060800100518020A060801100518021204080010001204080010001204080010001204080010001204080010001204080010001A03474D540A590A04080150000A0C0801120608001000180050000A0808014202080650000A0808014202080450000A2108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50000A0C080112060802100218025000080310EC021A0C0803108E01181D20C1012801222E080C120501020304051A0269641A07626C6F636B4E6F1A0476616C311A0476616C321A0476616C33200028003000220808042000280030002208080820002800300022080808200028003000220808052000280030002208080120002800300030013A04080150003A0C0801120608001000180050003A0808014202080650003A0808014202080450003A2108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50003A0C08011206080210021802500040904E480108D5011000188080042202000B285B300682F403034F524318", + "error": "Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero/NULL instead of throwing exception.", }, "printable": False, }, @@ -4027,7 +4231,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator settings={ "kafka_handle_error_mode": "stream", "kafka_flush_interval_ms": 1000, - } + }, ) instance.query( f""" @@ -4118,10 +4322,13 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator kafka_delete_topic(admin_client, topic_name) -@pytest.mark.parametrize('create_query_generator', [ - generate_old_create_table_query, - # generate_new_create_table_query TODO(antaljanosbenjamin): crashes CH -]) +@pytest.mark.parametrize( + "create_query_generator", + [ + generate_old_create_table_query, + # generate_new_create_table_query TODO(antaljanosbenjamin): crashes CH + ], +) def test_kafka_consumer_failover(kafka_cluster, create_query_generator): topic_name = "kafka_consumer_failover" + get_topic_postfix(create_query_generator) @@ -4129,16 +4336,18 @@ def test_kafka_consumer_failover(kafka_cluster, create_query_generator): consumer_group = f"{topic_name}_group" create_queries = [] for counter in range(3): - create_queries.append(create_query_generator( - f"kafka{counter+1}", - "key UInt64, value UInt64", - topic_list=topic_name, - consumer_group=consumer_group, - settings={ - "kafka_max_block_size": 1, - "kafka_poll_timeout_ms": 200, - } - )) + create_queries.append( + create_query_generator( + f"kafka{counter+1}", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=consumer_group, + settings={ + "kafka_max_block_size": 1, + "kafka_poll_timeout_ms": 200, + }, + ) + ) instance.query( f""" @@ -4188,7 +4397,9 @@ def test_kafka_consumer_failover(kafka_cluster, create_query_generator): producer.flush() count_query = "SELECT count() FROM test.destination" - prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > 0) + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > 0 + ) ## 2 attached, 2 working instance.query("DETACH TABLE test.kafka1") @@ -4203,7 +4414,9 @@ def test_kafka_consumer_failover(kafka_cluster, create_query_generator): partition=1, ) producer.flush() - prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > prev_count) + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > prev_count + ) ## 1 attached, 1 working instance.query("DETACH TABLE test.kafka2") @@ -4218,7 +4431,9 @@ def test_kafka_consumer_failover(kafka_cluster, create_query_generator): partition=1, ) producer.flush() - prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > prev_count) + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > prev_count + ) ## 2 attached, 2 working instance.query("ATTACH TABLE test.kafka1") @@ -4233,7 +4448,9 @@ def test_kafka_consumer_failover(kafka_cluster, create_query_generator): partition=1, ) producer.flush() - prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > prev_count) + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > prev_count + ) ## 1 attached, 1 working instance.query("DETACH TABLE test.kafka3") @@ -4248,7 +4465,9 @@ def test_kafka_consumer_failover(kafka_cluster, create_query_generator): partition=1, ) producer.flush() - prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > prev_count) + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > prev_count + ) ## 2 attached, 2 working instance.query("ATTACH TABLE test.kafka2") @@ -4263,7 +4482,9 @@ def test_kafka_consumer_failover(kafka_cluster, create_query_generator): partition=1, ) producer.flush() - prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > prev_count) + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > prev_count + ) ## 3 attached, 2 working instance.query("ATTACH TABLE test.kafka3") @@ -4278,7 +4499,9 @@ def test_kafka_consumer_failover(kafka_cluster, create_query_generator): partition=1, ) producer.flush() - prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > prev_count) + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > prev_count + ) ## 2 attached, same 2 working instance.query("DETACH TABLE test.kafka3") @@ -4293,7 +4516,9 @@ def test_kafka_consumer_failover(kafka_cluster, create_query_generator): partition=1, ) producer.flush() - prev_count = instance.query_with_retry(count_query, check_callback=lambda res: int(res) > prev_count) + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > prev_count + ) def test_kafka_predefined_configuration(kafka_cluster): @@ -4323,7 +4548,10 @@ def test_kafka_predefined_configuration(kafka_cluster): # https://github.com/ClickHouse/ClickHouse/issues/26643 -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_issue26643(kafka_cluster, create_query_generator): producer = KafkaProducer( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), @@ -4383,7 +4611,7 @@ def test_issue26643(kafka_cluster, create_query_generator): "kafka_schema": "message_with_repeated.proto:Message", "kafka_skip_broken_messages": 10000, "kafka_thread_per_consumer": thread_per_consumer, - } + }, ) instance.query( @@ -4440,8 +4668,10 @@ def test_issue26643(kafka_cluster, create_query_generator): assert TSV(result) == TSV(expected) - -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_num_consumers_limit(kafka_cluster, create_query_generator): instance.query("DROP TABLE IF EXISTS test.kafka") @@ -4452,12 +4682,15 @@ def test_num_consumers_limit(kafka_cluster, create_query_generator): "key UInt64, value UInt64", settings={ "kafka_num_consumers": 100, - "kafka_thread_per_consumer": thread_per_consumer - } + "kafka_thread_per_consumer": thread_per_consumer, + }, ) error = instance.query_and_get_error(create_query) - assert "BAD_ARGUMENTS" in error and "The number of consumers can not be bigger than" in error + assert ( + "BAD_ARGUMENTS" in error + and "The number of consumers can not be bigger than" in error + ) instance.query( f""" @@ -4469,7 +4702,10 @@ def test_num_consumers_limit(kafka_cluster, create_query_generator): instance.query("DROP TABLE test.kafka") -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_format_with_prefix_and_suffix(kafka_cluster, create_query_generator): topic_name = "custom" + get_topic_postfix(create_query_generator) @@ -4479,7 +4715,7 @@ def test_format_with_prefix_and_suffix(kafka_cluster, create_query_generator): "key UInt64, value UInt64", topic_list=topic_name, consumer_group=topic_name, - format="CustomSeparated" + format="CustomSeparated", ) instance.query( f""" @@ -4498,13 +4734,19 @@ def test_format_with_prefix_and_suffix(kafka_cluster, create_query_generator): assert len(messages) == 2 assert ( - "".join(messages) == "\n0\t0\n\n\n10\t100\n\n" + "".join(messages) + == "\n0\t0\n\n\n10\t100\n\n" ) -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_max_rows_per_message(kafka_cluster, create_query_generator): - topic_name = "custom_max_rows_per_message" + get_topic_postfix(create_query_generator) + topic_name = "custom_max_rows_per_message" + get_topic_postfix( + create_query_generator + ) with kafka_topic(get_admin_client(kafka_cluster), topic_name): num_rows = 5 @@ -4519,7 +4761,7 @@ def test_max_rows_per_message(kafka_cluster, create_query_generator): "format_custom_result_before_delimiter": "\n", "format_custom_result_after_delimiter": "\n", "kafka_max_rows_per_message": 3, - } + }, ) instance.query( f""" @@ -4546,13 +4788,19 @@ def test_max_rows_per_message(kafka_cluster, create_query_generator): == "\n0\t0\n10\t100\n20\t200\n\n\n30\t300\n40\t400\n\n" ) - instance.query_with_retry("SELECT count() FROM test.view", check_callback=lambda res: int(res) == num_rows) + instance.query_with_retry( + "SELECT count() FROM test.view", + check_callback=lambda res: int(res) == num_rows, + ) result = instance.query("SELECT * FROM test.view") assert result == "0\t0\n10\t100\n20\t200\n30\t300\n40\t400\n" -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_row_based_formats(kafka_cluster, create_query_generator): admin_client = get_admin_client(kafka_cluster) @@ -4589,7 +4837,7 @@ def test_row_based_formats(kafka_cluster, create_query_generator): topic_list=topic_name, consumer_group=topic_name, format=format_name, - settings={"kafka_max_rows_per_message": max_rows_per_message} + settings={"kafka_max_rows_per_message": max_rows_per_message}, ) instance.query( @@ -4606,11 +4854,16 @@ def test_row_based_formats(kafka_cluster, create_query_generator): """ ) - messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count, need_decode=False) + messages = kafka_consume_with_retry( + kafka_cluster, topic_name, message_count, need_decode=False + ) assert len(messages) == message_count - instance.query_with_retry("SELECT count() FROM test.view", check_callback=lambda res: int(res) == num_rows) + instance.query_with_retry( + "SELECT count() FROM test.view", + check_callback=lambda res: int(res) == num_rows, + ) result = instance.query("SELECT * FROM test.view") expected = "" @@ -4619,7 +4872,10 @@ def test_row_based_formats(kafka_cluster, create_query_generator): assert result == expected -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_block_based_formats_1(kafka_cluster, create_query_generator): topic_name = "pretty_space" + get_topic_postfix(create_query_generator) @@ -4629,7 +4885,7 @@ def test_block_based_formats_1(kafka_cluster, create_query_generator): "key UInt64, value UInt64", topic_list=topic_name, consumer_group=topic_name, - format="PrettySpace" + format="PrettySpace", ) instance.query( f""" @@ -4662,7 +4918,10 @@ def test_block_based_formats_1(kafka_cluster, create_query_generator): ] -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_block_based_formats_2(kafka_cluster, create_query_generator): admin_client = get_admin_client(kafka_cluster) num_rows = 100 @@ -4683,7 +4942,7 @@ def test_block_based_formats_2(kafka_cluster, create_query_generator): "key UInt64, value UInt64", topic_list=topic_name, consumer_group=topic_name, - format=format_name + format=format_name, ) instance.query( @@ -4699,10 +4958,17 @@ def test_block_based_formats_2(kafka_cluster, create_query_generator): INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; """ ) - messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count, need_decode=False) + messages = kafka_consume_with_retry( + kafka_cluster, topic_name, message_count, need_decode=False + ) assert len(messages) == message_count - rows = int(instance.query_with_retry("SELECT count() FROM test.view", check_callback=lambda res: int(res) == num_rows)) + rows = int( + instance.query_with_retry( + "SELECT count() FROM test.view", + check_callback=lambda res: int(res) == num_rows, + ) + ) assert rows == num_rows @@ -5049,7 +5315,6 @@ def test_formats_errors(kafka_cluster): "MySQLDump", ]: with kafka_topic(admin_client, format_name): - table_name = f"kafka_{format_name}" instance.query( @@ -5081,9 +5346,9 @@ def test_formats_errors(kafka_cluster): ) num_errors = int( - instance.query_with_retry( + instance.query_with_retry( f"SELECT length(exceptions.text) from system.kafka_consumers where database = 'test' and table = '{table_name}'", - check_callback=lambda res: int(res) > 0 + check_callback=lambda res: int(res) > 0, ) ) @@ -5093,7 +5358,10 @@ def test_formats_errors(kafka_cluster): instance.query("DROP TABLE test.view") -@pytest.mark.parametrize('create_query_generator', [generate_old_create_table_query, generate_new_create_table_query]) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) def test_multiple_read_in_materialized_views(kafka_cluster, create_query_generator): topic_name = "multiple_read_from_mv" + get_topic_postfix(create_query_generator) From 419660d1b08741c563253927b55e2d7666e737ed Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Sat, 8 Jun 2024 21:52:28 +0000 Subject: [PATCH 084/644] Fix one more test --- src/Storages/Kafka/KafkaConsumer2.cpp | 36 +----------------- src/Storages/Kafka/KafkaConsumer2.h | 2 - src/Storages/Kafka/StorageKafka2.cpp | 40 +++++++++----------- src/Storages/Kafka/StorageKafka2.h | 2 + tests/integration/test_storage_kafka/test.py | 7 +--- 5 files changed, 23 insertions(+), 64 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 1320b939612..8a6c7e31910 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -17,7 +17,6 @@ #include #include -#include "base/scope_guard.h" namespace CurrentMetrics { @@ -45,7 +44,6 @@ namespace ErrorCodes } using namespace std::chrono_literals; -static constexpr auto MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS = 15000; static constexpr auto EVENT_POLL_TIMEOUT = 50ms; static constexpr auto DRAIN_TIMEOUT_MS = 5000ms; @@ -122,7 +120,6 @@ KafkaConsumer2::KafkaConsumer2( assignment.reset(); queues.clear(); needs_offset_update = true; - waited_for_assignment = 0; }); consumer->set_rebalance_error_callback( @@ -210,13 +207,8 @@ void KafkaConsumer2::pollEvents() consumer_has_subscription = !consumer->get_subscription().empty(); } auto msg = consumer->poll(EVENT_POLL_TIMEOUT); - LOG_TRACE(log, "Consumer has subscription: {}", consumer_has_subscription); // All the partition queues are detached, so the consumer shouldn't be able to poll any messages chassert(!msg && "Consumer returned a message when it was not expected"); - - auto consumer_queue = consumer->get_consumer_queue(); - for(auto i = 0; i < max_tries && consumer_queue.get_length() > 0; ++i) - consumer->poll(EVENT_POLL_TIMEOUT); }; KafkaConsumer2::TopicPartitionCounts KafkaConsumer2::getPartitionCounts() const @@ -322,32 +314,8 @@ ReadBufferPtr KafkaConsumer2::consume(const TopicPartition & topic_partition, co if (new_messages.empty()) { - // While we wait for an assignment after subscription, we'll poll zero messages anyway. - // If we're doing a manual select then it's better to get something after a wait, then immediate nothing. - if (!assignment.has_value()) - { - waited_for_assignment += poll_timeout; // slightly innaccurate, but rough calculation is ok. - if (waited_for_assignment < MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS) - { - continue; - } - else - { - LOG_WARNING(log, "Can't get assignment. Will keep trying."); - stalled_status = StalledStatus::NO_ASSIGNMENT; - return nullptr; - } - } - else if (assignment->empty()) - { - LOG_TRACE(log, "Empty assignment."); - return nullptr; - } - else - { - LOG_TRACE(log, "Stalled"); - return nullptr; - } + LOG_TRACE(log, "Stalled"); + return nullptr; } else { diff --git a/src/Storages/Kafka/KafkaConsumer2.h b/src/Storages/Kafka/KafkaConsumer2.h index 16d12c8723d..ff0fae35b67 100644 --- a/src/Storages/Kafka/KafkaConsumer2.h +++ b/src/Storages/Kafka/KafkaConsumer2.h @@ -141,8 +141,6 @@ private: StalledStatus stalled_status = StalledStatus::NO_MESSAGES_RETURNED; - size_t waited_for_assignment = 0; - const std::atomic & stopped; // order is important, need to be destructed before consumer diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 911de671fb7..bf2df473793 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -100,6 +100,7 @@ extern const int TABLE_WAS_NOT_DROPPED; namespace { constexpr auto MAX_FAILED_POLL_ATTEMPTS = 10; +constexpr auto MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS = 15000; } StorageKafka2::StorageKafka2( @@ -163,21 +164,6 @@ StorageKafka2::StorageKafka2( tryLogCurrentException(log); } } - // for (auto try_count = 0; try_count < 5; ++try_count) - // { - // bool all_had_assignment = true; - // for (auto & consumer_info : consumers) - // { - // if (nullptr == consumer_info.consumer->getKafkaAssignment()) - // { - // all_had_assignment = false; - // consumer_info.consumer->pollEvents(); - // } - // } - - // if (all_had_assignment) - // break; - // } const auto first_replica = createTableIfNotExists(); @@ -875,8 +861,10 @@ StorageKafka2::PolledBatchInfo StorageKafka2::pollConsumer( KafkaConsumer2 & consumer, const TopicPartition & topic_partition, std::optional message_count, + Stopwatch & total_stopwatch, const ContextPtr & modified_context) { + LOG_TEST(log, "Polling consumer"); PolledBatchInfo batch_info; auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); Block non_virtual_header(storage_snapshot->metadata->getSampleBlockNonMaterialized()); @@ -936,9 +924,7 @@ StorageKafka2::PolledBatchInfo StorageKafka2::pollConsumer( ? kafka_settings->kafka_flush_interval_ms : getContext()->getSettingsRef().stream_flush_interval_ms; - Stopwatch total_stopwatch{CLOCK_MONOTONIC_COARSE}; - - const auto check_time_limit = [&max_execution_time, &total_stopwatch]() + const auto check_time_limit = [&max_execution_time, &total_stopwatch, this]() { if (max_execution_time != 0) { @@ -1139,8 +1125,6 @@ bool StorageKafka2::streamToViews(size_t idx) // 7. Execute the pipeline // 8. Write the offset to Keeper - Stopwatch watch; - auto table_id = getStorageID(); auto table = DatabaseCatalog::instance().getTable(table_id, getContext()); if (!table) @@ -1150,12 +1134,20 @@ bool StorageKafka2::streamToViews(size_t idx) ProfileEvents::increment(ProfileEvents::KafkaBackgroundReads); auto & consumer_info = consumers[idx]; + consumer_info.watch.restart(); auto & consumer = consumer_info.consumer; // To keep the consumer alive + const auto wait_for_assignment = consumer_info.locks.empty(); LOG_TRACE(log, "Polling consumer for events"); consumer->pollEvents(); + if (wait_for_assignment) + { + while (nullptr == consumer->getKafkaAssignment() && consumer_info.watch.elapsedMilliseconds() < MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS) + consumer->pollEvents(); + } + try { if (consumer->needsOffsetUpdate() || consumer_info.locks.empty()) @@ -1168,6 +1160,7 @@ bool StorageKafka2::streamToViews(size_t idx) { // The consumer lost its assignment and haven't received a new one. // By returning true this function reports the current consumer as a "stalled" stream, which + LOG_TRACE(log, "No assignment"); return true; } LOG_TRACE(log, "Consumer needs update offset"); @@ -1181,6 +1174,7 @@ bool StorageKafka2::streamToViews(size_t idx) if (!maybe_locks.has_value()) { // We couldn't acquire locks, probably some other consumers are still holding them. + LOG_TRACE(log, "Couldn't acquire locks"); return true; } @@ -1206,7 +1200,7 @@ bool StorageKafka2::streamToViews(size_t idx) const auto maybe_rows = streamFromConsumer(consumer_info); if (maybe_rows.has_value()) { - const auto milliseconds = watch.elapsedMilliseconds(); + const auto milliseconds = consumer_info.watch.elapsedMilliseconds(); LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(*maybe_rows), table_id.getNameForLogs(), milliseconds); } else @@ -1262,8 +1256,8 @@ std::optional StorageKafka2::streamFromConsumer(ConsumerAndAssignmentInf return; consumer_info.consumer->updateOffsets(consumer_info.topic_partitions); }); - auto [blocks, last_read_offset] - = pollConsumer(*consumer_info.consumer, topic_partition, consumer_info.locks[topic_partition].intent_size, kafka_context); + auto [blocks, last_read_offset] = pollConsumer( + *consumer_info.consumer, topic_partition, consumer_info.locks[topic_partition].intent_size, consumer_info.watch, kafka_context); if (blocks.empty()) { diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index 86d09c584f2..0fd19f66f52 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -102,6 +102,7 @@ private: TopicPartitions topic_partitions; zkutil::ZooKeeperPtr keeper; TopicPartitionLocks locks; + Stopwatch watch{CLOCK_MONOTONIC_COARSE}; }; struct PolledBatchInfo @@ -208,6 +209,7 @@ private: KafkaConsumer2 & consumer, const TopicPartition & topic_partition, std::optional message_count, + Stopwatch & watch, const ContextPtr & context); zkutil::ZooKeeperPtr getZooKeeper(); diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index ad4e0a0877f..19374ef2c96 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2860,13 +2860,10 @@ def test_kafka_produce_consume_avro(kafka_cluster, create_query_generator): assert int(expected_max_key) == (num_rows - 1) * 10 + @pytest.mark.parametrize( "create_query_generator", - [ - generate_old_create_table_query, - # TODO(antaljanosbenjamin): Something is off with timing - # generate_new_create_table_query - ], + [generate_old_create_table_query, generate_new_create_table_query], ) def test_kafka_flush_by_time(kafka_cluster, create_query_generator): admin_client = KafkaAdminClient( From f9f43283808b5551b5f07b716d9c4c20e970af46 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Sat, 8 Jun 2024 21:57:19 +0000 Subject: [PATCH 085/644] Style fixes --- src/Storages/Kafka/KafkaConsumer2.cpp | 11 ++-------- src/Storages/Kafka/KafkaConsumer2.h | 2 +- src/Storages/Kafka/StorageKafka2.cpp | 30 +++++++++++---------------- src/Storages/Kafka/StorageKafka2.h | 12 +++++------ 4 files changed, 21 insertions(+), 34 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 8a6c7e31910..1e2ea3fd43a 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -38,11 +38,6 @@ extern const Event KafkaConsumerErrors; namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_COMMIT_OFFSET; -} - using namespace std::chrono_literals; static constexpr auto EVENT_POLL_TIMEOUT = 50ms; static constexpr auto DRAIN_TIMEOUT_MS = 5000ms; @@ -201,7 +196,7 @@ void KafkaConsumer2::pollEvents() { static constexpr int64_t max_tries = 5; auto consumer_has_subscription = !consumer->get_subscription().empty(); - for(auto i = 0; i < max_tries && !consumer_has_subscription; ++i) + for (auto i = 0; i < max_tries && !consumer_has_subscription; ++i) { consumer->subscribe(topics); consumer_has_subscription = !consumer->get_subscription().empty(); @@ -394,9 +389,7 @@ void KafkaConsumer2::commit(const TopicPartition & topic_partition) { // The failure is not the biggest issue, it only counts when a table is dropped and recreated, otherwise the offsets are taken from keeper. ProfileEvents::increment(ProfileEvents::KafkaCommitFailures); - LOG_INFO( - log, - "All commit attempts failed"); + LOG_INFO(log, "All commit attempts failed"); } else { diff --git a/src/Storages/Kafka/KafkaConsumer2.h b/src/Storages/Kafka/KafkaConsumer2.h index ff0fae35b67..d7ec227d0bd 100644 --- a/src/Storages/Kafka/KafkaConsumer2.h +++ b/src/Storages/Kafka/KafkaConsumer2.h @@ -110,7 +110,7 @@ public: /// nullptr when there are no messages to process. ReadBufferPtr consume(const TopicPartition & topic_partition, const std::optional & message_count); - void commit(const TopicPartition& topic_partition); + void commit(const TopicPartition & topic_partition); // Return values for the message that's being read. String currentTopic() const { return current[-1].get_topic(); } diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index bf2df473793..c28a45d1f32 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -29,18 +30,18 @@ #include #include #include -#include #include #include #include #include #include -#include +#include "Common/config_version.h" #include #include #include #include #include +#include #include #include #include @@ -49,7 +50,6 @@ #include #include #include "Storages/Kafka/KafkaConsumer2.h" -#include "Common/config_version.h" #if USE_KRB5 # include @@ -65,8 +65,8 @@ namespace CurrentMetrics { +// TODO: Add proper metrics, similar to old StorageKafka extern const Metric KafkaBackgroundReads; -extern const Metric KafkaConsumersInUse; extern const Metric KafkaWrites; } @@ -77,7 +77,6 @@ extern const Event KafkaBackgroundReads; extern const Event KafkaMessagesRead; extern const Event KafkaMessagesFailed; extern const Event KafkaRowsRead; -extern const Event KafkaRowsRejected; extern const Event KafkaWrites; } @@ -91,7 +90,6 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; -extern const int QUERY_NOT_ALLOWED; extern const int REPLICA_ALREADY_EXISTS; extern const int TABLE_IS_DROPPED; extern const int TABLE_WAS_NOT_DROPPED; @@ -324,12 +322,7 @@ KafkaConsumer2Ptr StorageKafka2::createConsumer(size_t consumer_number) } return std::make_shared( - consumer_impl, - log, - getPollMaxBatchSize(), - getPollTimeoutMillisecond(), - tasks.back()->stream_cancelled, - topics); + consumer_impl, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), tasks.back()->stream_cancelled, topics); } @@ -833,14 +826,14 @@ StorageKafka2::lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const Topi } -void StorageKafka2::saveCommittedOffset( - zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition) +void StorageKafka2::saveCommittedOffset(zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition) { const auto partition_prefix = getTopicPartitionPath(topic_partition); keeper_to_use.createOrUpdate(partition_prefix / commit_file_name, toString(topic_partition.offset), zkutil::CreateMode::Persistent); // This is best effort, if it fails we will try to remove in the next round keeper_to_use.tryRemove(partition_prefix / intent_file_name, -1); - LOG_TEST(log, "Saved offset {} for topic-partition [{}:{}]", topic_partition.offset, topic_partition.topic, topic_partition.partition_id); + LOG_TEST( + log, "Saved offset {} for topic-partition [{}:{}]", topic_partition.offset, topic_partition.topic, topic_partition.partition_id); } void StorageKafka2::saveIntent(zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition, int64_t intent) @@ -1026,8 +1019,8 @@ StorageKafka2::PolledBatchInfo StorageKafka2::pollConsumer( } if (!consumer.hasMorePolledMessages() - && (total_rows >= getMaxBlockSize() || !check_time_limit() - || failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS || consumer.needsOffsetUpdate())) + && (total_rows >= getMaxBlockSize() || !check_time_limit() || failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS + || consumer.needsOffsetUpdate())) { LOG_TRACE( log, @@ -1201,7 +1194,8 @@ bool StorageKafka2::streamToViews(size_t idx) if (maybe_rows.has_value()) { const auto milliseconds = consumer_info.watch.elapsedMilliseconds(); - LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(*maybe_rows), table_id.getNameForLogs(), milliseconds); + LOG_DEBUG( + log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(*maybe_rows), table_id.getNameForLogs(), milliseconds); } else { diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index 0fd19f66f52..a2cbdce51a0 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -1,21 +1,21 @@ #pragma once #include +#include #include #include #include #include -#include #include #include +#include #include -#include #include #include -#include #include +#include #include #include @@ -201,9 +201,9 @@ private: void dropReplica(); // Takes lock over topic partitions and set's the committed offset in topic_partitions - std::optional lockTopicPartitions(zkutil::ZooKeeper& keeper_to_use, const TopicPartitions & topic_partitions); - void saveCommittedOffset(zkutil::ZooKeeper& keeper_to_use,const TopicPartition & topic_partition); - void saveIntent(zkutil::ZooKeeper& keeper_to_use,const TopicPartition & topic_partition, int64_t intent); + std::optional lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const TopicPartitions & topic_partitions); + void saveCommittedOffset(zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition); + void saveIntent(zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition, int64_t intent); PolledBatchInfo pollConsumer( KafkaConsumer2 & consumer, From 6df87b1cbfdcee0c70a87be66cb2b00d21ef2e84 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Sat, 8 Jun 2024 22:02:21 +0000 Subject: [PATCH 086/644] Fix build --- src/Storages/Kafka/StorageKafka2.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index c28a45d1f32..0bc484b4d45 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -917,7 +917,7 @@ StorageKafka2::PolledBatchInfo StorageKafka2::pollConsumer( ? kafka_settings->kafka_flush_interval_ms : getContext()->getSettingsRef().stream_flush_interval_ms; - const auto check_time_limit = [&max_execution_time, &total_stopwatch, this]() + const auto check_time_limit = [&max_execution_time, &total_stopwatch]() { if (max_execution_time != 0) { From be64d1bd32526b0f9b60d67f6ad31ea806c55797 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Sat, 8 Jun 2024 22:16:52 +0000 Subject: [PATCH 087/644] Fix tests --- tests/integration/test_storage_kafka/test.py | 22 +++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 19374ef2c96..d4d47311483 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -48,6 +48,8 @@ if is_arm(): # TODO: add test for SELECT LIMIT is working. +KAFKA_TOPIC_OLD = "old_t" +KAFKA_CONSUMER_GROUP_OLD = "old_cg" KAFKA_TOPIC_NEW = "new_t" KAFKA_CONSUMER_GROUP_NEW = "new_cg" @@ -61,8 +63,8 @@ instance = cluster.add_instance( with_zookeeper=True, # For Replicated Table macros={ "kafka_broker": "kafka1", - "kafka_topic_old": "old", - "kafka_group_name_old": "old", + "kafka_topic_old": KAFKA_TOPIC_OLD, + "kafka_group_name_old": KAFKA_CONSUMER_GROUP_OLD, "kafka_topic_new": KAFKA_TOPIC_NEW, "kafka_group_name_new": KAFKA_CONSUMER_GROUP_NEW, "kafka_client_id": "instance", @@ -517,13 +519,13 @@ def test_kafka_settings_old_syntax(kafka_cluster): ignore_error=True, ) ) == TSV( - """kafka_broker kafka1 + f"""kafka_broker kafka1 kafka_client_id instance kafka_format_json_each_row JSONEachRow -kafka_group_name_new new -kafka_group_name_old old -kafka_topic_new new -kafka_topic_old old +kafka_group_name_new {KAFKA_CONSUMER_GROUP_NEW} +kafka_group_name_old {KAFKA_CONSUMER_GROUP_OLD} +kafka_topic_new new_t +kafka_topic_old old_t """ ) @@ -540,7 +542,7 @@ kafka_topic_old old messages = [] for i in range(50): messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "old", messages) + kafka_produce(kafka_cluster, KAFKA_TOPIC_OLD, messages) result = "" while True: @@ -550,7 +552,7 @@ kafka_topic_old old kafka_check_result(result, True) - members = describe_consumer_group(kafka_cluster, "old") + members = describe_consumer_group(kafka_cluster, KAFKA_CONSUMER_GROUP_OLD) assert members[0]["client_id"] == "ClickHouse-instance-test-kafka" # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:9092 --describe --members --group old --verbose")) @@ -593,7 +595,7 @@ def test_kafka_settings_new_syntax(kafka_cluster): kafka_check_result(result, True) - members = describe_consumer_group(kafka_cluster, "new") + members = describe_consumer_group(kafka_cluster, KAFKA_CONSUMER_GROUP_NEW) assert members[0]["client_id"] == "instance test 1234" From 7b90eccd57e2ae0a2379a1fb72c5e5ea48d552b2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 8 Jun 2024 22:26:00 +0000 Subject: [PATCH 088/644] Automatic style fix --- tests/integration/test_storage_kafka/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index d4d47311483..74532de4878 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2862,7 +2862,6 @@ def test_kafka_produce_consume_avro(kafka_cluster, create_query_generator): assert int(expected_max_key) == (num_rows - 1) * 10 - @pytest.mark.parametrize( "create_query_generator", [generate_old_create_table_query, generate_new_create_table_query], From eb8064aea67c01ef65ebd1c307c77f16bfc93199 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Sat, 8 Jun 2024 23:13:58 +0000 Subject: [PATCH 089/644] Style fix --- src/Storages/Kafka/StorageKafka2.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 0bc484b4d45..15a5388aa0b 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -72,7 +72,6 @@ extern const Metric KafkaWrites; namespace ProfileEvents { -extern const Event KafkaDirectReads; extern const Event KafkaBackgroundReads; extern const Event KafkaMessagesRead; extern const Event KafkaMessagesFailed; From 90b5ad3613ea7e3b4dea202975407569d0aaee84 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 9 Jun 2024 19:31:20 +0000 Subject: [PATCH 090/644] fix tidy build --- src/IO/WriteBufferFromPocoSocketChunked.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 689389ba2ea..ecc33180140 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -85,7 +85,10 @@ protected: void nextImpl() override { if (!chunked) - return WriteBufferFromPocoSocket::nextImpl(); + { + WriteBufferFromPocoSocket::nextImpl(); + return; + } /// next() after finishChunk ar the end of the buffer if (finishing < sizeof(*chunk_size_ptr)) From b40c93165142579e871c064d02d2d60d6c526e86 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 11 Jun 2024 09:20:22 +0000 Subject: [PATCH 091/644] Set `shutdown-called` flag on shutdown to stop writes --- src/Storages/Kafka/StorageKafka2.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 15a5388aa0b..6a30045fd1f 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -286,6 +286,7 @@ void StorageKafka2::startup() void StorageKafka2::shutdown(bool) { + shutdown_called = true; for (auto & task : tasks) { // Interrupt streaming thread From 0691c01427fac16a1b469a4ec11104e7d49569b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 11 Jun 2024 09:21:12 +0000 Subject: [PATCH 092/644] Fix crash in case of consumer receives no consumer groups on assignment --- src/Storages/Kafka/StorageKafka2.cpp | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 6a30045fd1f..f75e6044ddd 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -289,9 +289,13 @@ void StorageKafka2::shutdown(bool) shutdown_called = true; for (auto & task : tasks) { + LOG_TRACE(log, "Cancelling streams"); // Interrupt streaming thread task->stream_cancelled = true; + } + for (auto & task : tasks) + { LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); } @@ -1108,16 +1112,8 @@ void StorageKafka2::threadFunc(size_t idx) bool StorageKafka2::streamToViews(size_t idx) { - // What to do? - // 1. Select a topic partition to consume from - // 2. Do a casual poll for every other consumer to keep them alive - // 3. Get the necessary data from Keeper - // 4. Get the corresponding consumer - // 5. Pull messages - // 6. Create a BlockList from it - // 7. Execute the pipeline - // 8. Write the offset to Keeper - + // This function is written assuming that each consumer has their own thread. This means once this is changed, this function should be revisited. + // The return values should be revisited, as stalling all consumers because of a single one stalled is not a good idea. auto table_id = getStorageID(); auto table = DatabaseCatalog::instance().getTable(table_id, getContext()); if (!table) @@ -1189,6 +1185,11 @@ bool StorageKafka2::streamToViews(size_t idx) consumer_info.consumer->updateOffsets(consumer_info.topic_partitions); } + if (consumer_info.topic_partitions.empty()) + { + LOG_TRACE(log, "Consumer {} has assignment, but has no partitions, probably because there are more consumers in the consumer group than partitions.", idx); + return true; + } LOG_TRACE(log, "Trying to consume from consumer {}", idx); const auto maybe_rows = streamFromConsumer(consumer_info); if (maybe_rows.has_value()) From fb49cf503e4159549348c76ebf9c3ca686b9f02f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 11 Jun 2024 16:47:05 +0000 Subject: [PATCH 093/644] some fixes --- src/IO/WriteBufferFromPocoSocketChunked.h | 31 ++++++++++++++++++++++- 1 file changed, 30 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index ecc33180140..4325ab2bd4b 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -35,7 +35,7 @@ public: /// If current chunk is empty it means we are finishing a chunk previously sent by next(), /// we want to convert current chunk header into end-of-chunk marker and initialize next chunk. - /// We don't need to wary about if it's the end of the buffer because next() always sends the whole buffer + /// We don't need to worry about if it's the end of the buffer because next() always sends the whole buffer /// so it should be a beginning of the buffer. chassert(reinterpret_cast(chunk_size_ptr) == working_buffer.begin()); @@ -50,6 +50,13 @@ public: return; } + /// Previously finished chunk wasn't sent yet + if (last_finish_chunk == chunk_size_ptr) + { + chunk_started = false; + LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); + } + /// Fill up current chunk size *chunk_size_ptr = toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr))); @@ -59,7 +66,10 @@ public: static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), *chunk_size_ptr); else + { chunk_started = false; + LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), *chunk_size_ptr); + } LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); @@ -81,6 +91,18 @@ public: last_finish_chunk = chunk_size_ptr; } + ~WriteBufferFromPocoSocketChunked() override + { + try + { + finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + protected: void nextImpl() override { @@ -173,6 +195,13 @@ protected: last_finish_chunk = nullptr; } + void finalizeImpl() override + { + if (offset() == sizeof(*chunk_size_ptr)) + pos -= sizeof(*chunk_size_ptr); + WriteBufferFromPocoSocket::finalizeImpl(); + } + Poco::Net::SocketAddress peerAddress() { return peer_address; From 11bda3f5f749375301fdc944693984a744ba81ab Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 12 Jun 2024 18:23:50 +0000 Subject: [PATCH 094/644] Create consumers in startup --- src/Storages/Kafka/StorageKafka2.cpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index f75e6044ddd..de27adbbe3f 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -278,6 +278,18 @@ StorageKafka2::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapsho void StorageKafka2::startup() { + for (size_t i = 0; i < num_consumers; ++i) + { + try + { + consumers.emplace_back(ConsumerAndAssignmentInfo{.consumer = createConsumer(i), .keeper = keeper}); + ++num_created_consumers; + } + catch (const cppkafka::Exception &) + { + tryLogCurrentException(log); + } + } // Start the reader thread for (auto & task : tasks) task->holder->activateAndSchedule(); From ba76a06f5677e7de556781a4c06cc947f392e0c5 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 14 Jun 2024 01:35:08 +0000 Subject: [PATCH 095/644] potentially very serious bug is fixed for secure socket --- src/IO/ReadBufferFromPocoSocket.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index af58efc7e10..6361fed01dd 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -134,11 +134,14 @@ ReadBufferFromPocoSocketBase::ReadBufferFromPocoSocketBase(Poco::Net::Socket & s bool ReadBufferFromPocoSocketBase::poll(size_t timeout_microseconds) const { - if (available()) + /// For secure socket it is important to check if any remaining data available in underlying decryption buffer - + /// read always retrives the whole encrypted frame from the wire and puts it into underlying buffer while returning only requested size - + /// further poll() can block though there is still data to read in the underlying decryption buffer. + if (available() || socket.impl()->available()) return true; Stopwatch watch; - bool res = socket.poll(timeout_microseconds, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR); + bool res = socket.impl()->poll(timeout_microseconds, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR); ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds()); return res; } From 97aea863767a58fd65274777913865201ea906e3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 14 Jun 2024 01:56:05 +0000 Subject: [PATCH 096/644] fix style --- src/IO/ReadBufferFromPocoSocket.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 6361fed01dd..bbf9f96404f 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -135,7 +135,7 @@ ReadBufferFromPocoSocketBase::ReadBufferFromPocoSocketBase(Poco::Net::Socket & s bool ReadBufferFromPocoSocketBase::poll(size_t timeout_microseconds) const { /// For secure socket it is important to check if any remaining data available in underlying decryption buffer - - /// read always retrives the whole encrypted frame from the wire and puts it into underlying buffer while returning only requested size - + /// read always retrieves the whole encrypted frame from the wire and puts it into underlying buffer while returning only requested size - /// further poll() can block though there is still data to read in the underlying decryption buffer. if (available() || socket.impl()->available()) return true; From 14a13d54c0ff56b0e6326ac75bb7136e44d814d1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 14 Jun 2024 15:56:14 +0000 Subject: [PATCH 097/644] fix UB misaligned address --- src/IO/WriteBufferFromPocoSocketChunked.h | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 4325ab2bd4b..c668ea2c505 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -6,6 +6,18 @@ #include +namespace +{ + +template +const T & setValue(T * typed_ptr, std::type_identity_t val) +{ + memcpy(typed_ptr, &val, sizeof(T)); + return *typed_ptr; +} + +} + namespace DB { @@ -40,7 +52,7 @@ public: chassert(reinterpret_cast(chunk_size_ptr) == working_buffer.begin()); - *chunk_size_ptr = 0; + setValue(chunk_size_ptr, 0); /// Initialize next chunk chunk_size_ptr = reinterpret_cast(pos); pos += std::min(available(), sizeof(*chunk_size_ptr)); @@ -58,7 +70,7 @@ public: } /// Fill up current chunk size - *chunk_size_ptr = toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr))); + setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); if (!chunk_started) LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", @@ -174,7 +186,7 @@ protected: pos -= sizeof(*chunk_size_ptr); else // fill up current chunk size { - *chunk_size_ptr = toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr))); + setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); if (!chunk_started) { chunk_started = true; From 5b082051451356b2c1d3152489e5d51cd75d2d6a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 15 Jun 2024 00:22:51 +0000 Subject: [PATCH 098/644] some refactoring --- src/IO/WriteBufferFromPocoSocketChunked.cpp | 207 ++++++++++++++++++ src/IO/WriteBufferFromPocoSocketChunked.h | 220 +------------------- 2 files changed, 217 insertions(+), 210 deletions(-) create mode 100644 src/IO/WriteBufferFromPocoSocketChunked.cpp diff --git a/src/IO/WriteBufferFromPocoSocketChunked.cpp b/src/IO/WriteBufferFromPocoSocketChunked.cpp new file mode 100644 index 00000000000..324f8ae3a02 --- /dev/null +++ b/src/IO/WriteBufferFromPocoSocketChunked.cpp @@ -0,0 +1,207 @@ +#include +#include +#include + + +namespace +{ + +template +const T & setValue(T * typed_ptr, std::type_identity_t val) +{ + memcpy(typed_ptr, &val, sizeof(T)); + return *typed_ptr; +} + +} + +namespace DB +{ + +void WriteBufferFromPocoSocketChunked::enableChunked() +{ + chunked = true; + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(pos); + pos += std::min(available(), sizeof(*chunk_size_ptr)); +} + +void WriteBufferFromPocoSocketChunked::finishChunk() +{ + if (!chunked) + return; + + if (pos <= reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr)) + { + if (chunk_size_ptr == last_finish_chunk) // prevent duplicate finish chunk + return; + + /// If current chunk is empty it means we are finishing a chunk previously sent by next(), + /// we want to convert current chunk header into end-of-chunk marker and initialize next chunk. + /// We don't need to worry about if it's the end of the buffer because next() always sends the whole buffer + /// so it should be a beginning of the buffer. + + chassert(reinterpret_cast(chunk_size_ptr) == working_buffer.begin()); + + setValue(chunk_size_ptr, 0); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(pos); + pos += std::min(available(), sizeof(*chunk_size_ptr)); + + last_finish_chunk = chunk_size_ptr; + + return; + } + + /// Previously finished chunk wasn't sent yet + if (last_finish_chunk == chunk_size_ptr) + { + chunk_started = false; + LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); + } + + /// Fill up current chunk size + setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); + + if (!chunk_started) + LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", + ourAddress().toString(), peerAddress().toString(), + static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), + *chunk_size_ptr); + else + { + chunk_started = false; + LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), *chunk_size_ptr); + } + + LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); + + if (available() < sizeof(*chunk_size_ptr)) + { + finishing = available(); + pos += available(); + chunk_size_ptr = reinterpret_cast(pos); + return; + } + + /// Buffer end-of-chunk + *reinterpret_cast(pos) = 0; + pos += sizeof(*chunk_size_ptr); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(pos); + pos += std::min(available(), sizeof(*chunk_size_ptr)); + + last_finish_chunk = chunk_size_ptr; +} + +WriteBufferFromPocoSocketChunked::~WriteBufferFromPocoSocketChunked() +{ + try + { + finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + +void WriteBufferFromPocoSocketChunked::nextImpl() +{ + if (!chunked) + { + WriteBufferFromPocoSocket::nextImpl(); + return; + } + + /// next() after finishChunk ar the end of the buffer + if (finishing < sizeof(*chunk_size_ptr)) + { + pos -= finishing; + /// Send current chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Send end-of-chunk directly + UInt32 s = 0; + socketSendBytes(reinterpret_cast(&s), sizeof(s)); + + finishing = sizeof(*chunk_size_ptr); + + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + last_finish_chunk = chunk_size_ptr; + + return; + } + + /// Send end-of-chunk buffered by finishChunk + if (offset() == 2 * sizeof(*chunk_size_ptr) && last_finish_chunk == chunk_size_ptr) + { + pos -= sizeof(*chunk_size_ptr); + /// Send end-of-chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + last_finish_chunk = chunk_size_ptr; + + return; + } + + /// Prevent sending empty chunk + if (offset() == sizeof(*chunk_size_ptr)) + { + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + return; + } + + /// Finish chunk at the end of the buffer + if (working_buffer.end() - reinterpret_cast(chunk_size_ptr) <= static_cast(sizeof(*chunk_size_ptr))) + { + pos = reinterpret_cast(chunk_size_ptr); + /// Send current chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + last_finish_chunk = nullptr; + + return; + } + + if (pos - reinterpret_cast(chunk_size_ptr) == sizeof(*chunk_size_ptr)) // next() after finishChunk + pos -= sizeof(*chunk_size_ptr); + else // fill up current chunk size + { + setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); + if (!chunk_started) + { + chunk_started = true; + LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", + ourAddress().toString(), peerAddress().toString(), + static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), + *chunk_size_ptr); + } + else + LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), *chunk_size_ptr); + } + /// Send current chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + last_finish_chunk = nullptr; +} + +void WriteBufferFromPocoSocketChunked::finalizeImpl() +{ + if (offset() == sizeof(*chunk_size_ptr)) + pos -= sizeof(*chunk_size_ptr); + WriteBufferFromPocoSocket::finalizeImpl(); +} + +} diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index c668ea2c505..269c6d66dda 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -1,23 +1,9 @@ #pragma once -#include "base/defines.h" #include #include -#include -namespace -{ - -template -const T & setValue(T * typed_ptr, std::type_identity_t val) -{ - memcpy(typed_ptr, &val, sizeof(T)); - return *typed_ptr; -} - -} - namespace DB { @@ -27,208 +13,22 @@ public: explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, buf_size), log(getLogger("Protocol")) {} explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, write_event_, buf_size), log(getLogger("Protocol")) {} - void enableChunked() - { - chunked = true; - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(pos); - pos += std::min(available(), sizeof(*chunk_size_ptr)); - } - - void finishChunk() - { - if (!chunked) - return; - - if (pos <= reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr)) - { - if (chunk_size_ptr == last_finish_chunk) // prevent duplicate finish chunk - return; - - /// If current chunk is empty it means we are finishing a chunk previously sent by next(), - /// we want to convert current chunk header into end-of-chunk marker and initialize next chunk. - /// We don't need to worry about if it's the end of the buffer because next() always sends the whole buffer - /// so it should be a beginning of the buffer. - - chassert(reinterpret_cast(chunk_size_ptr) == working_buffer.begin()); - - setValue(chunk_size_ptr, 0); - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(pos); - pos += std::min(available(), sizeof(*chunk_size_ptr)); - - last_finish_chunk = chunk_size_ptr; - - return; - } - - /// Previously finished chunk wasn't sent yet - if (last_finish_chunk == chunk_size_ptr) - { - chunk_started = false; - LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); - } - - /// Fill up current chunk size - setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); - - if (!chunk_started) - LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", - ourAddress().toString(), peerAddress().toString(), - static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), - *chunk_size_ptr); - else - { - chunk_started = false; - LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), *chunk_size_ptr); - } - - LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); - - if (available() < sizeof(*chunk_size_ptr)) - { - finishing = available(); - pos += available(); - chunk_size_ptr = reinterpret_cast(pos); - return; - } - - /// Buffer end-of-chunk - *reinterpret_cast(pos) = 0; - pos += sizeof(*chunk_size_ptr); - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(pos); - pos += std::min(available(), sizeof(*chunk_size_ptr)); - - last_finish_chunk = chunk_size_ptr; - } - - ~WriteBufferFromPocoSocketChunked() override - { - try - { - finalize(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } + void enableChunked(); + void finishChunk(); + ~WriteBufferFromPocoSocketChunked() override; protected: - void nextImpl() override - { - if (!chunked) - { - WriteBufferFromPocoSocket::nextImpl(); - return; - } + void nextImpl() override; + void finalizeImpl() override; + Poco::Net::SocketAddress peerAddress() const { return peer_address; } + Poco::Net::SocketAddress ourAddress() const { return our_address; } - /// next() after finishChunk ar the end of the buffer - if (finishing < sizeof(*chunk_size_ptr)) - { - pos -= finishing; - /// Send current chunk - WriteBufferFromPocoSocket::nextImpl(); - /// Send end-of-chunk directly - UInt32 s = 0; - socketSendBytes(reinterpret_cast(&s), sizeof(s)); - - finishing = sizeof(*chunk_size_ptr); - - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(working_buffer.begin()); - nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); - - last_finish_chunk = chunk_size_ptr; - - return; - } - - /// Send end-of-chunk buffered by finishChunk - if (offset() == 2 * sizeof(*chunk_size_ptr) && last_finish_chunk == chunk_size_ptr) - { - pos -= sizeof(*chunk_size_ptr); - /// Send end-of-chunk - WriteBufferFromPocoSocket::nextImpl(); - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(working_buffer.begin()); - nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); - - last_finish_chunk = chunk_size_ptr; - - return; - } - - /// Prevent sending empty chunk - if (offset() == sizeof(*chunk_size_ptr)) - { - nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); - return; - } - - /// Finish chunk at the end of the buffer - if (working_buffer.end() - reinterpret_cast(chunk_size_ptr) <= static_cast(sizeof(*chunk_size_ptr))) - { - pos = reinterpret_cast(chunk_size_ptr); - /// Send current chunk - WriteBufferFromPocoSocket::nextImpl(); - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(working_buffer.begin()); - nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); - - last_finish_chunk = nullptr; - - return; - } - - if (pos - reinterpret_cast(chunk_size_ptr) == sizeof(*chunk_size_ptr)) // next() after finishChunk - pos -= sizeof(*chunk_size_ptr); - else // fill up current chunk size - { - setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); - if (!chunk_started) - { - chunk_started = true; - LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", - ourAddress().toString(), peerAddress().toString(), - static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), - *chunk_size_ptr); - } - else - LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), *chunk_size_ptr); - } - /// Send current chunk - WriteBufferFromPocoSocket::nextImpl(); - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(working_buffer.begin()); - nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); - - last_finish_chunk = nullptr; - } - - void finalizeImpl() override - { - if (offset() == sizeof(*chunk_size_ptr)) - pos -= sizeof(*chunk_size_ptr); - WriteBufferFromPocoSocket::finalizeImpl(); - } - - Poco::Net::SocketAddress peerAddress() - { - return peer_address; - } - - Poco::Net::SocketAddress ourAddress() - { - return our_address; - } private: LoggerPtr log; bool chunked = false; - UInt32 * last_finish_chunk = nullptr; // pointer to the last chunk header created by finishChunk - bool chunk_started = false; // chunk started flag - UInt32 * chunk_size_ptr = nullptr; // pointer to the chunk size holder in the buffer + UInt32 * last_finish_chunk = nullptr; // pointer to the last chunk header created by finishChunk + bool chunk_started = false; // chunk started flag + UInt32 * chunk_size_ptr = nullptr; // pointer to the chunk size holder in the buffer size_t finishing = sizeof(*chunk_size_ptr); // indicates not enough buffer for end-of-chunk marker }; From c448b0e2ae366c25b5ef5943d550acd181b04fd1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 18 Jun 2024 08:21:10 +0000 Subject: [PATCH 099/644] Do not require setting experimental flag on attach --- src/Storages/Kafka/StorageKafkaCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp index adfe1086858..2e2d53dd8d1 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.cpp +++ b/src/Storages/Kafka/StorageKafkaCommon.cpp @@ -467,7 +467,7 @@ void registerStorageKafka(StorageFactory & factory) return std::make_shared( args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); - if (!args.getLocalContext()->getSettingsRef().allow_experimental_kafka_store_offsets_in_keeper) + if (!args.getLocalContext()->getSettingsRef().allow_experimental_kafka_store_offsets_in_keeper && !args.query.attach) throw Exception( ErrorCodes::SUPPORT_IS_DISABLED, "Storing the Kafka offsets in Keeper is experimental. Set `allow_experimental_kafka_store_offsets_in_keeper` setting " From 20bac3ed5fff99b7c31a185506022c0cb39ade85 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 18 Jun 2024 10:25:46 +0000 Subject: [PATCH 100/644] Remove unnecessary rdkafka ops --- src/Storages/Kafka/KafkaConsumer2.cpp | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 1e2ea3fd43a..41ce7c43131 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -194,16 +194,13 @@ void KafkaConsumer2::drainConsumerQueue() void KafkaConsumer2::pollEvents() { - static constexpr int64_t max_tries = 5; - auto consumer_has_subscription = !consumer->get_subscription().empty(); - for (auto i = 0; i < max_tries && !consumer_has_subscription; ++i) - { - consumer->subscribe(topics); - consumer_has_subscription = !consumer->get_subscription().empty(); - } auto msg = consumer->poll(EVENT_POLL_TIMEOUT); // All the partition queues are detached, so the consumer shouldn't be able to poll any messages chassert(!msg && "Consumer returned a message when it was not expected"); + + // static constexpr int64_t max_tries = 5; + // for(auto i = 0; i < max_tries; ++i) + // consumer->poll(EVENT_POLL_TIMEOUT); }; KafkaConsumer2::TopicPartitionCounts KafkaConsumer2::getPartitionCounts() const From 7fa6111865093e0f7e396faae4013275df371fcc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 18 Jun 2024 11:41:46 +0000 Subject: [PATCH 101/644] Make StorageKafka2 handle keeper session better --- src/Storages/Kafka/StorageKafka2.cpp | 384 +++++++++++++++++++-------- src/Storages/Kafka/StorageKafka2.h | 35 ++- 2 files changed, 303 insertions(+), 116 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index de27adbbe3f..86330cbf122 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -92,6 +93,8 @@ extern const int LOGICAL_ERROR; extern const int REPLICA_ALREADY_EXISTS; extern const int TABLE_IS_DROPPED; extern const int TABLE_WAS_NOT_DROPPED; +extern const int NO_ZOOKEEPER; +extern const int REPLICA_IS_ALREADY_ACTIVE; } namespace @@ -109,6 +112,8 @@ StorageKafka2::StorageKafka2( : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , keeper(getContext()->getZooKeeper()) + , keeper_path(kafka_settings_->kafka_keeper_path.value) + , replica_path(keeper_path + "/replicas/" + kafka_settings_->kafka_replica_name.value) , kafka_settings(std::move(kafka_settings_)) , macros_info{.table_id = table_id_} , topics(parseTopics(getContext()->getMacros()->expand(kafka_settings->kafka_topic_list.value, macros_info))) @@ -122,11 +127,12 @@ StorageKafka2::StorageKafka2( , max_rows_per_message(kafka_settings->kafka_max_rows_per_message.value) , schema_name(getContext()->getMacros()->expand(kafka_settings->kafka_schema.value, macros_info)) , num_consumers(kafka_settings->kafka_num_consumers.value) - , log(getLogger("StorageKafka2 (" + table_id_.table_name + ")")) + , log(getLogger(String("StorageKafka2 ") + table_id_.getNameForLogs())) , semaphore(0, static_cast(num_consumers)) , settings_adjustments(createSettingsAdjustments()) , thread_per_consumer(kafka_settings->kafka_thread_per_consumer.value) , collection_name(collection_name_) + , active_node_identifier(toString(ServerUUID::get())) { if (kafka_settings->kafka_num_consumers > 1 && !thread_per_consumer) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "With multiple consumer you have to use thread per consumer!"); @@ -149,23 +155,13 @@ StorageKafka2::StorageKafka2( tasks.emplace_back(std::make_shared(std::move(task))); } - for (size_t i = 0; i < num_consumers; ++i) - { - try - { - consumers.push_back(ConsumerAndAssignmentInfo{.consumer = createConsumer(i), .keeper = keeper}); - ++num_created_consumers; - } - catch (const cppkafka::Exception &) - { - tryLogCurrentException(log); - } - } - const auto first_replica = createTableIfNotExists(); if (!first_replica) createReplica(); + + activating_task = getContext()->getSchedulePool().createTask(log->name() + "(activating task)", [this]() { activate(); }); + activating_task->deactivate(); } VirtualColumnsDescription StorageKafka2::createVirtuals(StreamingHandleErrorMode handle_error_mode) @@ -189,6 +185,150 @@ VirtualColumnsDescription StorageKafka2::createVirtuals(StreamingHandleErrorMode return desc; } +void StorageKafka2::partialShutdown() +{ + for (auto & task : tasks) + { + LOG_TRACE(log, "Cancelling streams"); + task->stream_cancelled = true; + } + + for (auto & task : tasks) + { + LOG_TRACE(log, "Waiting for cleanup"); + task->holder->deactivate(); + } + is_active = false; +} + +bool StorageKafka2::activate() +{ + LOG_TEST(log, "activate task"); + if (is_active && !getZooKeeper()->expired()) + { + LOG_TEST(log, "No need to activate"); + return true; + } + + if (first_time) + { + LOG_DEBUG(log, "Activating replica"); + assert(!is_active); + } + else if (!is_active) + { + LOG_WARNING(log, "Table was not active. Will try to activate it"); + } + else if (getZooKeeper()->expired()) + { + LOG_WARNING(log, "ZooKeeper session has expired. Switching to a new session"); + partialShutdown(); + } + else + { + UNREACHABLE(); + } + + try + { + setZooKeeper(); + } + catch (const Coordination::Exception &) + { + /// The exception when you try to zookeeper_init usually happens if DNS does not work or the connection with ZK fails + tryLogCurrentException(log, "Failed to establish a new ZK connection. Will try again"); + assert(!is_active); + return false; + } + + if (shutdown_called) + return false; + + auto activate_in_keeper = [this]() + { + try + { + auto zookeeper = getZooKeeper(); + + String is_active_path = fs::path(replica_path) / "is_active"; + zookeeper->deleteEphemeralNodeIfContentMatches(is_active_path, active_node_identifier); + + /// Simultaneously declare that this replica is active, and update the host. + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest(is_active_path, active_node_identifier, zkutil::CreateMode::Ephemeral)); + + try + { + zookeeper->create(is_active_path, active_node_identifier, zkutil::CreateMode::Ephemeral); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::Error::ZNODEEXISTS) + throw Exception( + ErrorCodes::REPLICA_IS_ALREADY_ACTIVE, + "Replica {} appears to be already active. If you're sure it's not, " + "try again in a minute or remove znode {}/is_active manually", + replica_path, + replica_path); + + throw; + } + replica_is_active_node = zkutil::EphemeralNodeHolder::existing(is_active_path, *zookeeper); + + return true; + } + catch (...) + { + replica_is_active_node = nullptr; + + try + { + throw; + } + catch (const Coordination::Exception & e) + { + LOG_ERROR(log, "Couldn't start replica: {}. {}", e.what(), DB::getCurrentExceptionMessage(true)); + return false; + } + catch (const Exception & e) + { + if (e.code() != ErrorCodes::REPLICA_IS_ALREADY_ACTIVE) + throw; + + LOG_ERROR(log, "Couldn't start replica: {}. {}", e.what(), DB::getCurrentExceptionMessage(true)); + return false; + } + } + }; + + if (!activate_in_keeper()) + { + assert(storage.is_readonly); + return false; + } + + is_active = true; + + // Start the reader threads + for (auto & task : tasks) + { + task->stream_cancelled = false; + task->holder->activateAndSchedule(); + } + + if (first_time) + first_time = false; + + LOG_DEBUG(log, "Table activated successfully"); + return true; +} + +void StorageKafka2::assertActive() const +{ + // TODO(antaljanosbenjamin): change LOGICAL_ERROR to something sensible + if (!is_active) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table is not active (replica path: {})", replica_path); +} SettingsChanges StorageKafka2::createSettingsAdjustments() { @@ -282,7 +422,8 @@ void StorageKafka2::startup() { try { - consumers.emplace_back(ConsumerAndAssignmentInfo{.consumer = createConsumer(i), .keeper = keeper}); + consumers.push_back(ConsumerAndAssignmentInfo{.consumer = createConsumer(i), .keeper = getZooKeeper()}); + LOG_DEBUG(log, "Created #{} consumer", num_created_consumers); ++num_created_consumers; } catch (const cppkafka::Exception &) @@ -290,28 +431,15 @@ void StorageKafka2::startup() tryLogCurrentException(log); } } - // Start the reader thread - for (auto & task : tasks) - task->holder->activateAndSchedule(); + activating_task->activateAndSchedule(); } void StorageKafka2::shutdown(bool) { shutdown_called = true; - for (auto & task : tasks) - { - LOG_TRACE(log, "Cancelling streams"); - // Interrupt streaming thread - task->stream_cancelled = true; - } - - for (auto & task : tasks) - { - LOG_TRACE(log, "Waiting for cleanup"); - task->holder->deactivate(); - } - + activating_task->deactivate(); + partialShutdown(); LOG_TRACE(log, "Closing consumers"); consumers.clear(); LOG_TRACE(log, "Consumers closed"); @@ -372,7 +500,7 @@ cppkafka::Configuration StorageKafka2::getConsumerConfiguration(size_t consumer_ conf.set("enable.partition.eof", "false"); // Ignore EOF messages for (auto & property : conf.get_all()) - LOG_TRACE(log, "Consumer set property {}:{}", property.first, property.second); + LOG_TEST(log, "Consumer set property {}:{}", property.first, property.second); return conf; } @@ -389,7 +517,7 @@ cppkafka::Configuration StorageKafka2::getProducerConfiguration() updateProducerConfiguration(conf); for (auto & property : conf.get_all()) - LOG_TRACE(log, "Producer set property {}:{}", property.first, property.second); + LOG_TEST(log, "Producer set property {}:{}", property.first, property.second); return conf; } @@ -558,32 +686,32 @@ std::optional getNumber(zkutil::ZooKeeper & keeper, const fs::path & pa bool StorageKafka2::createTableIfNotExists() { - const auto & keeper_path = fs::path(kafka_settings->kafka_keeper_path.value); - - const auto & replicas_path = keeper_path / "replicas"; + // Heavily based on StorageReplicatedMergeTree::createTableIfNotExists + const auto my_keeper_path = fs::path(keeper_path); + const auto replicas_path = my_keeper_path / "replicas"; for (auto i = 0; i < 1000; ++i) { if (keeper->exists(replicas_path)) { - LOG_DEBUG(log, "This table {} is already created, will add new replica", String(keeper_path)); + LOG_DEBUG(log, "This table {} is already created, will add new replica", keeper_path); return false; } /// There are leftovers from incompletely dropped table. - if (keeper->exists(keeper_path / "dropped")) + if (keeper->exists(my_keeper_path / "dropped")) { /// This condition may happen when the previous drop attempt was not completed /// or when table is dropped by another replica right now. /// This is Ok because another replica is definitely going to drop the table. - LOG_WARNING(log, "Removing leftovers from table {} (this might take several minutes)", String(keeper_path)); - String drop_lock_path = keeper_path / "dropped" / "lock"; + LOG_WARNING(log, "Removing leftovers from table {}", keeper_path); + String drop_lock_path = my_keeper_path / "dropped" / "lock"; Coordination::Error code = keeper->tryCreate(drop_lock_path, "", zkutil::CreateMode::Ephemeral); if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) { - LOG_WARNING(log, "The leftovers from table {} were removed by another replica", String(keeper_path)); + LOG_WARNING(log, "The leftovers from table {} were removed by another replica", keeper_path); } else if (code != Coordination::Error::ZOK) { @@ -592,7 +720,7 @@ bool StorageKafka2::createTableIfNotExists() else { auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(drop_lock_path, *keeper); - if (!removeTableNodesFromZooKeeper(metadata_drop_lock)) + if (!removeTableNodesFromZooKeeper(keeper, metadata_drop_lock)) { /// Someone is recursively removing table right now, we cannot create new table until old one is removed continue; @@ -605,7 +733,7 @@ bool StorageKafka2::createTableIfNotExists() ops.emplace_back(zkutil::makeCreateRequest(keeper_path, "", zkutil::CreateMode::Persistent)); - const auto topics_path = keeper_path / "topics"; + const auto topics_path = my_keeper_path / "topics"; ops.emplace_back(zkutil::makeCreateRequest(topics_path, "", zkutil::CreateMode::Persistent)); for (const auto & topic : topics) @@ -621,15 +749,14 @@ bool StorageKafka2::createTableIfNotExists() // Create the first replica ops.emplace_back(zkutil::makeCreateRequest(replicas_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back( - zkutil::makeCreateRequest(replicas_path / kafka_settings->kafka_replica_name.value, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path, "", zkutil::CreateMode::Persistent)); Coordination::Responses responses; const auto code = keeper->tryMulti(ops, responses); if (code == Coordination::Error::ZNODEEXISTS) { - LOG_INFO(log, "It looks like the table {} was created by another replica at the same moment, will retry", String(keeper_path)); + LOG_INFO(log, "It looks like the table {} was created by another replica at the same moment, will retry", keeper_path); continue; } else if (code != Coordination::Error::ZOK) @@ -637,7 +764,7 @@ bool StorageKafka2::createTableIfNotExists() zkutil::KeeperMultiException::check(code, ops, responses); } - LOG_INFO(log, "Table {} created successfully ", String(keeper_path)); + LOG_INFO(log, "Table {} created successfully ", keeper_path); return true; } @@ -649,25 +776,25 @@ bool StorageKafka2::createTableIfNotExists() } -bool StorageKafka2::removeTableNodesFromZooKeeper(const zkutil::EphemeralNodeHolder::Ptr & drop_lock) +bool StorageKafka2::removeTableNodesFromZooKeeper(zkutil::ZooKeeperPtr keeper_to_use, const zkutil::EphemeralNodeHolder::Ptr & drop_lock) { bool completely_removed = false; Strings children; - if (const auto code = keeper->tryGetChildren(kafka_settings->kafka_keeper_path.value, children); code == Coordination::Error::ZNONODE) + if (const auto code = keeper_to_use->tryGetChildren(keeper_path, children); code == Coordination::Error::ZNONODE) throw Exception(ErrorCodes::LOGICAL_ERROR, "There is a race condition between creation and removal. It's a bug"); - const auto keeper_path = fs::path(kafka_settings->kafka_keeper_path.value); + const auto my_keeper_path = fs::path(keeper_path); for (const auto & child : children) if (child != "dropped") - keeper->tryRemoveRecursive(keeper_path / child); + keeper_to_use->tryRemoveRecursive(my_keeper_path / child); Coordination::Requests ops; Coordination::Responses responses; ops.emplace_back(zkutil::makeRemoveRequest(drop_lock->getPath(), -1)); - ops.emplace_back(zkutil::makeRemoveRequest(keeper_path / "dropped", -1)); - ops.emplace_back(zkutil::makeRemoveRequest(keeper_path, -1)); - const auto code = keeper->tryMulti(ops, responses, /* check_session_valid */ true); + ops.emplace_back(zkutil::makeRemoveRequest(my_keeper_path / "dropped", -1)); + ops.emplace_back(zkutil::makeRemoveRequest(my_keeper_path, -1)); + const auto code = keeper_to_use->tryMulti(ops, responses, /* check_session_valid */ true); if (code == Coordination::Error::ZNONODE) { @@ -680,7 +807,7 @@ bool StorageKafka2::removeTableNodesFromZooKeeper(const zkutil::EphemeralNodeHol log, "Table was not completely removed from Keeper, {} still exists and may contain some garbage," "but someone is removing it right now.", - kafka_settings->kafka_keeper_path.value); + keeper_path); } else if (code != Coordination::Error::ZOK) { @@ -691,7 +818,7 @@ bool StorageKafka2::removeTableNodesFromZooKeeper(const zkutil::EphemeralNodeHol { drop_lock->setAlreadyRemoved(); completely_removed = true; - LOG_INFO(log, "Table {} was successfully removed from ZooKeeper", kafka_settings->kafka_keeper_path.value); + LOG_INFO(log, "Table {} was successfully removed from ZooKeeper", keeper_path); } return completely_removed; @@ -699,45 +826,50 @@ bool StorageKafka2::removeTableNodesFromZooKeeper(const zkutil::EphemeralNodeHol void StorageKafka2::createReplica() { - const auto replica_path = kafka_settings->kafka_keeper_path.value + "/replicas/" + kafka_settings->kafka_replica_name.value; + LOG_INFO(log, "Creating replica {}", replica_path); + // TODO: This can cause issues if a new table is created with the same path. To make this work, we should store some metadata + // about the table to be able to identify that the same table is created, not a new one. const auto code = keeper->tryCreate(replica_path, "", zkutil::CreateMode::Persistent); - if (code == Coordination::Error::ZNODEEXISTS) - throw Exception(ErrorCodes::REPLICA_ALREADY_EXISTS, "Replica {} already exists", replica_path); - else if (code == Coordination::Error::ZNONODE) - throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} was suddenly removed", kafka_settings->kafka_keeper_path.value); - else if (code != Coordination::Error::ZOK) - throw Coordination::Exception::fromPath(code, replica_path); - LOG_INFO(log, "Replica {} created", replica_path); + switch (code) + { + case Coordination::Error::ZNODEEXISTS: + LOG_INFO(log, "Replica {} already exists, will try to use it", replica_path); + break; + case Coordination::Error::ZOK: + LOG_INFO(log, "Replica {} created", replica_path); + break; + case Coordination::Error::ZNONODE: + throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} was suddenly removed", keeper_path); + default: + throw Coordination::Exception::fromPath(code, replica_path); + } } void StorageKafka2::dropReplica() { - if (keeper->expired()) - throw Exception(ErrorCodes::TABLE_WAS_NOT_DROPPED, "Table was not dropped because ZooKeeper session has expired."); - - auto replica_path = kafka_settings->kafka_keeper_path.value + "/replicas/" + kafka_settings->kafka_replica_name.value; + LOG_INFO(log, "Trying to drop replica {}", replica_path); + auto my_keeper = getZooKeeperIfTableShutDown(); LOG_INFO(log, "Removing replica {}", replica_path); - if (!keeper->exists(replica_path)) + if (!my_keeper->exists(replica_path)) { LOG_INFO(log, "Removing replica {} does not exist", replica_path); return; } { - keeper->tryRemoveChildrenRecursive(replica_path); + my_keeper->tryRemoveChildrenRecursive(replica_path); - if (keeper->tryRemove(replica_path) != Coordination::Error::ZOK) + if (my_keeper->tryRemove(replica_path) != Coordination::Error::ZOK) LOG_ERROR(log, "Replica was not completely removed from Keeper, {} still exists and may contain some garbage.", replica_path); } /// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line. Strings replicas; - if (Coordination::Error::ZOK != keeper->tryGetChildren(kafka_settings->kafka_keeper_path.value + "/replicas", replicas) - || !replicas.empty()) + if (Coordination::Error::ZOK != my_keeper->tryGetChildren(keeper_path + "/replicas", replicas) || !replicas.empty()) return; LOG_INFO(log, "{} is the last replica, will remove table", replica_path); @@ -758,11 +890,12 @@ void StorageKafka2::dropReplica() /// (The existence of child node does not allow to remove parent node). Coordination::Requests ops; Coordination::Responses responses; - String drop_lock_path = kafka_settings->kafka_keeper_path.value + "/dropped/lock"; - ops.emplace_back(zkutil::makeRemoveRequest(kafka_settings->kafka_keeper_path.value + "/replicas", -1)); - ops.emplace_back(zkutil::makeCreateRequest(kafka_settings->kafka_keeper_path.value + "/dropped", "", zkutil::CreateMode::Persistent)); + fs::path my_keeper_path = keeper_path; + String drop_lock_path = my_keeper_path / "dropped" / "lock"; + ops.emplace_back(zkutil::makeRemoveRequest(my_keeper_path / "replicas", -1)); + ops.emplace_back(zkutil::makeCreateRequest(my_keeper_path / "dropped", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(drop_lock_path, "", zkutil::CreateMode::Ephemeral)); - Coordination::Error code = keeper->tryMulti(ops, responses); + Coordination::Error code = my_keeper->tryMulti(ops, responses); if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) { @@ -778,9 +911,9 @@ void StorageKafka2::dropReplica() } else { - auto drop_lock = zkutil::EphemeralNodeHolder::existing(drop_lock_path, *keeper); - LOG_INFO(log, "Removing table {} (this might take several minutes)", kafka_settings->kafka_keeper_path.value); - removeTableNodesFromZooKeeper(drop_lock); + auto drop_lock = zkutil::EphemeralNodeHolder::existing(drop_lock_path, *my_keeper); + LOG_INFO(log, "Removing table {} (this might take several minutes)", keeper_path); + removeTableNodesFromZooKeeper(my_keeper, drop_lock); } } @@ -1073,6 +1206,7 @@ void StorageKafka2::threadFunc(size_t idx) { assert(idx < tasks.size()); auto task = tasks[idx]; + std::optional maybe_stall_reason; try { auto table_id = getStorageID(); @@ -1082,21 +1216,19 @@ void StorageKafka2::threadFunc(size_t idx) { auto start_time = std::chrono::steady_clock::now(); - mv_attached.store(true); - // Keep streaming as long as there are attached views and streaming is not cancelled while (!task->stream_cancelled && num_created_consumers > 0) { + maybe_stall_reason.reset(); if (!checkDependencies(table_id)) break; LOG_DEBUG(log, "Started streaming to {} attached views", num_views); // Exit the loop & reschedule if some stream stalled - auto some_stream_is_stalled = streamToViews(idx); - if (some_stream_is_stalled) + if (maybe_stall_reason = streamToViews(idx); maybe_stall_reason.has_value()) { - LOG_TRACE(log, "Stream(s) stalled. Reschedule."); + LOG_TRACE(log, "Stream stalled."); break; } @@ -1115,14 +1247,18 @@ void StorageKafka2::threadFunc(size_t idx) tryLogCurrentException(__PRETTY_FUNCTION__); } - mv_attached.store(false); - - // Wait for attached views if (!task->stream_cancelled) - task->holder->scheduleAfter(KAFKA_RESCHEDULE_MS); + { + // Keeper related problems should be solved relatively fast, it makes sense wait less time + if (maybe_stall_reason.has_value() + && (*maybe_stall_reason == StallReason::KeeperSessionEnded || *maybe_stall_reason == StallReason::CouldNotAcquireLocks)) + task->holder->scheduleAfter(KAFKA_RESCHEDULE_MS / 10); + else + task->holder->scheduleAfter(KAFKA_RESCHEDULE_MS); + } } -bool StorageKafka2::streamToViews(size_t idx) +std::optional StorageKafka2::streamToViews(size_t idx) { // This function is written assuming that each consumer has their own thread. This means once this is changed, this function should be revisited. // The return values should be revisited, as stalling all consumers because of a single one stalled is not a good idea. @@ -1140,21 +1276,24 @@ bool StorageKafka2::streamToViews(size_t idx) // To keep the consumer alive const auto wait_for_assignment = consumer_info.locks.empty(); - LOG_TRACE(log, "Polling consumer for events"); + LOG_TRACE(log, "Polling consumer {} for events", idx); consumer->pollEvents(); if (wait_for_assignment) { while (nullptr == consumer->getKafkaAssignment() && consumer_info.watch.elapsedMilliseconds() < MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS) consumer->pollEvents(); + LOG_INFO(log, "Consumer has assignment: {}", nullptr == consumer->getKafkaAssignment()); } try { if (consumer->needsOffsetUpdate() || consumer_info.locks.empty()) { + LOG_TRACE(log, "Consumer needs update offset"); // First release the locks so let other consumers acquire them ASAP consumer_info.locks.clear(); + consumer_info.topic_partitions.clear(); const auto * current_assignment = consumer->getKafkaAssignment(); if (current_assignment == nullptr) @@ -1162,13 +1301,15 @@ bool StorageKafka2::streamToViews(size_t idx) // The consumer lost its assignment and haven't received a new one. // By returning true this function reports the current consumer as a "stalled" stream, which LOG_TRACE(log, "No assignment"); - return true; + return StallReason::NoAssignment; } - LOG_TRACE(log, "Consumer needs update offset"); consumer_info.consume_from_topic_partition_index = 0; - consumer_info.locks.clear(); - consumer_info.topic_partitions.clear(); + if (consumer_info.keeper->expired()) + { + consumer_info.keeper = getZooKeeperAndAssertActive(); + LOG_TEST(log, "Got new zookeeper"); + } auto maybe_locks = lockTopicPartitions(*consumer_info.keeper, *current_assignment); @@ -1176,7 +1317,7 @@ bool StorageKafka2::streamToViews(size_t idx) { // We couldn't acquire locks, probably some other consumers are still holding them. LOG_TRACE(log, "Couldn't acquire locks"); - return true; + return StallReason::CouldNotAcquireLocks; } consumer_info.locks = std::move(*maybe_locks); @@ -1200,7 +1341,7 @@ bool StorageKafka2::streamToViews(size_t idx) if (consumer_info.topic_partitions.empty()) { LOG_TRACE(log, "Consumer {} has assignment, but has no partitions, probably because there are more consumers in the consumer group than partitions.", idx); - return true; + return StallReason::NoPartitions; } LOG_TRACE(log, "Trying to consume from consumer {}", idx); const auto maybe_rows = streamFromConsumer(consumer_info); @@ -1213,22 +1354,22 @@ bool StorageKafka2::streamToViews(size_t idx) else { LOG_DEBUG(log, "Couldn't stream any messages"); - return true; + return StallReason::NoMessages; } } catch (const zkutil::KeeperException & e) { if (Coordination::isHardwareError(e.code)) { - // Clear ephemeral nodes here as we got a new keeper here + LOG_INFO(log, "Cleaning up topic-partitions locks because of exception: {}", e.displayText()); consumer_info.locks.clear(); - consumer_info.keeper = getZooKeeper(); - return true; + activating_task->schedule(); + return StallReason::KeeperSessionEnded; } throw; } - return false; + return {}; } @@ -1300,30 +1441,51 @@ std::optional StorageKafka2::streamFromConsumer(ConsumerAndAssignmentInf } lock_info.committed_offset = last_read_offset + 1; topic_partition.offset = last_read_offset + 1; - consumer_info.consumer->commit(topic_partition); saveCommittedOffset(keeper_to_use, topic_partition); + consumer_info.consumer->commit(topic_partition); lock_info.intent_size.reset(); needs_offset_reset = false; return rows; } - -zkutil::ZooKeeperPtr StorageKafka2::getZooKeeper() +void StorageKafka2::setZooKeeper() +{ + std::unique_lock lock{keeper_mutex}; + keeper = getContext()->getZooKeeper(); +} + +zkutil::ZooKeeperPtr StorageKafka2::tryGetZooKeeper() const { std::unique_lock lock{keeper_mutex}; - if (keeper->expired()) - { - keeper = keeper->startNewSession(); - } return keeper; } +zkutil::ZooKeeperPtr StorageKafka2::getZooKeeper() const +{ + auto res = tryGetZooKeeper(); + if (!res) + throw Exception(ErrorCodes::NO_ZOOKEEPER, "Cannot get ZooKeeper"); + return res; +} + +zkutil::ZooKeeperPtr StorageKafka2::getZooKeeperAndAssertActive() const +{ + auto res = getZooKeeper(); + assertActive(); + return res; +} + +zkutil::ZooKeeperPtr StorageKafka2::getZooKeeperIfTableShutDown() const +{ + zkutil::ZooKeeperPtr new_zookeeper = getContext()->getZooKeeper(); + new_zookeeper->sync(keeper_path); + return new_zookeeper; +} fs::path StorageKafka2::getTopicPartitionPath(const TopicPartition & topic_partition) { - return fs::path(kafka_settings->kafka_keeper_path.value) / "topics" / topic_partition.topic / "partitions" - / std::to_string(topic_partition.partition_id); + return fs::path(keeper_path) / "topics" / topic_partition.topic / "partitions" / std::to_string(topic_partition.partition_id); } } diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index a2cbdce51a0..99c97caf9da 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -127,8 +127,10 @@ private: }; // Configuration and state - std::mutex keeper_mutex; + mutable std::mutex keeper_mutex; zkutil::ZooKeeperPtr keeper; + String keeper_path; + String replica_path; std::unique_ptr kafka_settings; Macros::MacroExpansionInfo macros_info; const Names topics; @@ -142,7 +144,6 @@ private: LoggerPtr log; Poco::Semaphore semaphore; const SettingsChanges settings_adjustments; - std::atomic mv_attached = false; /// Can differ from num_consumers in case of exception in startup() (or if startup() hasn't been called). /// In this case we still need to be able to shutdown() properly. size_t num_created_consumers = 0; /// number of actually created consumers. @@ -156,6 +157,16 @@ private: String collection_name; std::atomic shutdown_called = false; + // Handling replica activation. + std::atomic is_active = false; + zkutil::EphemeralNodeHolderPtr replica_is_active_node; + BackgroundSchedulePool::TaskHolder activating_task; + String active_node_identifier; + bool first_time = true; + bool activate(); + void partialShutdown(); + + void assertActive() const; SettingsChanges createSettingsAdjustments(); KafkaConsumer2Ptr createConsumer(size_t consumer_number); // Returns full consumer related configuration, also the configuration @@ -186,7 +197,16 @@ private: static Names parseTopics(String topic_list); static String getDefaultClientId(const StorageID & table_id_); - bool streamToViews(size_t idx); + enum class StallReason + { + NoAssignment, + CouldNotAcquireLocks, + NoPartitions, + NoMessages, + KeeperSessionEnded, + }; + + std::optional streamToViews(size_t idx); std::optional streamFromConsumer(ConsumerAndAssignmentInfo & consumer_info); @@ -195,7 +215,7 @@ private: // Returns true if this is the first replica bool createTableIfNotExists(); // Returns true if all of the nodes were cleaned up - bool removeTableNodesFromZooKeeper(const zkutil::EphemeralNodeHolder::Ptr & drop_lock); + bool removeTableNodesFromZooKeeper(zkutil::ZooKeeperPtr keeper_to_use, const zkutil::EphemeralNodeHolder::Ptr & drop_lock); // Creates only the replica in ZooKeeper. Shouldn't be called on the first replica as it is created in createTableIfNotExists void createReplica(); void dropReplica(); @@ -212,7 +232,12 @@ private: Stopwatch & watch, const ContextPtr & context); - zkutil::ZooKeeperPtr getZooKeeper(); + void setZooKeeper(); + zkutil::ZooKeeperPtr tryGetZooKeeper() const; + zkutil::ZooKeeperPtr getZooKeeper() const; + zkutil::ZooKeeperPtr getZooKeeperAndAssertActive() const; + zkutil::ZooKeeperPtr getZooKeeperIfTableShutDown() const; + std::filesystem::path getTopicPartitionPath(const TopicPartition & topic_partition); From 6e1a9015100d00113c6367619bb63bd9f04ecb0f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 18 Jun 2024 11:42:58 +0000 Subject: [PATCH 102/644] Make big chunk of integration tests to work with new storage kafka --- tests/integration/test_storage_kafka/test.py | 90 +++++++++++--------- 1 file changed, 49 insertions(+), 41 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 74532de4878..994ed8cb7b1 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1856,6 +1856,8 @@ def test_kafka_recreate_kafka_table(kafka_cluster, create_query_generator, log_l """ ) + instance.rotate_logs() + kafka_produce(kafka_cluster, "recreate_kafka_table", messages) instance.query(create_query) @@ -1935,15 +1937,17 @@ def test_librdkafka_compression(kafka_cluster, create_query_generator, log_line) instance.query( """{create_query}; - CREATE MATERIALIZED VIEW test.consumer Engine=Log AS + CREATE TABLE test.view (key UInt64, value String) + ENGINE = MergeTree() + ORDER BY key; + + CREATE MATERIALIZED VIEW test.consumer TO test.view AS SELECT * FROM test.kafka; """.format( create_query=create_query_generator( "kafka", "key UInt64, value String", topic_list=topic_name, - # brokers="kafka1:19092", - # consumer_group=f"{topic_name}_group", format="JSONEachRow", settings={"kafka_flush_interval_ms": 1000}, ), @@ -1955,12 +1959,12 @@ def test_librdkafka_compression(kafka_cluster, create_query_generator, log_line) instance.wait_for_log_line( log_line.format(offset=number_of_messages, topic=topic_name) ) - - result = instance.query("SELECT * FROM test.consumer") + result = instance.query("SELECT * FROM test.view") assert TSV(result) == TSV(expected) instance.query("DROP TABLE test.kafka SYNC") instance.query("DROP TABLE test.consumer SYNC") + instance.query("DROP TABLE test.view SYNC") @pytest.mark.parametrize( @@ -2586,22 +2590,23 @@ def test_kafka_virtual_columns2(kafka_cluster, create_query_generator, log_line) instance.rotate_logs() -# TODO(antaljanosbenjamin) -def test_kafka_producer_consumer_separate_settings(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator, do_direct_read", + [(generate_old_create_table_query, True), (generate_new_create_table_query, False)], +) +def test_kafka_producer_consumer_separate_settings(kafka_cluster, create_query_generator, do_direct_read): + instance.rotate_logs() instance.query( - """ - DROP TABLE IF EXISTS test.test_kafka; - CREATE TABLE test.test_kafka (key UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'separate_settings', - kafka_group_name = 'test', - kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\\n'; - """ + create_query_generator( + "test_kafka", + "key UInt64", + topic_list="separate_settings", + consumer_group="test" + ) ) - instance.query("SELECT * FROM test.test_kafka") + if do_direct_read: + instance.query("SELECT * FROM test.test_kafka") instance.query("INSERT INTO test.test_kafka VALUES (1)") assert instance.contains_in_log("Kafka producer created") @@ -2616,11 +2621,11 @@ def test_kafka_producer_consumer_separate_settings(kafka_cluster): # and producer configurations assert "heartbeat.interval.ms" in warn - kafka_consumer_applyed_properties = instance.grep_in_log("Consumer set property") - kafka_producer_applyed_properties = instance.grep_in_log("Producer set property") + kafka_consumer_applied_properties = instance.grep_in_log("Consumer set property") + kafka_producer_applied_properties = instance.grep_in_log("Producer set property") - assert kafka_consumer_applyed_properties is not None - assert kafka_producer_applyed_properties is not None + assert kafka_consumer_applied_properties is not None + assert kafka_producer_applied_properties is not None # global settings should be applied for consumer and producer global_settings = { @@ -2630,38 +2635,38 @@ def test_kafka_producer_consumer_separate_settings(kafka_cluster): for name, value in global_settings.items(): property_in_log = f"{name}:{value}" - assert property_in_log in kafka_consumer_applyed_properties - assert property_in_log in kafka_producer_applyed_properties + assert property_in_log in kafka_consumer_applied_properties + assert property_in_log in kafka_producer_applied_properties settings_topic__separate_settings__consumer = {"session.timeout.ms": "6001"} for name, value in settings_topic__separate_settings__consumer.items(): property_in_log = f"{name}:{value}" - assert property_in_log in kafka_consumer_applyed_properties - assert property_in_log not in kafka_producer_applyed_properties + assert property_in_log in kafka_consumer_applied_properties + assert property_in_log not in kafka_producer_applied_properties producer_settings = {"transaction.timeout.ms": "60001"} for name, value in producer_settings.items(): property_in_log = f"{name}:{value}" - assert property_in_log not in kafka_consumer_applyed_properties - assert property_in_log in kafka_producer_applyed_properties + assert property_in_log not in kafka_consumer_applied_properties + assert property_in_log in kafka_producer_applied_properties # Should be ignored, because it is inside producer tag producer_legacy_syntax__topic_separate_settings = {"message.timeout.ms": "300001"} for name, value in producer_legacy_syntax__topic_separate_settings.items(): property_in_log = f"{name}:{value}" - assert property_in_log not in kafka_consumer_applyed_properties - assert property_in_log not in kafka_producer_applyed_properties + assert property_in_log not in kafka_consumer_applied_properties + assert property_in_log not in kafka_producer_applied_properties # Old syntax, applied on consumer and producer legacy_syntax__topic_separated_settings = {"heartbeat.interval.ms": "302"} for name, value in legacy_syntax__topic_separated_settings.items(): property_in_log = f"{name}:{value}" - assert property_in_log in kafka_consumer_applyed_properties - assert property_in_log in kafka_producer_applyed_properties + assert property_in_log in kafka_consumer_applied_properties + assert property_in_log in kafka_producer_applied_properties @pytest.mark.parametrize( @@ -4324,7 +4329,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator "create_query_generator", [ generate_old_create_table_query, - # generate_new_create_table_query TODO(antaljanosbenjamin): crashes CH + generate_new_create_table_query, ], ) def test_kafka_consumer_failover(kafka_cluster, create_query_generator): @@ -4823,6 +4828,7 @@ def test_row_based_formats(kafka_cluster, create_query_generator): logging.debug("Checking {format_name}") topic_name = format_name + get_topic_postfix(create_query_generator) + table_name = f"kafka_{format_name}" with kafka_topic(admin_client, topic_name): num_rows = 10 @@ -4830,7 +4836,7 @@ def test_row_based_formats(kafka_cluster, create_query_generator): message_count = num_rows / max_rows_per_message create_query = create_query_generator( - "kafka", + table_name, "key UInt64, value UInt64", topic_list=topic_name, consumer_group=topic_name, @@ -4841,14 +4847,14 @@ def test_row_based_formats(kafka_cluster, create_query_generator): instance.query( f""" DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.kafka; + DROP TABLE IF EXISTS test.{table_name}; {create_query}; CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.kafka; + SELECT key, value FROM test.{table_name}; - INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}); + INSERT INTO test.{table_name} SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}); """ ) @@ -4934,9 +4940,11 @@ def test_block_based_formats_2(kafka_cluster, create_query_generator): "JSONCompactColumns", ]: topic_name = format_name + get_topic_postfix(create_query_generator) + table_name = f"kafka_{format_name}" + logging.debug(f"Checking format {format_name}") with kafka_topic(admin_client, topic_name): create_query = create_query_generator( - "kafka", + table_name, "key UInt64, value UInt64", topic_list=topic_name, consumer_group=topic_name, @@ -4946,14 +4954,14 @@ def test_block_based_formats_2(kafka_cluster, create_query_generator): instance.query( f""" DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.kafka; + DROP TABLE IF EXISTS test.{table_name}; {create_query}; CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.kafka; + SELECT key, value FROM test.{table_name}; - INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; + INSERT INTO test.{table_name} SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; """ ) messages = kafka_consume_with_retry( From 03500bbe2f62859cfe734b97558eeebf63332101 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 18 Jun 2024 11:43:05 +0000 Subject: [PATCH 103/644] Style fix --- tests/integration/test_storage_kafka/configs/kafka.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/configs/kafka.xml b/tests/integration/test_storage_kafka/configs/kafka.xml index b10db879b72..a846fdbb295 100644 --- a/tests/integration/test_storage_kafka/configs/kafka.xml +++ b/tests/integration/test_storage_kafka/configs/kafka.xml @@ -48,7 +48,7 @@ - 30001 + 30001 60001 From f2e4ec28cb034c709321e9c74ceafc1a6e28f26a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 18 Jun 2024 11:51:42 +0000 Subject: [PATCH 104/644] Automatic style fix --- tests/integration/test_storage_kafka/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 994ed8cb7b1..81132a9a60f 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2594,14 +2594,16 @@ def test_kafka_virtual_columns2(kafka_cluster, create_query_generator, log_line) "create_query_generator, do_direct_read", [(generate_old_create_table_query, True), (generate_new_create_table_query, False)], ) -def test_kafka_producer_consumer_separate_settings(kafka_cluster, create_query_generator, do_direct_read): +def test_kafka_producer_consumer_separate_settings( + kafka_cluster, create_query_generator, do_direct_read +): instance.rotate_logs() instance.query( create_query_generator( "test_kafka", "key UInt64", topic_list="separate_settings", - consumer_group="test" + consumer_group="test", ) ) From 65fc6fe8a363f0c80d947c62fcaf3a5f434086ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 18 Jun 2024 14:30:23 +0000 Subject: [PATCH 105/644] Fix style --- src/Storages/Kafka/KafkaConsumer2.cpp | 3 ++- src/Storages/Kafka/StorageKafka2.cpp | 1 - 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 41ce7c43131..84b6f5153ed 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -252,7 +252,8 @@ void KafkaConsumer2::updateOffsets(const TopicPartitions & topic_partitions) topic_partitions.begin(), topic_partitions.end(), std::back_inserter(original_topic_partitions), - [](const TopicPartition & tp) { + [](const TopicPartition & tp) + { return cppkafka::TopicPartition{tp.topic, tp.partition_id, tp.offset}; }); initializeQueues(original_topic_partitions); diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 86330cbf122..080201cae9e 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -92,7 +92,6 @@ extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; extern const int REPLICA_ALREADY_EXISTS; extern const int TABLE_IS_DROPPED; -extern const int TABLE_WAS_NOT_DROPPED; extern const int NO_ZOOKEEPER; extern const int REPLICA_IS_ALREADY_ACTIVE; } From 968c1d94bdf20814f406cfd26af120cc78c50486 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 18 Jun 2024 20:07:43 +0000 Subject: [PATCH 106/644] Add description to settings and settings changes --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 1 + src/Storages/Kafka/KafkaSettings.h | 4 ++-- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a2759285174..df675a09a04 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -705,7 +705,7 @@ class IColumn; M(UInt64, max_size_to_preallocate_for_aggregation, 100'000'000, "For how many elements it is allowed to preallocate space in all hash tables in total before aggregation", 0) \ \ M(Bool, kafka_disable_num_consumers_limit, false, "Disable limit on kafka_num_consumers that depends on the number of available CPU cores", 0) \ - M(Bool, allow_experimental_kafka_store_offsets_in_keeper, false, "Allow experimental feature to store Kafka related offsets in Keeper", 0) \ + M(Bool, allow_experimental_kafka_store_offsets_in_keeper, false, "Allow experimental feature to store Kafka related offsets in ClickHouse Keeper. When enabled a ClickHouse Keeper path and replica name can be specified to the Kafka table engine. As a result instead of the regular Kafka engine, a new type of storage engine will be used that stores the committed offsets primarily in ClickHouse Keeper", 0) \ M(Bool, enable_software_prefetch_in_aggregation, true, "Enable use of software prefetch in aggregation", 0) \ M(Bool, allow_aggregate_partitions_independently, false, "Enable independent aggregation of partitions on separate threads when partition key suits group by key. Beneficial when number of partitions close to number of cores and partitions have roughly the same size", 0) \ M(Bool, force_aggregate_partitions_independently, false, "Force the use of optimization when it is applicable, but heuristics decided not to use it", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 9352b22132f..6dcf8ddb95d 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -95,6 +95,7 @@ static std::map sett {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, + {"allow_experimental_kafka_store_offsets_in_keeper", false, false, "Allow the usage of experimental Kafka storage engine that stores the committed offsets in ClickHouse Keeper"}, }}, {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index c9ee42b54aa..9ca5e189f0e 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -38,8 +38,8 @@ const auto KAFKA_CONSUMERS_POOL_TTL_MS_MAX = 600'000; M(StreamingHandleErrorMode, kafka_handle_error_mode, StreamingHandleErrorMode::DEFAULT, "How to handle errors for Kafka engine. Possible values: default (throw an exception after rabbitmq_skip_broken_messages broken messages), stream (save broken messages and errors in virtual columns _raw_message, _error).", 0) \ M(Bool, kafka_commit_on_select, false, "Commit messages when select query is made", 0) \ M(UInt64, kafka_max_rows_per_message, 1, "The maximum number of rows produced in one kafka message for row-based formats.", 0) \ - M(String, kafka_keeper_path, "", "TODO(antaljanosbenjamin)", 0) \ - M(String, kafka_replica_name, "", "TODO(antaljanosbenjamin)", 0) \ + M(String, kafka_keeper_path, "", "The path to the table in ClickHouse Keeper", 0) \ + M(String, kafka_replica_name, "", "The replica name in ClickHouse Keeper", 0) \ #define OBSOLETE_KAFKA_SETTINGS(M, ALIAS) \ MAKE_OBSOLETE(M, Char, kafka_row_delimiter, '\0') \ From 29546d1655b53676868361534f4002d6c339253e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 18 Jun 2024 20:17:32 +0000 Subject: [PATCH 107/644] Add minimal docs --- .../table-engines/integrations/kafka.md | 37 +++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index 7bdc856c9fd..f899fea97de 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -251,6 +251,43 @@ The number of rows in one Kafka message depends on whether the format is row-bas - For row-based formats the number of rows in one Kafka message can be controlled by setting `kafka_max_rows_per_message`. - For block-based formats we cannot divide block into smaller parts, but the number of rows in one block can be controlled by general setting [max_block_size](../../../operations/settings/settings.md#setting-max_block_size). +## Experimental engine to store committed offsets in ClickHouse Keeper + +If `allow_experimental_kafka_store_offsets_in_keeper` is enabled, then two more settings can be specified to the Kafka table engine: + - `kafka_keeper_path` specifies the path to the table in ClickHouse Keeper + - `kafka_replica_name` specifies the replica name in ClickHouse Keeper + +Either both of the settings must be specified or neither of them. When both of them is specified, then a new, experimental Kafka engine will be used. The new engine doesn't depend on storing the committed offsets in Kafka,but stores them in ClickHouse Keeper. It still tries to commit the offsets to Kafka, but it only depends on those offsets when the table is created. In any other circumstances (table is restarted, or recovered after some error) the offsets stored in ClickHouse Keeper will be used to consume messages from. Apart from the committed offset, it also stores how many messages were consumed in the last batch, so if the insert fails, the same amount of messages will be consumed, thus enabling deduplication if necessary. + +Example: + +``` sql +CREATE TABLE experimental_kafka (key UInt64, value UInt64) +ENGINE = Kafka('localhost:19092', 'my-topic', 'my-consumer', 'JSONEachRow') +SETTINGS + kafka_keeper_path = '/clickhouse/{database}/experimental_kafka', + kafka_replica_name = 'r1' +SETTINGS allow_experimental_kafka_store_offsets_in_keeper=1; +``` + +Or to utilize the `uuid` and `replica` macros similarly to ReplicatedMergeTree: + +``` sql +CREATE TABLE experimental_kafka (key UInt64, value UInt64) +ENGINE = Kafka('localhost:19092', 'my-topic', 'my-consumer', 'JSONEachRow') +SETTINGS + kafka_keeper_path = '/clickhouse/{database}/{uuid}', + kafka_replica_name = '{replica}' +SETTINGS allow_experimental_kafka_store_offsets_in_keeper=1; +``` + +### Known limitations + +As the new engine is experimental, it is not production ready yet. There are few known limitations of the implementation: + - The biggest limitation is the engine doesn't support direct reading from Kafka topic (insertion works, but reading doesn't), thus the direct `SELECT` queries will fail. + - Rapidly dropping and recreating the table or specifying the same ClickHouse Keeper path to different engines might cause issues. As best practice you can use the `{uuid}` to avoid clashing paths. + - To make repeatable reads possible messages cannot be consumed from multiple partitions on a single thread. On the other hand the Kafka consumers has to be polled regularly to keep them alive. As a result of these two we decided to only allow creating multiple consumer if `kafka_thread_per_consumer` is enabled, otherwise it is too complicated to avoid issues regarding polling consumers regularly. + **See Also** - [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns) From 11456d5815e1ea1398924ecb6648cb504be9a5dd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 19 Jun 2024 09:43:14 +0000 Subject: [PATCH 108/644] Fix build with new fmt --- src/Storages/Kafka/KafkaConsumer.cpp | 1 + src/Storages/Kafka/KafkaConsumer.h | 5 ----- src/Storages/Kafka/KafkaConsumer2.cpp | 24 ++++++++++++------------ src/Storages/Kafka/StorageKafka2.cpp | 8 ++++---- src/Storages/Kafka/StorageKafka2.h | 4 ++-- src/Storages/Kafka/StorageKafkaCommon.h | 12 ++++++++++++ 6 files changed, 31 insertions(+), 23 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index 9ba42b9875e..f4385163323 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index 4daf8652c3b..a3bc97779b3 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -1,14 +1,12 @@ #pragma once #include -#include #include #include #include #include -#include #include namespace CurrentMetrics @@ -199,6 +197,3 @@ private: }; } - -template <> struct fmt::formatter : fmt::ostream_formatter {}; -template <> struct fmt::formatter : fmt::ostream_formatter {}; diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 84b6f5153ed..d59a06bc672 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -1,22 +1,22 @@ -// Needs to go first because its partial specialization of fmt::formatter -// should be defined before any instantiation -#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 CurrentMetrics { diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 080201cae9e..ce3630e39af 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -954,10 +954,10 @@ StorageKafka2::lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const Topi for (; tp_it != topic_partitions.end(); ++tp_it, ++path_it) { using zkutil::EphemeralNodeHolder; - LockedTopicPartitionInfo lock_info{.lock = EphemeralNodeHolder::existing(*path_it / lock_file_name, keeper_to_use)}; - - lock_info.committed_offset = getNumber(keeper_to_use, *path_it / commit_file_name); - lock_info.intent_size = getNumber(keeper_to_use, *path_it / intent_file_name); + LockedTopicPartitionInfo lock_info{ + EphemeralNodeHolder::existing(*path_it / lock_file_name, keeper_to_use), + getNumber(keeper_to_use, *path_it / commit_file_name), + getNumber(keeper_to_use, *path_it / intent_file_name)}; LOG_TRACE( log, diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index 99c97caf9da..0d6734ac0eb 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -99,9 +99,9 @@ private: { KafkaConsumer2Ptr consumer; /// available consumers size_t consume_from_topic_partition_index{0}; - TopicPartitions topic_partitions; + TopicPartitions topic_partitions{}; zkutil::ZooKeeperPtr keeper; - TopicPartitionLocks locks; + TopicPartitionLocks locks{}; Stopwatch watch{CLOCK_MONOTONIC_COARSE}; }; diff --git a/src/Storages/Kafka/StorageKafkaCommon.h b/src/Storages/Kafka/StorageKafkaCommon.h index 59a7983136d..bed09e9a9cd 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.h +++ b/src/Storages/Kafka/StorageKafkaCommon.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include namespace Poco @@ -59,3 +61,13 @@ struct KafkaConfigLoader const Names & topics); }; } + + +template <> +struct fmt::formatter : fmt::ostream_formatter +{ +}; +template <> +struct fmt::formatter : fmt::ostream_formatter +{ +}; From aee61f7ea2ed2d5a71b6bbe604b4865c4f0cd5b2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 19 Jun 2024 11:27:25 +0000 Subject: [PATCH 109/644] Fix asserts --- src/Storages/Kafka/StorageKafka2.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index ce3630e39af..754c2dfa926 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -302,7 +302,7 @@ bool StorageKafka2::activate() if (!activate_in_keeper()) { - assert(storage.is_readonly); + assert(!is_active); return false; } @@ -1203,7 +1203,7 @@ StorageKafka2::PolledBatchInfo StorageKafka2::pollConsumer( void StorageKafka2::threadFunc(size_t idx) { - assert(idx < tasks.size()); + chassert(idx < tasks.size()); auto task = tasks[idx]; std::optional maybe_stall_reason; try From a93385836f1c2ab24927a3db1e04de6126a1fd53 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Sun, 23 Jun 2024 19:32:47 +0000 Subject: [PATCH 110/644] Expect errors when polling for events --- src/Storages/Kafka/KafkaConsumer2.cpp | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index d59a06bc672..d471c263653 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -194,13 +194,17 @@ void KafkaConsumer2::drainConsumerQueue() void KafkaConsumer2::pollEvents() { - auto msg = consumer->poll(EVENT_POLL_TIMEOUT); - // All the partition queues are detached, so the consumer shouldn't be able to poll any messages - chassert(!msg && "Consumer returned a message when it was not expected"); - - // static constexpr int64_t max_tries = 5; - // for(auto i = 0; i < max_tries; ++i) - // consumer->poll(EVENT_POLL_TIMEOUT); + static constexpr auto max_tries = 5; + for (auto i = 0; i < max_tries; ++i) + { + auto msg = consumer->poll(EVENT_POLL_TIMEOUT); + if (!msg) + return; + // All the partition queues are detached, so the consumer shouldn't be able to poll any real messages + const auto err = msg.get_error(); + chassert(RD_KAFKA_RESP_ERR_NO_ERROR != err.get_error() && "Consumer returned a message when it was not expected"); + LOG_ERROR(log, "Consumer received error while polling events, code {}, error '{}'", err.get_error(), err.to_string()); + } }; KafkaConsumer2::TopicPartitionCounts KafkaConsumer2::getPartitionCounts() const From 8cc25827edf45a9928b5ba403a334298d53191c3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 24 Jun 2024 08:30:37 +0000 Subject: [PATCH 111/644] Extend known limitations --- docs/en/engines/table-engines/integrations/kafka.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index f899fea97de..8c9cd18d117 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -287,6 +287,7 @@ As the new engine is experimental, it is not production ready yet. There are few - The biggest limitation is the engine doesn't support direct reading from Kafka topic (insertion works, but reading doesn't), thus the direct `SELECT` queries will fail. - Rapidly dropping and recreating the table or specifying the same ClickHouse Keeper path to different engines might cause issues. As best practice you can use the `{uuid}` to avoid clashing paths. - To make repeatable reads possible messages cannot be consumed from multiple partitions on a single thread. On the other hand the Kafka consumers has to be polled regularly to keep them alive. As a result of these two we decided to only allow creating multiple consumer if `kafka_thread_per_consumer` is enabled, otherwise it is too complicated to avoid issues regarding polling consumers regularly. + - Consumers created by the new storage engine do not show up in [`system.kafka_consumers`](../../../operations/system-tables/kafka_consumers.md) table. **See Also** From aadf1536a40bd53c6a1b6359cf652854f134599b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 25 Jun 2024 22:28:01 +0000 Subject: [PATCH 112/644] fix protocol --- src/IO/WriteBufferFromPocoSocketChunked.cpp | 30 ++++++++------------- 1 file changed, 11 insertions(+), 19 deletions(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.cpp b/src/IO/WriteBufferFromPocoSocketChunked.cpp index 324f8ae3a02..a83b976ae09 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.cpp +++ b/src/IO/WriteBufferFromPocoSocketChunked.cpp @@ -24,6 +24,8 @@ void WriteBufferFromPocoSocketChunked::enableChunked() /// Initialize next chunk chunk_size_ptr = reinterpret_cast(pos); pos += std::min(available(), sizeof(*chunk_size_ptr)); + /// Pretend finishChunk() was just called to prevent sending empty chunk if finishChunk() called immediately + last_finish_chunk = chunk_size_ptr; } void WriteBufferFromPocoSocketChunked::finishChunk() @@ -33,7 +35,8 @@ void WriteBufferFromPocoSocketChunked::finishChunk() if (pos <= reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr)) { - if (chunk_size_ptr == last_finish_chunk) // prevent duplicate finish chunk + /// Prevent duplicate finish chunk (and finish chunk right after enableChunked()) + if (chunk_size_ptr == last_finish_chunk) return; /// If current chunk is empty it means we are finishing a chunk previously sent by next(), @@ -85,7 +88,7 @@ void WriteBufferFromPocoSocketChunked::finishChunk() } /// Buffer end-of-chunk - *reinterpret_cast(pos) = 0; + setValue(reinterpret_cast(pos), 0); pos += sizeof(*chunk_size_ptr); /// Initialize next chunk chunk_size_ptr = reinterpret_cast(pos); @@ -114,7 +117,7 @@ void WriteBufferFromPocoSocketChunked::nextImpl() return; } - /// next() after finishChunk ar the end of the buffer + /// next() after finishChunk at the end of the buffer if (finishing < sizeof(*chunk_size_ptr)) { pos -= finishing; @@ -135,21 +138,6 @@ void WriteBufferFromPocoSocketChunked::nextImpl() return; } - /// Send end-of-chunk buffered by finishChunk - if (offset() == 2 * sizeof(*chunk_size_ptr) && last_finish_chunk == chunk_size_ptr) - { - pos -= sizeof(*chunk_size_ptr); - /// Send end-of-chunk - WriteBufferFromPocoSocket::nextImpl(); - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(working_buffer.begin()); - nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); - - last_finish_chunk = chunk_size_ptr; - - return; - } - /// Prevent sending empty chunk if (offset() == sizeof(*chunk_size_ptr)) { @@ -172,8 +160,12 @@ void WriteBufferFromPocoSocketChunked::nextImpl() return; } + bool initialize_last_finish_chunk = false; if (pos - reinterpret_cast(chunk_size_ptr) == sizeof(*chunk_size_ptr)) // next() after finishChunk + { pos -= sizeof(*chunk_size_ptr); + initialize_last_finish_chunk = true; + } else // fill up current chunk size { setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); @@ -194,7 +186,7 @@ void WriteBufferFromPocoSocketChunked::nextImpl() chunk_size_ptr = reinterpret_cast(working_buffer.begin()); nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); - last_finish_chunk = nullptr; + last_finish_chunk = initialize_last_finish_chunk ? chunk_size_ptr : nullptr; } void WriteBufferFromPocoSocketChunked::finalizeImpl() From 9eec8344279082a3d02583c092f3c90b85a76fa3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 26 Jun 2024 03:19:16 +0000 Subject: [PATCH 113/644] better chunked protocol negotiation, comments, review suggestions --- src/Client/Connection.cpp | 37 +++++++++++++++------- src/IO/ReadBufferFromPocoSocketChunked.cpp | 11 ++++--- src/IO/ReadBufferFromPocoSocketChunked.h | 4 +-- src/IO/WriteBufferFromPocoSocketChunked.h | 10 ++++-- src/Server/TCPHandler.cpp | 17 +++++++--- src/Server/TCPHandler.h | 4 +-- 6 files changed, 56 insertions(+), 27 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 9f727b974ee..c41229c7226 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -208,11 +208,20 @@ void Connection::connect(const ConnectionTimeouts & timeouts) sendHello(); receiveHello(timeouts.handshake_timeout); - bool out_chunked = false; - bool in_chunked = false; - if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) { + /// Client side of chunked protocol negotiation. + /// Server advertises its protocol capabilities (separate for send and recieve channels) by sending + /// in its 'Hello' response one of four types - chunked, notchunked, chunked_optional, notchunked_optional. + /// Not optional types are strict meaning that server only supports this type, optional means that + /// server prefer this type but capable to work in opposite. + /// Client selects which type it is going to communicate based on the settings from config or arguments, + /// and sends either "chunked" or "notchunked" protocol request in addendum section of handshake. + /// Client can detect if server's protocol capabilities are not compatible with client's settings (for example + /// server strictly requires chunked protocol but client's settings only allowes notchunked protocol) - in such case + /// client should interrup this connection. However if client continues with incompatible protocol type request, server + /// will send appropriate exception and disconnect client. + auto is_chunked = [](const String & chunked_srv_str, const String & chunked_cl_str, const String & direction) { bool chunked_srv = chunked_srv_str.starts_with("chunked"); @@ -235,20 +244,24 @@ void Connection::connect(const ConnectionTimeouts & timeouts) return chunked_srv; }; - out_chunked = is_chunked(proto_recv_chunked_srv, proto_send_chunked, "send"); - in_chunked = is_chunked(proto_send_chunked_srv, proto_recv_chunked, "recv"); + proto_send_chunked = is_chunked(proto_recv_chunked_srv, proto_send_chunked, "send") ? "chunked" : "notchunked"; + proto_recv_chunked = is_chunked(proto_send_chunked_srv, proto_recv_chunked, "recv") ? "chunked" : "notchunked"; + } + else + { + if (proto_send_chunked == "chunked" || proto_recv_chunked == "chunked") + throw NetException( + ErrorCodes::NETWORK_ERROR, + "Incompatible protocol: server's version is too old and doesn't support chunked protocol while client settings require it."); } if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) sendAddendum(); - if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) - { - if (out_chunked) - out->enableChunked(); - if (in_chunked) - in->enableChunked(); - } + if (proto_send_chunked == "chunked") + out->enableChunked(); + if (proto_recv_chunked == "chunked") + in->enableChunked(); LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.", server_name, server_version_major, server_version_minor, server_version_patch); diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 798be547e99..07598f2adf4 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -28,6 +28,7 @@ void ReadBufferFromPocoSocketChunked::enableChunked() return; chunked = 1; data_end = buffer().end(); + /// Resize working buffer so any next read will call nextImpl working_buffer.resize(offset()); chunk_left = 0; next_chunk = 0; @@ -51,7 +52,7 @@ bool ReadBufferFromPocoSocketChunked::poll(size_t timeout_microseconds) const } -bool ReadBufferFromPocoSocketChunked::load_next_chunk(Position c_pos, bool cont) +bool ReadBufferFromPocoSocketChunked::loadNextChunk(Position c_pos, bool cont) { auto buffered = std::min(static_cast(data_end - c_pos), sizeof(next_chunk)); @@ -73,7 +74,7 @@ bool ReadBufferFromPocoSocketChunked::load_next_chunk(Position c_pos, bool cont) return true; } -bool ReadBufferFromPocoSocketChunked::process_chunk_left(Position c_pos) +bool ReadBufferFromPocoSocketChunked::processChunkLeft(Position c_pos) { if (data_end - c_pos < chunk_left) { @@ -88,7 +89,7 @@ bool ReadBufferFromPocoSocketChunked::process_chunk_left(Position c_pos) c_pos += chunk_left; - if (!load_next_chunk(c_pos, true)) + if (!loadNextChunk(c_pos, true)) return false; chunk_left = 0; @@ -115,7 +116,7 @@ bool ReadBufferFromPocoSocketChunked::nextImpl() if (c_pos > data_end) c_pos = data_end; - if (!load_next_chunk(c_pos)) + if (!loadNextChunk(c_pos)) return false; chunk_left = next_chunk; @@ -159,7 +160,7 @@ bool ReadBufferFromPocoSocketChunked::nextImpl() c_pos = buffer().begin(); } - return process_chunk_left(c_pos); + return processChunkLeft(c_pos); } } diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index acf0edafe0a..943a50f5d08 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -92,8 +92,8 @@ public: Poco::Net::SocketAddress ourAddress() { return our_address; } protected: - bool load_next_chunk(Position c_pos, bool cont = false); - bool process_chunk_left(Position c_pos); + bool loadNextChunk(Position c_pos, bool cont = false); + bool processChunkLeft(Position c_pos); bool nextImpl() override; Poco::Net::SocketAddress our_address; diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 269c6d66dda..8270ca445c9 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -10,8 +10,14 @@ namespace DB class WriteBufferFromPocoSocketChunked: public WriteBufferFromPocoSocket { public: - explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, buf_size), log(getLogger("Protocol")) {} - explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, write_event_, buf_size), log(getLogger("Protocol")) {} + explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, buf_size), log(getLogger("Protocol")) + { + chassert(buf_size <= std::numeric_limits>::max() && buf_size > sizeof(*chunk_size_ptr)); + } + explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, write_event_, buf_size), log(getLogger("Protocol")) + { + chassert(buf_size <= std::numeric_limits>::max() && buf_size > sizeof(*chunk_size_ptr)); + } void enableChunked(); void finishChunk(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 960860a3c13..3093c508c22 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -277,19 +277,28 @@ void TCPHandler::runImpl() if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) receiveAddendum(); - if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) { + /// Server side of chunked protocol negotiation. + /// Server advertises its protocol capabilities (separate for send and recieve channels) by sending + /// in its 'Hello' response one of four types - chunked, notchunked, chunked_optional, notchunked_optional. + /// Not optional types are strict meaning that server only supports this type, optional means that + /// server prefer this type but capable to work in opposite. + /// Client selects which type it is going to communicate based on the settings from config or arguments, + /// and sends either "chunked" or "notchunked" protocol request in addendum section of handshake. + /// Client can detect if server's protocol capabilities are not compatible with client's settings (for example + /// server strictly requires chunked protocol but client's settings only allowes notchunked protocol) - in such case + /// client should interrup this connection. However if client continues with incompatible protocol type request, server + /// will send appropriate exception and disconnect client. + auto is_chunked = [](const String & chunked_srv_str, const String & chunked_cl_str, const String & direction) { bool chunked_srv = chunked_srv_str.starts_with("chunked"); bool optional_srv = chunked_srv_str.ends_with("_optional"); bool chunked_cl = chunked_cl_str.starts_with("chunked"); - bool optional_cl = chunked_cl_str.ends_with("_optional"); if (optional_srv) return chunked_cl; - if (optional_cl) - return chunked_srv; + if (chunked_cl != chunked_srv) throw NetException( ErrorCodes::NETWORK_ERROR, diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 88c8fc6d52c..f6400161041 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -187,8 +187,8 @@ private: UInt64 client_version_minor = 0; UInt64 client_version_patch = 0; UInt32 client_tcp_protocol_version = 0; - String proto_send_chunked_cl; - String proto_recv_chunked_cl; + String proto_send_chunked_cl = "notchunked"; + String proto_recv_chunked_cl = "notchunked"; String quota_key; /// Connection settings, which are extracted from a context. From 6112ef710c2d949c3c8824fcf0e7c148f5deaea4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 26 Jun 2024 03:43:28 +0000 Subject: [PATCH 114/644] fix style --- src/Client/Connection.cpp | 8 ++++---- src/Server/TCPHandler.cpp | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index c41229c7226..14ffff10081 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -211,17 +211,17 @@ void Connection::connect(const ConnectionTimeouts & timeouts) if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) { /// Client side of chunked protocol negotiation. - /// Server advertises its protocol capabilities (separate for send and recieve channels) by sending + /// Server advertises its protocol capabilities (separate for send and receive channels) by sending /// in its 'Hello' response one of four types - chunked, notchunked, chunked_optional, notchunked_optional. /// Not optional types are strict meaning that server only supports this type, optional means that /// server prefer this type but capable to work in opposite. /// Client selects which type it is going to communicate based on the settings from config or arguments, /// and sends either "chunked" or "notchunked" protocol request in addendum section of handshake. /// Client can detect if server's protocol capabilities are not compatible with client's settings (for example - /// server strictly requires chunked protocol but client's settings only allowes notchunked protocol) - in such case - /// client should interrup this connection. However if client continues with incompatible protocol type request, server + /// server strictly requires chunked protocol but client's settings only allows notchunked protocol) - in such case + /// client should interrupt this connection. However if client continues with incompatible protocol type request, server /// will send appropriate exception and disconnect client. - + auto is_chunked = [](const String & chunked_srv_str, const String & chunked_cl_str, const String & direction) { bool chunked_srv = chunked_srv_str.starts_with("chunked"); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 3093c508c22..d5afb624e77 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -279,15 +279,15 @@ void TCPHandler::runImpl() { /// Server side of chunked protocol negotiation. - /// Server advertises its protocol capabilities (separate for send and recieve channels) by sending + /// Server advertises its protocol capabilities (separate for send and receive channels) by sending /// in its 'Hello' response one of four types - chunked, notchunked, chunked_optional, notchunked_optional. /// Not optional types are strict meaning that server only supports this type, optional means that /// server prefer this type but capable to work in opposite. /// Client selects which type it is going to communicate based on the settings from config or arguments, /// and sends either "chunked" or "notchunked" protocol request in addendum section of handshake. /// Client can detect if server's protocol capabilities are not compatible with client's settings (for example - /// server strictly requires chunked protocol but client's settings only allowes notchunked protocol) - in such case - /// client should interrup this connection. However if client continues with incompatible protocol type request, server + /// server strictly requires chunked protocol but client's settings only allows notchunked protocol) - in such case + /// client should interrupt this connection. However if client continues with incompatible protocol type request, server /// will send appropriate exception and disconnect client. auto is_chunked = [](const String & chunked_srv_str, const String & chunked_cl_str, const String & direction) From 3f3305a63a1218dc944ac7b3a8540f084a57a039 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 26 Jun 2024 04:33:52 +0000 Subject: [PATCH 115/644] fix server settings --- src/Server/TCPHandler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d5afb624e77..40fd3848455 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -310,8 +310,8 @@ void TCPHandler::runImpl() return chunked_srv; }; - bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "chunked"), proto_recv_chunked_cl, "send"); - bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "chunked"), proto_send_chunked_cl, "recv"); + bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "chunked_optional"), proto_recv_chunked_cl, "send"); + bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "chunked_optional"), proto_send_chunked_cl, "recv"); if (out_chunked) out->enableChunked(); From 32e6bed4ee8aecf97ddd289ca869f8da096d58af Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 26 Jun 2024 14:04:33 +0000 Subject: [PATCH 116/644] bug fix, ubsan paranoia fix --- src/IO/WriteBufferFromPocoSocketChunked.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.cpp b/src/IO/WriteBufferFromPocoSocketChunked.cpp index a83b976ae09..b6d9efda815 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.cpp +++ b/src/IO/WriteBufferFromPocoSocketChunked.cpp @@ -7,10 +7,9 @@ namespace { template -const T & setValue(T * typed_ptr, std::type_identity_t val) +void setValue(T * typed_ptr, std::type_identity_t val) { - memcpy(typed_ptr, &val, sizeof(T)); - return *typed_ptr; + memcpy(static_cast(typed_ptr), &val, sizeof(T)); } } @@ -84,6 +83,7 @@ void WriteBufferFromPocoSocketChunked::finishChunk() finishing = available(); pos += available(); chunk_size_ptr = reinterpret_cast(pos); + last_finish_chunk = chunk_size_ptr; return; } From 30a9c38c9596b40555c8ec041257b53cd10b9abc Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 26 Jun 2024 20:43:13 +0000 Subject: [PATCH 117/644] fix buffer size check --- src/IO/ReadBufferFromPocoSocketChunked.cpp | 10 +++++----- src/IO/WriteBufferFromPocoSocketChunked.cpp | 11 +++++++++++ src/IO/WriteBufferFromPocoSocketChunked.h | 11 +++-------- 3 files changed, 19 insertions(+), 13 deletions(-) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 07598f2adf4..93afeadba60 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -16,11 +16,11 @@ ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Sock {} ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size) - : ReadBufferFromPocoSocketBase(socket_, read_event_, buf_size), our_address(socket_.address()), log(getLogger("Protocol")) - -{ - chassert(buf_size <= std::numeric_limits::max()); -} + : ReadBufferFromPocoSocketBase( + socket_, read_event_, + std::min(buf_size, static_cast(std::numeric_limits::max()))), + our_address(socket_.address()), log(getLogger("Protocol")) +{} void ReadBufferFromPocoSocketChunked::enableChunked() { diff --git a/src/IO/WriteBufferFromPocoSocketChunked.cpp b/src/IO/WriteBufferFromPocoSocketChunked.cpp index b6d9efda815..98c5126c24b 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.cpp +++ b/src/IO/WriteBufferFromPocoSocketChunked.cpp @@ -17,6 +17,17 @@ void setValue(T * typed_ptr, std::type_identity_t val) namespace DB { +WriteBufferFromPocoSocketChunked::WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size) + : WriteBufferFromPocoSocketChunked(socket_, ProfileEvents::end(), buf_size) +{} + +WriteBufferFromPocoSocketChunked::WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size) + : WriteBufferFromPocoSocket( + socket_, write_event_, + std::clamp(buf_size, sizeof(*chunk_size_ptr) + 1, static_cast(std::numeric_limits>::max()))), + log(getLogger("Protocol")) +{} + void WriteBufferFromPocoSocketChunked::enableChunked() { chunked = true; diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 8270ca445c9..13a277e3bfb 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -10,14 +11,8 @@ namespace DB class WriteBufferFromPocoSocketChunked: public WriteBufferFromPocoSocket { public: - explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, buf_size), log(getLogger("Protocol")) - { - chassert(buf_size <= std::numeric_limits>::max() && buf_size > sizeof(*chunk_size_ptr)); - } - explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, write_event_, buf_size), log(getLogger("Protocol")) - { - chassert(buf_size <= std::numeric_limits>::max() && buf_size > sizeof(*chunk_size_ptr)); - } + explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); void enableChunked(); void finishChunk(); From 71d71bd5fe8884b5f2b11e06302dce8e511f5b7c Mon Sep 17 00:00:00 2001 From: morning-color Date: Thu, 27 Jun 2024 20:06:14 +0800 Subject: [PATCH 118/644] Add rows_before_group_by_counter --- src/Client/ClientBase.cpp | 2 + src/Core/Settings.h | 1 + src/Processors/Formats/IOutputFormat.cpp | 3 +- src/Processors/Formats/IOutputFormat.h | 8 ++++ src/Processors/Formats/LazyOutputFormat.cpp | 4 ++ src/Processors/Formats/LazyOutputFormat.h | 1 + .../Formats/PullingOutputFormat.cpp | 5 ++- src/Processors/Formats/PullingOutputFormat.h | 1 + src/Processors/IProcessor.h | 6 +++ src/Processors/Sources/DelayedSource.h | 2 + src/Processors/Sources/RemoteSource.cpp | 30 ++++++++++----- src/Processors/Sources/RemoteSource.h | 3 ++ .../Transforms/AggregatingTransform.cpp | 4 +- .../Transforms/AggregatingTransform.h | 11 ++++-- src/QueryPipeline/ProfileInfo.cpp | 17 +++++++++ src/QueryPipeline/ProfileInfo.h | 13 +++++++ src/QueryPipeline/QueryPipeline.cpp | 37 +++++++++++++------ src/Server/GRPCServer.cpp | 1 + src/Server/grpc_protos/clickhouse_grpc.proto | 2 + 19 files changed, 123 insertions(+), 28 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 15a4836ef7a..dbb67d230d5 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -522,6 +522,8 @@ void ClientBase::onProfileInfo(const ProfileInfo & profile_info) { if (profile_info.hasAppliedLimit() && output_format) output_format->setRowsBeforeLimit(profile_info.getRowsBeforeLimit()); + if (profile_info.hasAppliedGroupBy() && output_format) + output_format->setRowsBeforeGroupBy(profile_info.getRowsBeforeGroupBy()); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e3c122467bd..09291d4300d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1200,6 +1200,7 @@ class IColumn; M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \ \ M(Bool, exact_rows_before_limit, false, "When enabled, ClickHouse will provide exact value for rows_before_limit_at_least statistic, but with the cost that the data before limit will have to be read completely", 0) \ + M(Bool, exact_rows_before_group_by, false, "When enabled, ClickHouse will provide exact value for rows_before_group_by_at_least statistic, but with the cost that the data before group by will have to be read completely", 0) \ M(UInt64, cross_to_inner_join_rewrite, 1, "Use inner join instead of comma/cross join if there are joining expressions in the WHERE section. Values: 0 - no rewrite, 1 - apply if possible for comma/cross, 2 - force rewrite all comma joins, cross - if possible", 0) \ \ M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \ diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index 88a6fb1e92f..4191bf9f0fe 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -71,7 +71,8 @@ void IOutputFormat::work() { if (rows_before_limit_counter && rows_before_limit_counter->hasAppliedLimit()) setRowsBeforeLimit(rows_before_limit_counter->get()); - + if (rows_before_group_by_counter && rows_before_group_by_counter->hasAppliedLimit()) + setRowsBeforeGroupBy(rows_before_group_by_counter->get()); finalize(); if (auto_flush) flush(); diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index cae2ab7691e..9bb7cccb612 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -41,6 +41,12 @@ public: /// Counter to calculate rows_before_limit_at_least in processors pipeline. void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_counter.swap(counter); } + /// Value for rows_before_group_by_at_least field. + virtual void setRowsBeforeGroupBy(size_t /*rows_before_limit*/) { } + + /// Counter to calculate rows_before_group_by_at_least in processors pipeline. + void setRowsBeforeGroupByCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_group_by_counter.swap(counter); } + /// Notify about progress. Method could be called from different threads. /// Passed value are delta, that must be summarized. virtual void onProgress(const Progress & /*progress*/) {} @@ -151,6 +157,7 @@ protected: Progress progress; bool applied_limit = false; size_t rows_before_limit = 0; + size_t rows_before_group_by = 0; Chunk totals; Chunk extremes; }; @@ -185,6 +192,7 @@ protected: bool need_write_suffix = true; RowsBeforeLimitCounterPtr rows_before_limit_counter; + RowsBeforeGroupByCounterPtr rows_before_group_by_counter; Statistics statistics; private: diff --git a/src/Processors/Formats/LazyOutputFormat.cpp b/src/Processors/Formats/LazyOutputFormat.cpp index 4f6b10dd068..63423628e57 100644 --- a/src/Processors/Formats/LazyOutputFormat.cpp +++ b/src/Processors/Formats/LazyOutputFormat.cpp @@ -45,4 +45,8 @@ void LazyOutputFormat::setRowsBeforeLimit(size_t rows_before_limit) info.setRowsBeforeLimit(rows_before_limit); } +void LazyOutputFormat::setRowsBeforeGroupBy(size_t rows_before_group_by) +{ + info.setRowsBeforeGroupBy(rows_before_group_by); +} } diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index 9cf609ed2d7..83abb2ff1a1 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -28,6 +28,7 @@ public: ProfileInfo & getProfileInfo() { return info; } void setRowsBeforeLimit(size_t rows_before_limit) override; + void setRowsBeforeGroupBy(size_t rows_before_group_by) override; void onCancel() override { diff --git a/src/Processors/Formats/PullingOutputFormat.cpp b/src/Processors/Formats/PullingOutputFormat.cpp index b2378e62d34..646755deb6b 100644 --- a/src/Processors/Formats/PullingOutputFormat.cpp +++ b/src/Processors/Formats/PullingOutputFormat.cpp @@ -42,5 +42,8 @@ void PullingOutputFormat::setRowsBeforeLimit(size_t rows_before_limit) { info.setRowsBeforeLimit(rows_before_limit); } - +void PullingOutputFormat::setRowsBeforeGroupBy(size_t rows_before_group_by) +{ + info.setRowsBeforeGroupBy(rows_before_group_by); +} } diff --git a/src/Processors/Formats/PullingOutputFormat.h b/src/Processors/Formats/PullingOutputFormat.h index a8efb8dd962..c4d8cf4aab2 100644 --- a/src/Processors/Formats/PullingOutputFormat.h +++ b/src/Processors/Formats/PullingOutputFormat.h @@ -22,6 +22,7 @@ public: ProfileInfo & getProfileInfo() { return info; } void setRowsBeforeLimit(size_t rows_before_limit) override; + void setRowsBeforeGroupBy(size_t rows_before_group_by) override; bool expectMaterializedColumns() const override { return false; } diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 63f32d8deb7..0df4b3168e3 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -24,6 +24,8 @@ using StorageLimitsList = std::list; class RowsBeforeLimitCounter; using RowsBeforeLimitCounterPtr = std::shared_ptr; +using RowsBeforeGroupByCounterPtr = std::shared_ptr; + class IProcessor; using ProcessorPtr = std::shared_ptr; using Processors = std::vector; @@ -366,6 +368,10 @@ public: /// This counter is used to calculate the number of rows right before any filtration of LimitTransform. virtual void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr /* counter */) {} + /// Set rows_before_group_by counter for current processor. + /// This counter is used to calculate the number of rows right before AggregatingTransform. + virtual void setRowsBeforeGroupByCounter(RowsBeforeGroupByCounterPtr /* counter */) { } + protected: virtual void onCancel() {} diff --git a/src/Processors/Sources/DelayedSource.h b/src/Processors/Sources/DelayedSource.h index 0b2751e18a6..bd100f29a47 100644 --- a/src/Processors/Sources/DelayedSource.h +++ b/src/Processors/Sources/DelayedSource.h @@ -31,12 +31,14 @@ public: OutputPort * getExtremesPort() { return extremes; } void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); } + void setRowsBeforeGroupByCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_group_by.swap(counter); } private: QueryPlanResourceHolder resources; Creator creator; Processors processors; RowsBeforeLimitCounterPtr rows_before_limit; + RowsBeforeLimitCounterPtr rows_before_group_by; /// Outputs for DelayedSource. OutputPort * main = nullptr; diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 3d7dd3f76b8..a78db630786 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -35,16 +35,25 @@ RemoteSource::RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation progress(value.read_rows, value.read_bytes); }); - query_executor->setProfileInfoCallback([this](const ProfileInfo & info) - { - if (rows_before_limit) + query_executor->setProfileInfoCallback( + [this](const ProfileInfo & info) { - if (info.hasAppliedLimit()) - rows_before_limit->add(info.getRowsBeforeLimit()); - else - manually_add_rows_before_limit_counter = true; /// Remote subquery doesn't contain a limit - } - }); + if (rows_before_limit) + { + if (info.hasAppliedLimit()) + rows_before_limit->add(info.getRowsBeforeLimit()); + else + manually_add_rows_before_limit_counter = true; /// Remote subquery doesn't contain a limit + } + + if (rows_before_group_by) + { + if (info.hasAppliedGroupBy()) + rows_before_group_by->add(info.getRowsBeforeGroupBy()); + else + manually_add_rows_before_group_by_counter = true; /// Remote subquery doesn't contain a group by + } + }); } RemoteSource::~RemoteSource() = default; @@ -162,7 +171,8 @@ std::optional RemoteSource::tryGenerate() { if (manually_add_rows_before_limit_counter) rows_before_limit->add(rows); - + if (manually_add_rows_before_group_by_counter) + rows_before_group_by->add(rows); query_executor->finish(); return {}; } diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index 052567bc261..b2ea6d50e01 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -26,6 +26,7 @@ public: String getName() const override { return "Remote"; } void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); } + void setRowsBeforeGroupByCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_group_by.swap(counter); } /// Stop reading from stream if output port is finished. void onUpdatePorts() override; @@ -45,6 +46,7 @@ private: bool add_aggregation_info = false; RemoteQueryExecutorPtr query_executor; RowsBeforeLimitCounterPtr rows_before_limit; + RowsBeforeLimitCounterPtr rows_before_group_by; const bool async_read; const bool async_query_sending; @@ -52,6 +54,7 @@ private: int fd = -1; size_t rows = 0; bool manually_add_rows_before_limit_counter = false; + bool manually_add_rows_before_group_by_counter = false; }; /// Totals source from RemoteQueryExecutor. diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 65f0612d738..7c0e222f89b 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -8,7 +8,6 @@ #include #include #include - #include @@ -684,7 +683,8 @@ void AggregatingTransform::consume(Chunk chunk) LOG_TRACE(log, "Aggregating"); is_consume_started = true; } - + if (rows_before_group_by_at_least) + rows_before_group_by_at_least->add(num_rows); src_rows += num_rows; src_bytes += chunk.bytes(); diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index e167acde067..6e7b04f9191 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -3,11 +3,13 @@ #include #include #include -#include -#include -#include +#include #include #include +#include +#include +#include + namespace CurrentMetrics { @@ -167,6 +169,7 @@ public: Status prepare() override; void work() override; Processors expandPipeline() override; + void setRowsBeforeGroupByCounter(RowsBeforeGroupByCounterPtr counter) override { rows_before_group_by_at_least.swap(counter); } protected: void consume(Chunk chunk); @@ -210,6 +213,8 @@ private: bool is_consume_started = false; + RowsBeforeGroupByCounterPtr rows_before_group_by_at_least; + void initGenerate(); }; diff --git a/src/QueryPipeline/ProfileInfo.cpp b/src/QueryPipeline/ProfileInfo.cpp index ee0ff8c69bf..cec179ecfad 100644 --- a/src/QueryPipeline/ProfileInfo.cpp +++ b/src/QueryPipeline/ProfileInfo.cpp @@ -16,6 +16,8 @@ void ProfileInfo::read(ReadBuffer & in) readBinary(applied_limit, in); readVarUInt(rows_before_limit, in); readBinary(calculated_rows_before_limit, in); + readBinary(applied_group_by, in); + readVarUInt(rows_before_group_by, in); } @@ -27,6 +29,8 @@ void ProfileInfo::write(WriteBuffer & out) const writeBinary(hasAppliedLimit(), out); writeVarUInt(getRowsBeforeLimit(), out); writeBinary(calculated_rows_before_limit, out); + writeBinary(hasAppliedGroupBy(), out); + writeVarUInt(getRowsBeforeGroupBy(), out); } @@ -41,6 +45,8 @@ void ProfileInfo::setFrom(const ProfileInfo & rhs, bool skip_block_size_info) applied_limit = rhs.applied_limit; rows_before_limit = rhs.rows_before_limit; calculated_rows_before_limit = rhs.calculated_rows_before_limit; + applied_group_by = rhs.applied_group_by; + rows_before_group_by = rhs.rows_before_group_by; } @@ -57,6 +63,17 @@ bool ProfileInfo::hasAppliedLimit() const return applied_limit; } +size_t ProfileInfo::getRowsBeforeGroupBy() const +{ + return rows_before_group_by; +} + + +bool ProfileInfo::hasAppliedGroupBy() const +{ + return applied_group_by; +} + void ProfileInfo::update(Block & block) { diff --git a/src/QueryPipeline/ProfileInfo.h b/src/QueryPipeline/ProfileInfo.h index 7a0a0c304e2..141adc7430d 100644 --- a/src/QueryPipeline/ProfileInfo.h +++ b/src/QueryPipeline/ProfileInfo.h @@ -32,6 +32,9 @@ struct ProfileInfo size_t getRowsBeforeLimit() const; bool hasAppliedLimit() const; + size_t getRowsBeforeGroupBy() const; + bool hasAppliedGroupBy() const; + void update(Block & block); void update(size_t num_rows, size_t num_bytes); @@ -51,11 +54,21 @@ struct ProfileInfo rows_before_limit = rows_before_limit_; } + /// Only for Processors. + void setRowsBeforeGroupBy(size_t rows_before_group_by_) + { + applied_group_by = true; + rows_before_group_by = rows_before_group_by_; + } + private: /// For these fields we make accessors, because they must be calculated beforehand. mutable bool applied_limit = false; /// Whether LIMIT was applied mutable size_t rows_before_limit = 0; mutable bool calculated_rows_before_limit = false; /// Whether the field rows_before_limit was calculated + + mutable bool applied_group_by = false; /// Whether GROUP BY was applied + mutable size_t rows_before_group_by = 0; }; } diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 935c006c217..5e0885ed4e8 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -1,15 +1,13 @@ #include #include -#include -#include -#include -#include #include #include -#include -#include -#include +#include +#include +#include +#include +#include #include #include #include @@ -17,15 +15,18 @@ #include #include #include -#include +#include #include +#include #include #include #include #include -#include #include -#include +#include +#include +#include +#include namespace DB @@ -273,7 +274,20 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) output_format->setRowsBeforeLimitCounter(rows_before_limit_at_least); } } - +static void initRowsBeforeGroupBy(std::shared_ptr processors, IOutputFormat * output_format) +{ + if (!processors->empty()) + { + RowsBeforeGroupByCounterPtr rows_before_group_by_at_least = std::make_shared(); + for (auto & processor : *processors) + { + if (auto transform = std::dynamic_pointer_cast(processor)) + transform->setRowsBeforeGroupByCounter(rows_before_group_by_at_least); + } + rows_before_group_by_at_least->add(0); + output_format->setRowsBeforeLimitCounter(rows_before_group_by_at_least); + } +} QueryPipeline::QueryPipeline( QueryPlanResourceHolder resources_, @@ -521,6 +535,7 @@ void QueryPipeline::complete(std::shared_ptr format) extremes = nullptr; initRowsBeforeLimit(format.get()); + initRowsBeforeGroupBy(processors, format.get()); output_format = format.get(); processors->emplace_back(std::move(format)); diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 10b59751b22..37e4342f3b0 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1577,6 +1577,7 @@ namespace stats.set_allocated_bytes(info.bytes); stats.set_applied_limit(info.hasAppliedLimit()); stats.set_rows_before_limit(info.getRowsBeforeLimit()); + stats.set_rows_before_group_by(info.getRowsBeforeGroupBy()); } void Call::addLogsToResult() diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index c9ba6f28506..02b6988b8c0 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -179,6 +179,8 @@ message Stats { uint64 allocated_bytes = 3; bool applied_limit = 4; uint64 rows_before_limit = 5; + bool applied_group_by = 6; + uint64 rows_before_group_by = 7; } message Exception { From 04b8b1e76c467ae527202a75141ac8981a1c4ac5 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 2 Jul 2024 14:01:19 +0000 Subject: [PATCH 119/644] initial commit for Hive-style partitioning --- src/Core/Settings.h | 5 + src/Core/SettingsChangesHistory.h | 5 + .../ObjectStorage/StorageObjectStorage.cpp | 32 ++- .../StorageObjectStorageSource.cpp | 14 +- src/Storages/StorageFile.cpp | 39 +++- src/Storages/StorageURL.cpp | 16 +- src/Storages/VirtualColumnUtils.cpp | 52 ++++- src/Storages/VirtualColumnUtils.h | 7 +- .../__init__.py | 0 .../configs/cluster_azure.xml | 39 ++++ .../configs/cluster_hdfs.xml | 33 +++ .../configs/disable_profilers_azure.xml | 9 + .../configs/macro_hdfs.xml | 5 + .../configs/named_collections_azure.xml | 14 ++ .../configs/schema_cache_azure.xml | 3 + .../configs/schema_cache_hdfs.xml | 3 + .../configs/users_azure.xml | 9 + .../test_azure.py | 204 ++++++++++++++++++ .../test_hdfs.py | 81 +++++++ .../03203_hive_style_partitioning.reference | 96 +++++++++ .../03203_hive_style_partitioning.sh | 93 ++++++++ .../column1=Gordon/sample.parquet | Bin 0 -> 1308 bytes .../column1=Schmidt/sample.parquet | Bin 0 -> 1308 bytes .../column0=Elizabeth/sample.parquet | Bin 0 -> 1308 bytes .../sample.parquet | Bin 0 -> 1308 bytes .../column1=Gordon/sample.parquet | Bin 0 -> 1308 bytes .../column1=Schmidt/sample.parquet | Bin 0 -> 1308 bytes .../coumn0=Elizabeth/sample.parquet | Bin 0 -> 1308 bytes .../sample.parquet | Bin 0 -> 1308 bytes 29 files changed, 749 insertions(+), 10 deletions(-) create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/__init__.py create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py create mode 100644 tests/queries/0_stateless/03203_hive_style_partitioning.reference create mode 100755 tests/queries/0_stateless/03203_hive_style_partitioning.sh create mode 100644 tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet create mode 100644 tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet create mode 100644 tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/sample.parquet create mode 100644 tests/queries/0_stateless/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet create mode 100644 tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Gordon/sample.parquet create mode 100644 tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Schmidt/sample.parquet create mode 100644 tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/sample.parquet create mode 100644 tests/queries/0_stateless/data_minio/hive_partitioning/non_existing_column=Elizabeth/sample.parquet diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 14fe0924b40..738c0129d2d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1106,6 +1106,11 @@ class IColumn; M(Bool, input_format_tsv_skip_trailing_empty_lines, false, "Skip trailing empty lines in TSV format", 0) \ M(Bool, input_format_custom_skip_trailing_empty_lines, false, "Skip trailing empty lines in CustomSeparated format", 0) \ M(Bool, input_format_tsv_crlf_end_of_line, false, "If it is set true, file function will read TSV format with \\r\\n instead of \\n.", 0) \ + M(Bool, file_hive_partitioning, false, "Allows to use hive partitioning for file format", 0)\ + M(Bool, url_hive_partitioning, false, "Allows to use hive partitioning for url format", 0)\ + M(Bool, s3_hive_partitioning, false, "Allows to use hive partitioning for s3 format", 0)\ + M(Bool, azure_blob_storage_hive_partitioning, false, "Allows to use hive partitioning for AzureBlobStorage format", 0)\ + M(Bool, hdfs_hive_partitioning, false, "Allows to use hive partitioning for hdfs format", 0)\ \ M(Bool, input_format_native_allow_types_conversion, true, "Allow data types conversion in Native input format", 0) \ \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 4ac25a649b7..dd778149674 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -118,6 +118,11 @@ static const std::map +#include #include #include @@ -32,6 +33,19 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } + +bool checkIfHiveSettingEnabled(const ContextPtr & context, const std::string & storage_type_name) +{ + if (storage_type_name == "s3") + return context->getSettings().s3_hive_partitioning; + else if (storage_type_name == "hdfs") + return context->getSettings().hdfs_hive_partitioning; + else if (storage_type_name == "azure") + return context->getSettings().azure_blob_storage_hive_partitioning; + else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_name); +} + StorageObjectStorage::StorageObjectStorage( ConfigurationPtr configuration_, ObjectStoragePtr object_storage_, @@ -60,7 +74,23 @@ StorageObjectStorage::StorageObjectStorage( metadata.setConstraints(constraints_); metadata.setComment(comment); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns())); + auto file_iterator = StorageObjectStorageSource::createFileIterator( + configuration, + object_storage, + distributed_processing_, + context, + {}, // predicate + metadata.getColumns().getAll(), // virtual_columns + nullptr, // read_keys + {} // file_progress_callback + ); + + Strings paths; + + if (checkIfHiveSettingEnabled(context, configuration->getTypeName())) + if (auto file = file_iterator->next(0)) + paths = {file->getPath()}; + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), paths)); setInMemoryMetadata(metadata); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index aef783fc3c4..2741cfecf6b 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -13,6 +13,7 @@ #include #include #include +#include namespace fs = std::filesystem; @@ -195,13 +196,24 @@ Chunk StorageObjectStorageSource::generate() const auto & object_info = reader.getObjectInfo(); const auto & filename = object_info->getFileName(); chassert(object_info->metadata); + + auto hive_map = VirtualColumnUtils::parsePartitionMapFromPath(object_info->getPath()); + bool contains_virtual_column = std::any_of(hive_map.begin(), hive_map.end(), + [&](const auto& pair) { + return read_from_format_info.requested_virtual_columns.contains(pair.first); + }); + + if (!contains_virtual_column) + hive_map.clear(); // If we cannot find any virual column in requested, we don't add any of them to chunk + VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( chunk, read_from_format_info.requested_virtual_columns, { .path = getUniqueStoragePathIdentifier(*configuration, *object_info, false), .size = object_info->metadata->size_bytes, .filename = &filename, - .last_modified = object_info->metadata->last_modified + .last_modified = object_info->metadata->last_modified, + .hive_partitioning_map = hive_map }); return chunk; } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7f39ff615f0..0c32f29cb34 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -52,6 +52,7 @@ #include #include #include +#include #include #include @@ -1095,7 +1096,11 @@ void StorageFile::setStorageMetadata(CommonArguments args) storage_metadata.setConstraints(args.constraints); storage_metadata.setComment(args.comment); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); + + Strings paths_for_virtuals; + if (args.getContext()->getSettingsRef().file_hive_partitioning) + paths_for_virtuals = paths; + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), paths_for_virtuals)); } @@ -1437,6 +1442,15 @@ Chunk StorageFileSource::generate() chunk_size = input_format->getApproxBytesReadForChunk(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); + std::map hive_map; + if (getContext()->getSettingsRef().file_hive_partitioning) + { + hive_map = VirtualColumnUtils::parsePartitionMapFromPath(current_path); + + for (const auto& item : hive_map) + requested_virtual_columns.push_back(NameAndTypePair(item.first, std::make_shared())); + } + /// Enrich with virtual columns. VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( chunk, requested_virtual_columns, @@ -1444,7 +1458,8 @@ Chunk StorageFileSource::generate() .path = current_path, .size = current_file_size, .filename = (filename_override.has_value() ? &filename_override.value() : nullptr), - .last_modified = current_file_last_modified + .last_modified = current_file_last_modified, + .hive_partitioning_map = hive_map }); return chunk; @@ -1621,6 +1636,16 @@ void ReadFromFile::createIterator(const ActionsDAG::Node * predicate) storage->distributed_processing); } +void addPartitionColumnsToInfoHeader(Strings paths, ReadFromFormatInfo & info) +{ + for (const auto& path : paths) + { + auto map = VirtualColumnUtils::parsePartitionMapFromPath(path); + for (const auto& item : map) + info.source_header.insertUnique(ColumnWithTypeAndName(std::make_shared(), item.first)); + } +} + void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { createIterator(nullptr); @@ -1628,10 +1653,20 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui size_t num_streams = max_num_streams; size_t files_to_read = 0; + Strings paths; if (storage->archive_info) + { files_to_read = storage->archive_info->paths_to_archives.size(); + paths = storage->archive_info->paths_to_archives; + } else + { files_to_read = storage->paths.size(); + paths = storage->paths; + } + + if (getContext()->getSettingsRef().file_hive_partitioning) + addPartitionColumnsToInfoHeader(paths, info); if (max_num_streams > files_to_read) num_streams = files_to_read; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 895da028fc2..f6374701fc2 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -36,6 +36,7 @@ #include #include #include +#include #include #include @@ -151,7 +152,11 @@ IStorageURLBase::IStorageURLBase( storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); + + Strings uri_for_partitioning; + if (context_->getSettingsRef().url_hive_partitioning) + uri_for_partitioning = {uri}; + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), uri_for_partitioning)); } @@ -410,12 +415,17 @@ Chunk StorageURLSource::generate() size_t chunk_size = 0; if (input_format) chunk_size = input_format->getApproxBytesReadForChunk(); + std::map hive_map; + if (getContext()->getSettingsRef().url_hive_partitioning) + hive_map = VirtualColumnUtils::parsePartitionMapFromPath(curr_uri.getPath()); + progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( chunk, requested_virtual_columns, { .path = curr_uri.getPath(), - .size = current_file_size + .size = current_file_size, + .hive_partitioning_map = hive_map }); return chunk; } @@ -1170,6 +1180,7 @@ void ReadFromURL::createIterator(const ActionsDAG::Node * predicate) void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { createIterator(nullptr); + const auto & settings = context->getSettingsRef(); if (is_empty_glob) { @@ -1180,7 +1191,6 @@ void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const Buil Pipes pipes; pipes.reserve(num_streams); - const auto & settings = context->getSettingsRef(); const size_t max_parsing_threads = num_streams >= settings.max_parsing_threads ? 1 : (settings.max_parsing_threads / num_streams); for (size_t i = 0; i < num_streams; ++i) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 778c9e13adb..0b79e3b7a16 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -37,6 +36,7 @@ #include #include +#include #include #include "Functions/FunctionsLogical.h" #include "Functions/IFunction.h" @@ -115,7 +115,22 @@ NameSet getVirtualNamesForFileLikeStorage() return {"_path", "_file", "_size", "_time"}; } -VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns) +Strings parseVirtualColumnNameFromPath(const std::string & path) +{ + std::string pattern = "/([^/]+)=([^/]+)"; + // Map to store the key-value pairs + std::map key_values; + + re2::StringPiece input_piece(path); + std::string key; + Strings result; + while (RE2::FindAndConsume(&input_piece, pattern, &key)) + result.push_back(key); + + return result; +} + +VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, Strings paths) { VirtualColumnsDescription desc; @@ -132,6 +147,13 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription add_virtual("_size", makeNullable(std::make_shared())); add_virtual("_time", makeNullable(std::make_shared())); + for (const auto& path : paths) + { + auto names = parseVirtualColumnNameFromPath(path); + for (const auto& name : names) + add_virtual("_" + name, std::make_shared(std::make_shared())); + } + return desc; } @@ -178,6 +200,8 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const { if (column.name == "_file" || column.name == "_path") block.insert({column.type->createColumn(), column.type, column.name}); + if (!getVirtualNamesForFileLikeStorage().contains(column.name)) + block.insert({column.type->createColumn(), column.type, column.name}); } block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); @@ -189,6 +213,21 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const return block.getByName("_idx").column; } +std::map parsePartitionMapFromPath(const std::string & path) +{ + std::string pattern = "/([^/]+)=([^/]+)"; // Regex to capture key=value pairs + // Map to store the key-value pairs + std::map key_values; + + re2::StringPiece input_piece(path); + std::string key; + std::string value; + while (RE2::FindAndConsume(&input_piece, pattern, &key, &value)) + key_values["_" + key] = value; + + return key_values; +} + void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, VirtualsForFileLikeStorage virtual_values) @@ -226,6 +265,15 @@ void addRequestedFileLikeStorageVirtualsToChunk( else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } + else + { + auto it = virtual_values.hive_partitioning_map.find(virtual_column.getNameInStorage()); + if (it != virtual_values.hive_partitioning_map.end()) + { + chunk.addColumn(virtual_column.getTypeInStorage()->createColumnConst(chunk.getNumRows(), it->second)->convertToFullColumnIfConst()); + virtual_values.hive_partitioning_map.erase(it); + } + } } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index fbfbdd6c6cc..a03d4c7447f 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -6,6 +6,8 @@ #include #include +#include +#include #include @@ -47,7 +49,7 @@ auto extractSingleValueFromBlock(const Block & block, const String & name) } NameSet getVirtualNamesForFileLikeStorage(); -VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns); +VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, Strings paths = {}); ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); @@ -74,9 +76,12 @@ struct VirtualsForFileLikeStorage std::optional size { std::nullopt }; const String * filename { nullptr }; std::optional last_modified { std::nullopt }; + std::map hive_partitioning_map; }; +std::map parsePartitionMapFromPath(const std::string & path); + void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, VirtualsForFileLikeStorage virtual_values); diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/__init__.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml new file mode 100644 index 00000000000..ffa4673c9ee --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml @@ -0,0 +1,39 @@ + + + + + + node_0 + 9000 + + + node_1 + 9000 + + + node_2 + 9000 + + + + + + + + node_0 + 9000 + + + + + node_1 + 19000 + + + + + + + simple_cluster + + \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml new file mode 100644 index 00000000000..b99b21ea40b --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml @@ -0,0 +1,33 @@ + + + + + + node1 + 9000 + + + + + node1 + 19000 + + + + + + + + 127.0.0.1 + 9000 + + + + + 127.0.0.2 + 9000 + + + + + diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml new file mode 100644 index 00000000000..a39badbf8ec --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml @@ -0,0 +1,9 @@ + + + + + 0 + 0 + + + diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml new file mode 100644 index 00000000000..c2e11b47a5e --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml @@ -0,0 +1,5 @@ + + + test_cluster_two_shards + + \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml new file mode 100644 index 00000000000..bd7f9ff97f1 --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml @@ -0,0 +1,14 @@ + + + + cont + test_simple_write_named.csv + key UInt64, data String + CSV + + + devstoreaccount1 + Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + + + diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml new file mode 100644 index 00000000000..e2168ecd06d --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml @@ -0,0 +1,3 @@ + + 2 + \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml new file mode 100644 index 00000000000..37639649b5f --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml @@ -0,0 +1,3 @@ + + 2 + \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml new file mode 100644 index 00000000000..4b6ba057ecb --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml @@ -0,0 +1,9 @@ + + + + + default + 1 + + + diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py new file mode 100644 index 00000000000..c9b2c9fec2e --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py @@ -0,0 +1,204 @@ +#!/usr/bin/env python3 + +import pytest +import time + +from helpers.cluster import ClickHouseCluster, is_arm +import re + +from azure.storage.blob import BlobServiceClient +from helpers.cluster import ClickHouseCluster, ClickHouseInstance + +if is_arm(): + pytestmark = pytest.mark.skip + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node", + main_configs=["configs/named_collections_azure.xml", "configs/schema_cache_azure.xml"], + user_configs=["configs/disable_profilers_azure.xml", "configs/users_azure.xml"], + with_azurite=True, + ) + cluster.start() + container_client = cluster.blob_service_client.get_container_client("cont") + container_client.create_container() + yield cluster + finally: + cluster.shutdown() + + +def azure_query( + node, query, expect_error=False, try_num=10, settings={}, query_on_retry=None +): + for i in range(try_num): + try: + if expect_error: + return node.query_and_get_error(query, settings=settings) + else: + return node.query(query, settings=settings) + except Exception as ex: + retriable_errors = [ + "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", + "DB::Exception: Azure::Core::Http::TransportException: Connection closed before getting full response or response is less than expected", + "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", + "DB::Exception: Azure::Core::Http::TransportException: Error while polling for socket ready read", + "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", + "Azure::Core::Http::TransportException, e.what() = Connection closed before getting full response or response is less than expected", + "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", + "Azure::Core::Http::TransportException, e.what() = Error while polling for socket ready read", + ] + retry = False + for error in retriable_errors: + if error in str(ex): + retry = True + print(f"Try num: {i}. Having retriable error: {ex}") + time.sleep(i) + break + if not retry or i == try_num - 1: + raise Exception(ex) + if query_on_retry is not None: + node.query(query_on_retry) + continue + + +def get_azure_file_content(filename, port): + container_name = "cont" + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) + blob_service_client = BlobServiceClient.from_connection_string( + str(connection_string) + ) + container_client = blob_service_client.get_container_client(container_name) + blob_client = container_client.get_blob_client(filename) + download_stream = blob_client.download_blob() + return download_stream.readall().decode("utf-8") + + +@pytest.fixture(autouse=True, scope="function") +def delete_all_files(cluster): + port = cluster.env_variables["AZURITE_PORT"] + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) + blob_service_client = BlobServiceClient.from_connection_string(connection_string) + containers = blob_service_client.list_containers() + for container in containers: + container_client = blob_service_client.get_container_client(container) + blob_list = container_client.list_blobs() + for blob in blob_list: + print(blob) + blob_client = container_client.get_blob_client(blob) + blob_client.delete_blob() + + assert len(list(container_client.list_blobs())) == 0 + + yield + + +def test_azure_partitioning_with_one_parameter(cluster): + # type: (ClickHouseCluster) -> None + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 String, column2 String" + values = f"('Elizabeth', 'Gordon')" + path = "a/column1=Elizabeth/sample.csv" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + ) + + query = ( + f"SELECT column1, column2, _file, _path, _column1 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}')" + ) + assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ + "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format( + bucket="cont", max_path=path + ) + ] + + query = ( + f"SELECT column2 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" + ) + assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ + "Gordon" + ] + +def test_azure_partitioning_with_two_parameters(cluster): + # type: (ClickHouseCluster) -> None + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 String, column2 String" + values_1 = f"('Elizabeth', 'Gordon')" + values_2 = f"('Emilia', 'Gregor')" + path = "a/column1=Elizabeth/column2=Gordon/sample.csv" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + ) + + query = ( + f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" + ) + assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ + "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth\tGordon".format( + bucket="cont", max_path=path + ) + ] + + query = ( + f"SELECT column1 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2;" + ) + assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ + "Elizabeth" + ] + + query = ( + f"SELECT column1 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2 AND column1=_column1;" + ) + assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ + "Elizabeth" + ] + +def test_azure_partitioning_without_setting(cluster): + # type: (ClickHouseCluster) -> None + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 String, column2 String" + values_1 = f"('Elizabeth', 'Gordon')" + values_2 = f"('Emilia', 'Gregor')" + path = "a/column1=Elizabeth/column2=Gordon/sample.csv" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + ) + + query = ( + f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" + ) + pattern = re.compile(r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL) + + with pytest.raises(Exception, match=pattern): + azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 0}) diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py new file mode 100644 index 00000000000..38641b63960 --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py @@ -0,0 +1,81 @@ +#!/usr/bin/env python3 + +import pytest + +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster, is_arm +import re + +from helpers.cluster import ClickHouseCluster + +if is_arm(): + pytestmark = pytest.mark.skip + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", + main_configs=[ + "configs/macro_hdfs.xml", + "configs/schema_cache_hdfs.xml", + "configs/cluster_hdfs.xml", + ], + with_hdfs=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_hdfs_partitioning_with_one_parameter(started_cluster): + hdfs_api = started_cluster.hdfs_api + hdfs_api.write_data( + f"/column0=Elizabeth/parquet_1", f"Elizabeth\tGordon\n" + ) + assert ( + hdfs_api.read_data(f"/column0=Elizabeth/parquet_1") + == f"Elizabeth\tGordon\n" + ) + + r = node1.query( + "SELECT _column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", settings={"hdfs_hive_partitioning": 1} + ) + assert (r == f"Elizabeth\n") + +def test_hdfs_partitioning_with_two_parameters(started_cluster): + hdfs_api = started_cluster.hdfs_api + hdfs_api.write_data( + f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n" + ) + assert ( + hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") + == f"Elizabeth\tGordon\n" + ) + + r = node1.query( + "SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", settings={"hdfs_hive_partitioning": 1} + ) + assert (r == f"Gordon\n") + +def test_hdfs_partitioning_without_setting(started_cluster): + hdfs_api = started_cluster.hdfs_api + hdfs_api.write_data( + f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n" + ) + assert ( + hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") + == f"Elizabeth\tGordon\n" + ) + pattern = re.compile(r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL) + + with pytest.raises(QueryRuntimeException, match=pattern): + node1.query(f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", settings={"hdfs_hive_partitioning": 0}) + +if __name__ == "__main__": + cluster.start() + input("Cluster created, press any key to destroy...") + cluster.shutdown() diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference new file mode 100644 index 00000000000..6ef1fcdf652 --- /dev/null +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -0,0 +1,96 @@ +TESTING THE FILE HIVE PARTITIONING +first last Elizabeth +Jorge Frank Elizabeth +Hunter Moreno Elizabeth +Esther Guzman Elizabeth +Dennis Stephens Elizabeth +Nettie Franklin Elizabeth +Stanley Gibson Elizabeth +Eugenia Greer Elizabeth +Jeffery Delgado Elizabeth +Clara Cross Elizabeth +Elizabeth Gordon Elizabeth +Eva Schmidt Elizabeth Schmidt +Samuel Schmidt Elizabeth Schmidt +Eva Schmidt Elizabeth +Samuel Schmidt Elizabeth +Elizabeth Gordon Elizabeth Gordon +Elizabeth Gordon Elizabeth +Elizabeth Gordon Elizabeth Gordon +Elizabeth Gordon Elizabeth +first last Elizabeth +Jorge Frank Elizabeth +Hunter Moreno Elizabeth +Esther Guzman Elizabeth +Dennis Stephens Elizabeth +Nettie Franklin Elizabeth +Stanley Gibson Elizabeth +Eugenia Greer Elizabeth +Jeffery Delgado Elizabeth +Clara Cross Elizabeth +Elizabeth Gordon Elizabeth +1 +TESTING THE URL PARTITIONING +first last Elizabeth +Jorge Frank Elizabeth +Hunter Moreno Elizabeth +Esther Guzman Elizabeth +Dennis Stephens Elizabeth +Nettie Franklin Elizabeth +Stanley Gibson Elizabeth +Eugenia Greer Elizabeth +Jeffery Delgado Elizabeth +Clara Cross Elizabeth +Elizabeth Gordon Elizabeth +Eva Schmidt Elizabeth Schmidt +Samuel Schmidt Elizabeth Schmidt +Eva Schmidt Elizabeth +Samuel Schmidt Elizabeth +Elizabeth Gordon Elizabeth Gordon +Elizabeth Gordon Elizabeth +Elizabeth Gordon Elizabeth Gordon +Elizabeth Gordon Elizabeth +first last Elizabeth +Jorge Frank Elizabeth +Hunter Moreno Elizabeth +Esther Guzman Elizabeth +Dennis Stephens Elizabeth +Nettie Franklin Elizabeth +Stanley Gibson Elizabeth +Eugenia Greer Elizabeth +Jeffery Delgado Elizabeth +Clara Cross Elizabeth +Elizabeth Gordon Elizabeth +1 +TESTING THE S3 PARTITIONING +first last Elizabeth +Jorge Frank Elizabeth +Hunter Moreno Elizabeth +Esther Guzman Elizabeth +Dennis Stephens Elizabeth +Nettie Franklin Elizabeth +Stanley Gibson Elizabeth +Eugenia Greer Elizabeth +Jeffery Delgado Elizabeth +Clara Cross Elizabeth +Elizabeth Gordon Elizabeth +Eva Schmidt Elizabeth Schmidt +Samuel Schmidt Elizabeth Schmidt +Eva Schmidt Elizabeth +Samuel Schmidt Elizabeth +Elizabeth Gordon Elizabeth Gordon +Elizabeth Gordon Elizabeth +Elizabeth Gordon Elizabeth Gordon +Elizabeth Gordon Elizabeth +first last Elizabeth +Jorge Frank Elizabeth +Hunter Moreno Elizabeth +Esther Guzman Elizabeth +Dennis Stephens Elizabeth +Nettie Franklin Elizabeth +Stanley Gibson Elizabeth +Eugenia Greer Elizabeth +Jeffery Delgado Elizabeth +Clara Cross Elizabeth +Elizabeth Gordon Elizabeth +1 diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh new file mode 100755 index 00000000000..a5d4c85a33b --- /dev/null +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -0,0 +1,93 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" + + +$CLICKHOUSE_LOCAL -n -q """set file_hive_partitioning = 1; + +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; + +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; + +SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; + +SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; + +SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; + +SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; + +SELECT *, _non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" + +$CLICKHOUSE_LOCAL -n -q """set file_hive_partitioning = 0; + +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" + + +$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE URL PARTITIONING'" + + +$CLICKHOUSE_LOCAL -n -q """set url_hive_partitioning = 1; + +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; + +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; + +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; + +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; + +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; + +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; + +SELECT *, _non_existing_column FROM url('http://localhost:11111/test/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" + +$CLICKHOUSE_LOCAL -n -q """set url_hive_partitioning = 0; + +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" + + +$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3 PARTITIONING'" + + +$CLICKHOUSE_LOCAL -n -q """set s3_hive_partitioning = 1; + +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; + +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; + +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; + +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; + +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; + +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; + +SELECT *, _non_existing_column FROM s3('http://localhost:11111/test/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" + +$CLICKHOUSE_LOCAL -n -q """set s3_hive_partitioning = 0; + +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" + diff --git a/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Gordon/sample.parquet b/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Gordon/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Schmidt/sample.parquet b/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Schmidt/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/sample.parquet b/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_minio/hive_partitioning/non_existing_column=Elizabeth/sample.parquet b/tests/queries/0_stateless/data_minio/hive_partitioning/non_existing_column=Elizabeth/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 From 83462d743e76dcfa8fd35b8b30335682f86d9374 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 2 Jul 2024 16:13:44 +0200 Subject: [PATCH 120/644] enhance SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b0725340f46..607f9b6d858 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,6 +59,11 @@ static std::initializer_list Date: Tue, 2 Jul 2024 14:43:45 +0000 Subject: [PATCH 121/644] style check --- .../ObjectStorage/StorageObjectStorage.cpp | 2 +- .../StorageObjectStorageSource.cpp | 7 +-- .../test_azure.py | 43 +++++++++++++------ .../test_hdfs.py | 32 ++++++++------ .../03203_hive_style_partitioning.sh | 24 +++++++---- 5 files changed, 69 insertions(+), 39 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index b169f02940e..ae7c211330c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -86,7 +86,7 @@ StorageObjectStorage::StorageObjectStorage( ); Strings paths; - + if (checkIfHiveSettingEnabled(context, configuration->getTypeName())) if (auto file = file_iterator->next(0)) paths = {file->getPath()}; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 2741cfecf6b..afb23961312 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -198,13 +198,14 @@ Chunk StorageObjectStorageSource::generate() chassert(object_info->metadata); auto hive_map = VirtualColumnUtils::parsePartitionMapFromPath(object_info->getPath()); - bool contains_virtual_column = std::any_of(hive_map.begin(), hive_map.end(), - [&](const auto& pair) { + bool contains_virtual_column = std::any_of(hive_map.begin(), hive_map.end(), + [&](const auto& pair) + { return read_from_format_info.requested_virtual_columns.contains(pair.first); }); if (!contains_virtual_column) - hive_map.clear(); // If we cannot find any virual column in requested, we don't add any of them to chunk + hive_map.clear(); // If we cannot find any virtual column in requested, we don't add any of them to chunk VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( chunk, read_from_format_info.requested_virtual_columns, diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py index c9b2c9fec2e..0be697821f0 100644 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py @@ -12,14 +12,21 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance if is_arm(): pytestmark = pytest.mark.skip + @pytest.fixture(scope="module") def cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance( "node", - main_configs=["configs/named_collections_azure.xml", "configs/schema_cache_azure.xml"], - user_configs=["configs/disable_profilers_azure.xml", "configs/users_azure.xml"], + main_configs=[ + "configs/named_collections_azure.xml", + "configs/schema_cache_azure.xml", + ], + user_configs=[ + "configs/disable_profilers_azure.xml", + "configs/users_azure.xml", + ], with_azurite=True, ) cluster.start() @@ -121,7 +128,9 @@ def test_azure_partitioning_with_one_parameter(cluster): f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}')" ) - assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == [ "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format( bucket="cont", max_path=path ) @@ -132,9 +141,10 @@ def test_azure_partitioning_with_one_parameter(cluster): f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" ) - assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ - "Gordon" - ] + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == ["Gordon"] + def test_azure_partitioning_with_two_parameters(cluster): # type: (ClickHouseCluster) -> None @@ -155,7 +165,9 @@ def test_azure_partitioning_with_two_parameters(cluster): f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" ) - assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == [ "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth\tGordon".format( bucket="cont", max_path=path ) @@ -166,18 +178,19 @@ def test_azure_partitioning_with_two_parameters(cluster): f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2;" ) - assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ - "Elizabeth" - ] + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == ["Elizabeth"] query = ( f"SELECT column1 FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2 AND column1=_column1;" ) - assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ - "Elizabeth" - ] + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == ["Elizabeth"] + def test_azure_partitioning_without_setting(cluster): # type: (ClickHouseCluster) -> None @@ -198,7 +211,9 @@ def test_azure_partitioning_without_setting(cluster): f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" ) - pattern = re.compile(r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL) + pattern = re.compile( + r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL + ) with pytest.raises(Exception, match=pattern): azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 0}) diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py index 38641b63960..4667d18688a 100644 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py @@ -31,20 +31,18 @@ def started_cluster(): finally: cluster.shutdown() + def test_hdfs_partitioning_with_one_parameter(started_cluster): hdfs_api = started_cluster.hdfs_api - hdfs_api.write_data( - f"/column0=Elizabeth/parquet_1", f"Elizabeth\tGordon\n" - ) - assert ( - hdfs_api.read_data(f"/column0=Elizabeth/parquet_1") - == f"Elizabeth\tGordon\n" - ) + hdfs_api.write_data(f"/column0=Elizabeth/parquet_1", f"Elizabeth\tGordon\n") + assert hdfs_api.read_data(f"/column0=Elizabeth/parquet_1") == f"Elizabeth\tGordon\n" r = node1.query( - "SELECT _column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", settings={"hdfs_hive_partitioning": 1} + "SELECT _column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", + settings={"hdfs_hive_partitioning": 1}, ) - assert (r == f"Elizabeth\n") + assert r == f"Elizabeth\n" + def test_hdfs_partitioning_with_two_parameters(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -57,9 +55,11 @@ def test_hdfs_partitioning_with_two_parameters(started_cluster): ) r = node1.query( - "SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", settings={"hdfs_hive_partitioning": 1} + "SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", + settings={"hdfs_hive_partitioning": 1}, ) - assert (r == f"Gordon\n") + assert r == f"Gordon\n" + def test_hdfs_partitioning_without_setting(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -70,10 +70,16 @@ def test_hdfs_partitioning_without_setting(started_cluster): hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") == f"Elizabeth\tGordon\n" ) - pattern = re.compile(r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL) + pattern = re.compile( + r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL + ) with pytest.raises(QueryRuntimeException, match=pattern): - node1.query(f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", settings={"hdfs_hive_partitioning": 0}) + node1.query( + f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", + settings={"hdfs_hive_partitioning": 0}, + ) + if __name__ == "__main__": cluster.start() diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index a5d4c85a33b..83a8f87a813 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -8,7 +8,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" -$CLICKHOUSE_LOCAL -n -q """set file_hive_partitioning = 1; +$CLICKHOUSE_LOCAL -n -q """ +set file_hive_partitioning = 1; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -31,13 +32,15 @@ SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.pa $CLICKHOUSE_LOCAL -n -q """set file_hive_partitioning = 0; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE URL PARTITIONING'" -$CLICKHOUSE_LOCAL -n -q """set url_hive_partitioning = 1; +$CLICKHOUSE_LOCAL -n -q """ +set url_hive_partitioning = 1; SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -60,13 +63,15 @@ SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=*/ $CLICKHOUSE_LOCAL -n -q """set url_hive_partitioning = 0; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3 PARTITIONING'" -$CLICKHOUSE_LOCAL -n -q """set s3_hive_partitioning = 1; +$CLICKHOUSE_LOCAL -n -q """ +set s3_hive_partitioning = 1; SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -85,9 +90,12 @@ SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/c SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; SELECT *, _non_existing_column FROM s3('http://localhost:11111/test/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0; +""" -$CLICKHOUSE_LOCAL -n -q """set s3_hive_partitioning = 0; +$CLICKHOUSE_LOCAL -n -q """ +set s3_hive_partitioning = 0; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" From cd5cdcc124f95204a6f63e8a1ce4d7148d8fec7f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 2 Jul 2024 17:00:59 +0200 Subject: [PATCH 122/644] Shellcheck fix --- tests/queries/0_stateless/03203_hive_style_partitioning.sh | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 83a8f87a813..98c039f3454 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -30,7 +30,8 @@ SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/c SELECT *, _non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" -$CLICKHOUSE_LOCAL -n -q """set file_hive_partitioning = 0; +$CLICKHOUSE_LOCAL -n -q """ +set file_hive_partitioning = 0; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" @@ -61,7 +62,8 @@ SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=El SELECT *, _non_existing_column FROM url('http://localhost:11111/test/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" -$CLICKHOUSE_LOCAL -n -q """set url_hive_partitioning = 0; +$CLICKHOUSE_LOCAL -n -q """ +set url_hive_partitioning = 0; SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" @@ -98,4 +100,3 @@ set s3_hive_partitioning = 0; SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" - From dc9dc1676d8f8af74c20173927c6027623cc788c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 2 Jul 2024 17:37:47 +0200 Subject: [PATCH 123/644] add default for map in VirtualsForFileLikeStorage --- src/Storages/VirtualColumnUtils.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index a03d4c7447f..f9b49cc48ed 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -76,7 +76,7 @@ struct VirtualsForFileLikeStorage std::optional size { std::nullopt }; const String * filename { nullptr }; std::optional last_modified { std::nullopt }; - std::map hive_partitioning_map; + std::map hive_partitioning_map {}; }; From 61f863c4e1f1d99483af78824d1c5792059dc400 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 3 Jul 2024 13:47:18 +0000 Subject: [PATCH 124/644] fix ambiguous override of non-virtual --- src/Client/Connection.cpp | 2 +- src/IO/ReadBufferFromPocoSocketChunked.cpp | 8 ++++---- src/IO/ReadBufferFromPocoSocketChunked.h | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 803f68c69d6..198518d6314 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -1122,7 +1122,7 @@ bool Connection::poll(size_t timeout_microseconds) bool Connection::hasReadPendingData() const { - return last_input_packet_type.has_value() || in->hasPendingData(); + return last_input_packet_type.has_value() || in->hasBufferedData(); } diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 93afeadba60..4a1e3732a55 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -34,12 +34,12 @@ void ReadBufferFromPocoSocketChunked::enableChunked() next_chunk = 0; } -bool ReadBufferFromPocoSocketChunked::hasPendingData() const +bool ReadBufferFromPocoSocketChunked::hasBufferedData() const { - if (chunked) - return available() || static_cast(data_end - working_buffer.end()) > sizeof(next_chunk); + if (available()) + return true; - return ReadBufferFromPocoSocketBase::hasPendingData(); + return chunked && (static_cast(data_end - working_buffer.end()) > sizeof(next_chunk)); } bool ReadBufferFromPocoSocketChunked::poll(size_t timeout_microseconds) const diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index 943a50f5d08..8bc4024b978 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -84,7 +84,7 @@ public: void enableChunked(); - bool hasPendingData() const; + bool hasBufferedData() const; bool poll(size_t timeout_microseconds) const; From a99c803ddf137e91f43c8f26f549f74f71eab102 Mon Sep 17 00:00:00 2001 From: morning-color Date: Thu, 4 Jul 2024 17:26:06 +0800 Subject: [PATCH 125/644] Rename rows_before_group_by_at_least to rows_before_aggregation_at_least --- src/Client/ClientBase.cpp | 4 +- src/Core/Settings.h | 2 +- src/Formats/JSONUtils.cpp | 9 +- src/Formats/JSONUtils.h | 2 + src/Processors/Formats/IOutputFormat.cpp | 4 +- src/Processors/Formats/IOutputFormat.h | 15 +- ...ONColumnsWithMetadataBlockOutputFormat.cpp | 2 + ...JSONColumnsWithMetadataBlockOutputFormat.h | 5 + .../Formats/Impl/JSONRowOutputFormat.cpp | 2 + .../Formats/Impl/JSONRowOutputFormat.h | 5 + .../Impl/ParallelFormattingOutputFormat.h | 6 + .../Impl/TemplateBlockOutputFormat.cpp | 17 +- .../Formats/Impl/TemplateBlockOutputFormat.h | 8 +- .../Formats/Impl/XMLRowOutputFormat.cpp | 11 + .../Formats/Impl/XMLRowOutputFormat.h | 6 + src/Processors/Formats/LazyOutputFormat.cpp | 4 +- src/Processors/Formats/LazyOutputFormat.h | 2 +- .../Formats/PullingOutputFormat.cpp | 4 +- src/Processors/Formats/PullingOutputFormat.h | 2 +- src/Processors/IProcessor.h | 6 +- src/Processors/RowsBeforeLimitCounter.h | 2 +- src/Processors/Sources/DelayedSource.cpp | 6 + src/Processors/Sources/DelayedSource.h | 4 +- src/Processors/Sources/RemoteSource.cpp | 12 +- src/Processors/Sources/RemoteSource.h | 8 +- .../Transforms/AggregatingTransform.cpp | 4 +- .../Transforms/AggregatingTransform.h | 7 +- src/QueryPipeline/ProfileInfo.cpp | 20 +- src/QueryPipeline/ProfileInfo.h | 16 +- src/QueryPipeline/QueryPipeline.cpp | 21 +- src/Server/GRPCServer.cpp | 3 +- src/Server/grpc_protos/clickhouse_grpc.proto | 4 +- ...74_exact_rows_before_aggregation.reference | 355 ++++++++++++++++++ .../03174_exact_rows_before_aggregation.sql | 31 ++ 34 files changed, 538 insertions(+), 71 deletions(-) create mode 100644 tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference create mode 100644 tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index dbb67d230d5..4b82f30776f 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -522,8 +522,8 @@ void ClientBase::onProfileInfo(const ProfileInfo & profile_info) { if (profile_info.hasAppliedLimit() && output_format) output_format->setRowsBeforeLimit(profile_info.getRowsBeforeLimit()); - if (profile_info.hasAppliedGroupBy() && output_format) - output_format->setRowsBeforeGroupBy(profile_info.getRowsBeforeGroupBy()); + if (profile_info.hasAppliedAggregation() && output_format) + output_format->setRowsBeforeAggregation(profile_info.getRowsBeforeAggregation()); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 09291d4300d..2296a880bd6 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1200,7 +1200,7 @@ class IColumn; M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \ \ M(Bool, exact_rows_before_limit, false, "When enabled, ClickHouse will provide exact value for rows_before_limit_at_least statistic, but with the cost that the data before limit will have to be read completely", 0) \ - M(Bool, exact_rows_before_group_by, false, "When enabled, ClickHouse will provide exact value for rows_before_group_by_at_least statistic, but with the cost that the data before group by will have to be read completely", 0) \ + M(Bool, rows_before_aggregation, false, "When enabled, ClickHouse will provide exact value for rows_before_aggregation_at_least statistic, represents the number of rows read before aggregation", 0) \ M(UInt64, cross_to_inner_join_rewrite, 1, "Use inner join instead of comma/cross join if there are joining expressions in the WHERE section. Values: 0 - no rewrite, 1 - apply if possible for comma/cross, 2 - force rewrite all comma joins, cross - if possible", 0) \ \ M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \ diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index f0985f4a6b7..363e9344770 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -487,6 +487,8 @@ namespace JSONUtils size_t rows, size_t rows_before_limit, bool applied_limit, + size_t rows_before_aggregation, + bool applied_aggregation, const Stopwatch & watch, const Progress & progress, bool write_statistics, @@ -502,7 +504,12 @@ namespace JSONUtils writeTitle("rows_before_limit_at_least", out, 1, " "); writeIntText(rows_before_limit, out); } - + if (applied_aggregation) + { + writeFieldDelimiter(out, 2); + writeTitle("rows_before_aggregation_at_least", out, 1, " "); + writeIntText(rows_before_aggregation, out); + } if (write_statistics) { writeFieldDelimiter(out, 2); diff --git a/src/Formats/JSONUtils.h b/src/Formats/JSONUtils.h index 7ee111c1285..e2ac3467971 100644 --- a/src/Formats/JSONUtils.h +++ b/src/Formats/JSONUtils.h @@ -104,6 +104,8 @@ namespace JSONUtils size_t rows, size_t rows_before_limit, bool applied_limit, + size_t rows_before_aggregation, + bool applied_aggregation, const Stopwatch & watch, const Progress & progress, bool write_statistics, diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index 4191bf9f0fe..7eaecfab3dc 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -71,8 +71,8 @@ void IOutputFormat::work() { if (rows_before_limit_counter && rows_before_limit_counter->hasAppliedLimit()) setRowsBeforeLimit(rows_before_limit_counter->get()); - if (rows_before_group_by_counter && rows_before_group_by_counter->hasAppliedLimit()) - setRowsBeforeGroupBy(rows_before_group_by_counter->get()); + if (rows_before_aggregation_counter && rows_before_aggregation_counter->hasAppliedLimit()) + setRowsBeforeAggregation(rows_before_aggregation_counter->get()); finalize(); if (auto_flush) flush(); diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index 9bb7cccb612..1119797e7ff 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -36,16 +36,16 @@ public: void setAutoFlush() { auto_flush = true; } /// Value for rows_before_limit_at_least field. - virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) {} + virtual void setRowsBeforeLimit(size_t /*rows*/) { } /// Counter to calculate rows_before_limit_at_least in processors pipeline. void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_counter.swap(counter); } - /// Value for rows_before_group_by_at_least field. - virtual void setRowsBeforeGroupBy(size_t /*rows_before_limit*/) { } + /// Value for rows_before_aggregation_at_least field. + virtual void setRowsBeforeAggregation(size_t /*rows*/) { } - /// Counter to calculate rows_before_group_by_at_least in processors pipeline. - void setRowsBeforeGroupByCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_group_by_counter.swap(counter); } + /// Counter to calculate rows_before_aggregation_at_least in processors pipeline. + void setRowsBeforeAggregationCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_aggregation_counter.swap(counter); } /// Notify about progress. Method could be called from different threads. /// Passed value are delta, that must be summarized. @@ -157,7 +157,8 @@ protected: Progress progress; bool applied_limit = false; size_t rows_before_limit = 0; - size_t rows_before_group_by = 0; + bool applied_aggregation = false; + size_t rows_before_aggregation = 0; Chunk totals; Chunk extremes; }; @@ -192,7 +193,7 @@ protected: bool need_write_suffix = true; RowsBeforeLimitCounterPtr rows_before_limit_counter; - RowsBeforeGroupByCounterPtr rows_before_group_by_counter; + RowsBeforeAggregationCounterPtr rows_before_aggregation_counter; Statistics statistics; private: diff --git a/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.cpp b/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.cpp index 1e8f57aa9a6..2f285e3d202 100644 --- a/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.cpp @@ -81,6 +81,8 @@ void JSONColumnsWithMetadataBlockOutputFormat::finalizeImpl() rows, statistics.rows_before_limit, statistics.applied_limit, + statistics.rows_before_aggregation, + statistics.applied_aggregation, statistics.watch, statistics.progress, format_settings.write_statistics, diff --git a/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.h b/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.h index c72b4d87234..e5208440483 100644 --- a/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.h @@ -44,6 +44,11 @@ public: String getName() const override { return "JSONCompactColumnsBlockOutputFormat"; } void setRowsBeforeLimit(size_t rows_before_limit_) override { statistics.rows_before_limit = rows_before_limit_; statistics.applied_limit = true; } + void setRowsBeforeAggregation(size_t rows_before_aggregation_) override + { + statistics.rows_before_aggregation = rows_before_aggregation_; + statistics.applied_aggregation = true; + } void onProgress(const Progress & progress_) override { statistics.progress.incrementPiecewiseAtomically(progress_); } protected: diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index 20182d84917..fec24b10c11 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -116,6 +116,8 @@ void JSONRowOutputFormat::finalizeImpl() row_count, statistics.rows_before_limit, statistics.applied_limit, + statistics.rows_before_aggregation, + statistics.applied_aggregation, statistics.watch, statistics.progress, settings.write_statistics && exception_message.empty(), diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.h b/src/Processors/Formats/Impl/JSONRowOutputFormat.h index a38cd0e8db9..c36adb5ee3e 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.h @@ -35,6 +35,11 @@ public: statistics.applied_limit = true; statistics.rows_before_limit = rows_before_limit_; } + void setRowsBeforeAggregation(size_t rows_before_aggregation_) override + { + statistics.applied_aggregation = true; + statistics.rows_before_aggregation = rows_before_aggregation_; + } protected: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h index 341141dd633..66f8701161e 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h @@ -313,6 +313,12 @@ private: statistics.rows_before_limit = rows_before_limit; statistics.applied_limit = true; } + void setRowsBeforeAggregation(size_t rows_before_aggregation) override + { + std::lock_guard lock(statistics_mutex); + statistics.rows_before_aggregation = rows_before_aggregation; + statistics.applied_aggregation = true; + } }; } diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp index 1c43a0fa331..4bd6684dff4 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -42,9 +42,11 @@ TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, Writ case static_cast(ResultsetPart::TimeElapsed): case static_cast(ResultsetPart::RowsRead): case static_cast(ResultsetPart::BytesRead): + case static_cast(ResultsetPart::RowsBeforeAggregation): if (format.escaping_rules[i] == EscapingRule::None) - format.throwInvalidFormat("Serialization type for output part rows, rows_before_limit, time, " - "rows_read or bytes_read is not specified", i); + format.throwInvalidFormat( + "Serialization type for output part rows, rows, time, " + "rows_read or bytes_read is not specified", i); break; default: format.throwInvalidFormat("Invalid output part", i); @@ -80,7 +82,7 @@ TemplateBlockOutputFormat::ResultsetPart TemplateBlockOutputFormat::stringToResu return ResultsetPart::ExtremesMax; else if (part == "rows") return ResultsetPart::Rows; - else if (part == "rows_before_limit") + else if (part == "rows") return ResultsetPart::RowsBeforeLimit; else if (part == "time") return ResultsetPart::TimeElapsed; @@ -88,6 +90,8 @@ TemplateBlockOutputFormat::ResultsetPart TemplateBlockOutputFormat::stringToResu return ResultsetPart::RowsRead; else if (part == "bytes_read") return ResultsetPart::BytesRead; + else if (part == "rows_before_aggregation") + return ResultsetPart::RowsBeforeAggregation; else throw Exception(ErrorCodes::SYNTAX_ERROR, "Unknown output part {}", part); } @@ -161,7 +165,7 @@ void TemplateBlockOutputFormat::finalizeImpl() break; case ResultsetPart::RowsBeforeLimit: if (!statistics.applied_limit) - format.throwInvalidFormat("Cannot print rows_before_limit for this request", i); + format.throwInvalidFormat("Cannot print rows for this request", i); writeValue(statistics.rows_before_limit, format.escaping_rules[i]); break; case ResultsetPart::TimeElapsed: @@ -173,6 +177,11 @@ void TemplateBlockOutputFormat::finalizeImpl() case ResultsetPart::BytesRead: writeValue(statistics.progress.read_bytes.load(), format.escaping_rules[i]); break; + case ResultsetPart::RowsBeforeAggregation: + if (!statistics.applied_aggregation) + format.throwInvalidFormat("Cannot print rows_before_aggregation for this request", i); + writeValue(statistics.rows_before_aggregation, format.escaping_rules[i]); + break; default: break; } diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h index 53d98849482..5e88d79b4a8 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h @@ -21,6 +21,11 @@ public: String getName() const override { return "TemplateBlockOutputFormat"; } void setRowsBeforeLimit(size_t rows_before_limit_) override { statistics.rows_before_limit = rows_before_limit_; statistics.applied_limit = true; } + void setRowsBeforeAggregation(size_t rows_before_aggregation_) override + { + statistics.rows_before_aggregation = rows_before_aggregation_; + statistics.applied_aggregation = true; + } void onProgress(const Progress & progress_) override { statistics.progress.incrementPiecewiseAtomically(progress_); } enum class ResultsetPart : size_t @@ -33,7 +38,8 @@ public: RowsBeforeLimit, TimeElapsed, RowsRead, - BytesRead + BytesRead, + RowsBeforeAggregation }; static ResultsetPart stringToResultsetPart(const String & part); diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp index 52c161c3208..2fd0536ed02 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp @@ -191,6 +191,7 @@ void XMLRowOutputFormat::finalizeImpl() writeRowsBeforeLimitAtLeast(); + writeRowsBeforeAggregationAtLeast(); if (!exception_message.empty()) writeException(); @@ -219,6 +220,16 @@ void XMLRowOutputFormat::writeRowsBeforeLimitAtLeast() } } +void XMLRowOutputFormat::writeRowsBeforeAggregationAtLeast() +{ + if (statistics.applied_aggregation) + { + writeCString("\t", *ostr); + writeIntText(statistics.rows_before_aggregation, *ostr); + writeCString("\n", *ostr); + } +} + void XMLRowOutputFormat::writeStatistics() { writeCString("\t\n", *ostr); diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.h b/src/Processors/Formats/Impl/XMLRowOutputFormat.h index daf03539d0b..792acd118c8 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.h +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.h @@ -48,6 +48,11 @@ private: statistics.rows_before_limit = rows_before_limit_; } + void setRowsBeforeAggregation(size_t rows_before_aggregation_) override + { + statistics.applied_aggregation = true; + statistics.rows_before_aggregation = rows_before_aggregation_; + } void onRowsReadBeforeUpdate() override { row_count = getRowsReadBefore(); } void onProgress(const Progress & value) override; @@ -56,6 +61,7 @@ private: void writeExtremesElement(const char * title, const Columns & columns, size_t row_num); void writeRowsBeforeLimitAtLeast(); + void writeRowsBeforeAggregationAtLeast(); void writeStatistics(); void writeException(); diff --git a/src/Processors/Formats/LazyOutputFormat.cpp b/src/Processors/Formats/LazyOutputFormat.cpp index 63423628e57..dc099765870 100644 --- a/src/Processors/Formats/LazyOutputFormat.cpp +++ b/src/Processors/Formats/LazyOutputFormat.cpp @@ -45,8 +45,8 @@ void LazyOutputFormat::setRowsBeforeLimit(size_t rows_before_limit) info.setRowsBeforeLimit(rows_before_limit); } -void LazyOutputFormat::setRowsBeforeGroupBy(size_t rows_before_group_by) +void LazyOutputFormat::setRowsBeforeAggregation(size_t rows_before_aggregation) { - info.setRowsBeforeGroupBy(rows_before_group_by); + info.setRowsBeforeAggregation(rows_before_aggregation); } } diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index 83abb2ff1a1..b0936e3d258 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -28,7 +28,7 @@ public: ProfileInfo & getProfileInfo() { return info; } void setRowsBeforeLimit(size_t rows_before_limit) override; - void setRowsBeforeGroupBy(size_t rows_before_group_by) override; + void setRowsBeforeAggregation(size_t rows_before_aggregation) override; void onCancel() override { diff --git a/src/Processors/Formats/PullingOutputFormat.cpp b/src/Processors/Formats/PullingOutputFormat.cpp index 646755deb6b..37050fb9675 100644 --- a/src/Processors/Formats/PullingOutputFormat.cpp +++ b/src/Processors/Formats/PullingOutputFormat.cpp @@ -42,8 +42,8 @@ void PullingOutputFormat::setRowsBeforeLimit(size_t rows_before_limit) { info.setRowsBeforeLimit(rows_before_limit); } -void PullingOutputFormat::setRowsBeforeGroupBy(size_t rows_before_group_by) +void PullingOutputFormat::setRowsBeforeAggregation(size_t rows_before_aggregation) { - info.setRowsBeforeGroupBy(rows_before_group_by); + info.setRowsBeforeAggregation(rows_before_aggregation); } } diff --git a/src/Processors/Formats/PullingOutputFormat.h b/src/Processors/Formats/PullingOutputFormat.h index c4d8cf4aab2..f2546cca180 100644 --- a/src/Processors/Formats/PullingOutputFormat.h +++ b/src/Processors/Formats/PullingOutputFormat.h @@ -22,7 +22,7 @@ public: ProfileInfo & getProfileInfo() { return info; } void setRowsBeforeLimit(size_t rows_before_limit) override; - void setRowsBeforeGroupBy(size_t rows_before_group_by) override; + void setRowsBeforeAggregation(size_t rows_before_aggregation) override; bool expectMaterializedColumns() const override { return false; } diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 0df4b3168e3..a06958fed73 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -24,7 +24,7 @@ using StorageLimitsList = std::list; class RowsBeforeLimitCounter; using RowsBeforeLimitCounterPtr = std::shared_ptr; -using RowsBeforeGroupByCounterPtr = std::shared_ptr; +using RowsBeforeAggregationCounterPtr = std::shared_ptr; class IProcessor; using ProcessorPtr = std::shared_ptr; @@ -368,9 +368,9 @@ public: /// This counter is used to calculate the number of rows right before any filtration of LimitTransform. virtual void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr /* counter */) {} - /// Set rows_before_group_by counter for current processor. + /// Set rows_before_aggregation counter for current processor. /// This counter is used to calculate the number of rows right before AggregatingTransform. - virtual void setRowsBeforeGroupByCounter(RowsBeforeGroupByCounterPtr /* counter */) { } + virtual void setRowsBeforeAggregationCounter(RowsBeforeAggregationCounterPtr /* counter */) { } protected: virtual void onCancel() {} diff --git a/src/Processors/RowsBeforeLimitCounter.h b/src/Processors/RowsBeforeLimitCounter.h index f5eb40ff84a..5aa867ffc43 100644 --- a/src/Processors/RowsBeforeLimitCounter.h +++ b/src/Processors/RowsBeforeLimitCounter.h @@ -5,7 +5,7 @@ namespace DB { -/// This class helps to calculate rows_before_limit_at_least. +/// This class helps to calculate rows_before_limit_at_least and rows_before_aggregation_at_least. class RowsBeforeLimitCounter { public: diff --git a/src/Processors/Sources/DelayedSource.cpp b/src/Processors/Sources/DelayedSource.cpp index f7928f89015..788017e3df0 100644 --- a/src/Processors/Sources/DelayedSource.cpp +++ b/src/Processors/Sources/DelayedSource.cpp @@ -139,6 +139,12 @@ void DelayedSource::work() processor->setRowsBeforeLimitCounter(rows_before_limit); } + if (rows_before_aggregation) + { + for (auto & processor : processors) + processor->setRowsBeforeAggregationCounter(rows_before_aggregation); + } + synchronizePorts(totals_output, totals, header, processors); synchronizePorts(extremes_output, extremes, header, processors); } diff --git a/src/Processors/Sources/DelayedSource.h b/src/Processors/Sources/DelayedSource.h index bd100f29a47..3138a1ab42a 100644 --- a/src/Processors/Sources/DelayedSource.h +++ b/src/Processors/Sources/DelayedSource.h @@ -31,14 +31,14 @@ public: OutputPort * getExtremesPort() { return extremes; } void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); } - void setRowsBeforeGroupByCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_group_by.swap(counter); } + void setRowsBeforeAggregationCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_aggregation.swap(counter); } private: QueryPlanResourceHolder resources; Creator creator; Processors processors; RowsBeforeLimitCounterPtr rows_before_limit; - RowsBeforeLimitCounterPtr rows_before_group_by; + RowsBeforeLimitCounterPtr rows_before_aggregation; /// Outputs for DelayedSource. OutputPort * main = nullptr; diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index a78db630786..9a6fe239ee6 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -46,12 +46,12 @@ RemoteSource::RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation manually_add_rows_before_limit_counter = true; /// Remote subquery doesn't contain a limit } - if (rows_before_group_by) + if (rows_before_aggregation) { - if (info.hasAppliedGroupBy()) - rows_before_group_by->add(info.getRowsBeforeGroupBy()); + if (info.hasAppliedAggregation()) + rows_before_aggregation->add(info.getRowsBeforeAggregation()); else - manually_add_rows_before_group_by_counter = true; /// Remote subquery doesn't contain a group by + manually_add_rows_before_aggregation_counter = true; /// Remote subquery doesn't contain a group by } }); } @@ -171,8 +171,8 @@ std::optional RemoteSource::tryGenerate() { if (manually_add_rows_before_limit_counter) rows_before_limit->add(rows); - if (manually_add_rows_before_group_by_counter) - rows_before_group_by->add(rows); + if (manually_add_rows_before_aggregation_counter) + rows_before_aggregation->add(rows); query_executor->finish(); return {}; } diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index b2ea6d50e01..c2f4d6842bb 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -3,8 +3,8 @@ #include #include #include -#include +#include namespace DB { @@ -26,7 +26,7 @@ public: String getName() const override { return "Remote"; } void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); } - void setRowsBeforeGroupByCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_group_by.swap(counter); } + void setRowsBeforeAggregationCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_aggregation.swap(counter); } /// Stop reading from stream if output port is finished. void onUpdatePorts() override; @@ -46,7 +46,7 @@ private: bool add_aggregation_info = false; RemoteQueryExecutorPtr query_executor; RowsBeforeLimitCounterPtr rows_before_limit; - RowsBeforeLimitCounterPtr rows_before_group_by; + RowsBeforeLimitCounterPtr rows_before_aggregation; const bool async_read; const bool async_query_sending; @@ -54,7 +54,7 @@ private: int fd = -1; size_t rows = 0; bool manually_add_rows_before_limit_counter = false; - bool manually_add_rows_before_group_by_counter = false; + bool manually_add_rows_before_aggregation_counter = false; }; /// Totals source from RemoteQueryExecutor. diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 7c0e222f89b..684de0a3e8c 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -683,8 +683,8 @@ void AggregatingTransform::consume(Chunk chunk) LOG_TRACE(log, "Aggregating"); is_consume_started = true; } - if (rows_before_group_by_at_least) - rows_before_group_by_at_least->add(num_rows); + if (rows_before_aggregation_at_least) + rows_before_aggregation_at_least->add(num_rows); src_rows += num_rows; src_bytes += chunk.bytes(); diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 6e7b04f9191..53939ea6a99 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -169,7 +169,10 @@ public: Status prepare() override; void work() override; Processors expandPipeline() override; - void setRowsBeforeGroupByCounter(RowsBeforeGroupByCounterPtr counter) override { rows_before_group_by_at_least.swap(counter); } + void setRowsBeforeAggregationCounter(RowsBeforeAggregationCounterPtr counter) override + { + rows_before_aggregation_at_least.swap(counter); + } protected: void consume(Chunk chunk); @@ -213,7 +216,7 @@ private: bool is_consume_started = false; - RowsBeforeGroupByCounterPtr rows_before_group_by_at_least; + RowsBeforeAggregationCounterPtr rows_before_aggregation_at_least; void initGenerate(); }; diff --git a/src/QueryPipeline/ProfileInfo.cpp b/src/QueryPipeline/ProfileInfo.cpp index cec179ecfad..87729b7c90e 100644 --- a/src/QueryPipeline/ProfileInfo.cpp +++ b/src/QueryPipeline/ProfileInfo.cpp @@ -16,8 +16,8 @@ void ProfileInfo::read(ReadBuffer & in) readBinary(applied_limit, in); readVarUInt(rows_before_limit, in); readBinary(calculated_rows_before_limit, in); - readBinary(applied_group_by, in); - readVarUInt(rows_before_group_by, in); + readBinary(applied_aggregation, in); + readVarUInt(rows_before_aggregation, in); } @@ -29,8 +29,8 @@ void ProfileInfo::write(WriteBuffer & out) const writeBinary(hasAppliedLimit(), out); writeVarUInt(getRowsBeforeLimit(), out); writeBinary(calculated_rows_before_limit, out); - writeBinary(hasAppliedGroupBy(), out); - writeVarUInt(getRowsBeforeGroupBy(), out); + writeBinary(hasAppliedAggregation(), out); + writeVarUInt(getRowsBeforeAggregation(), out); } @@ -45,8 +45,8 @@ void ProfileInfo::setFrom(const ProfileInfo & rhs, bool skip_block_size_info) applied_limit = rhs.applied_limit; rows_before_limit = rhs.rows_before_limit; calculated_rows_before_limit = rhs.calculated_rows_before_limit; - applied_group_by = rhs.applied_group_by; - rows_before_group_by = rhs.rows_before_group_by; + applied_aggregation = rhs.applied_aggregation; + rows_before_aggregation = rhs.rows_before_aggregation; } @@ -63,15 +63,15 @@ bool ProfileInfo::hasAppliedLimit() const return applied_limit; } -size_t ProfileInfo::getRowsBeforeGroupBy() const +size_t ProfileInfo::getRowsBeforeAggregation() const { - return rows_before_group_by; + return rows_before_aggregation; } -bool ProfileInfo::hasAppliedGroupBy() const +bool ProfileInfo::hasAppliedAggregation() const { - return applied_group_by; + return applied_aggregation; } diff --git a/src/QueryPipeline/ProfileInfo.h b/src/QueryPipeline/ProfileInfo.h index 141adc7430d..e2467afd6f4 100644 --- a/src/QueryPipeline/ProfileInfo.h +++ b/src/QueryPipeline/ProfileInfo.h @@ -32,8 +32,8 @@ struct ProfileInfo size_t getRowsBeforeLimit() const; bool hasAppliedLimit() const; - size_t getRowsBeforeGroupBy() const; - bool hasAppliedGroupBy() const; + size_t getRowsBeforeAggregation() const; + bool hasAppliedAggregation() const; void update(Block & block); void update(size_t num_rows, size_t num_bytes); @@ -55,20 +55,20 @@ struct ProfileInfo } /// Only for Processors. - void setRowsBeforeGroupBy(size_t rows_before_group_by_) + void setRowsBeforeAggregation(size_t rows_before_aggregation_) { - applied_group_by = true; - rows_before_group_by = rows_before_group_by_; + applied_aggregation = true; + rows_before_aggregation = rows_before_aggregation_; } private: /// For these fields we make accessors, because they must be calculated beforehand. mutable bool applied_limit = false; /// Whether LIMIT was applied mutable size_t rows_before_limit = 0; - mutable bool calculated_rows_before_limit = false; /// Whether the field rows_before_limit was calculated + mutable bool calculated_rows_before_limit = false; /// Whether the field rows was calculated - mutable bool applied_group_by = false; /// Whether GROUP BY was applied - mutable size_t rows_before_group_by = 0; + mutable bool applied_aggregation = false; /// Whether GROUP BY was applied + mutable size_t rows_before_aggregation = 0; }; } diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 5e0885ed4e8..0e3eec21d50 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -274,18 +274,20 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) output_format->setRowsBeforeLimitCounter(rows_before_limit_at_least); } } -static void initRowsBeforeGroupBy(std::shared_ptr processors, IOutputFormat * output_format) +static void initRowsBeforeAggregation(std::shared_ptr processors, IOutputFormat * output_format) { if (!processors->empty()) { - RowsBeforeGroupByCounterPtr rows_before_group_by_at_least = std::make_shared(); + RowsBeforeAggregationCounterPtr rows_before_aggregation_at_least = std::make_shared(); for (auto & processor : *processors) { if (auto transform = std::dynamic_pointer_cast(processor)) - transform->setRowsBeforeGroupByCounter(rows_before_group_by_at_least); + transform->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); + if (auto remote = std::dynamic_pointer_cast(processor)) + remote->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); } - rows_before_group_by_at_least->add(0); - output_format->setRowsBeforeLimitCounter(rows_before_group_by_at_least); + rows_before_aggregation_at_least->add(0); + output_format->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); } } @@ -535,7 +537,14 @@ void QueryPipeline::complete(std::shared_ptr format) extremes = nullptr; initRowsBeforeLimit(format.get()); - initRowsBeforeGroupBy(processors, format.get()); + for (const auto context : resources.interpreter_context) + { + if (context->getSettingsRef().rows_before_aggregation) + { + initRowsBeforeAggregation(processors, format.get()); + break; + } + } output_format = format.get(); processors->emplace_back(std::move(format)); diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 37e4342f3b0..9651ce8f660 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1577,7 +1577,8 @@ namespace stats.set_allocated_bytes(info.bytes); stats.set_applied_limit(info.hasAppliedLimit()); stats.set_rows_before_limit(info.getRowsBeforeLimit()); - stats.set_rows_before_group_by(info.getRowsBeforeGroupBy()); + stats.set_applied_aggregation(info.hasAppliedAggregation()); + stats.set_rows_before_aggregation(info.getRowsBeforeAggregation()); } void Call::addLogsToResult() diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index 02b6988b8c0..2ada4e8a641 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -179,8 +179,8 @@ message Stats { uint64 allocated_bytes = 3; bool applied_limit = 4; uint64 rows_before_limit = 5; - bool applied_group_by = 6; - uint64 rows_before_group_by = 7; + bool applied_aggregation = 6; + uint64 rows_before_aggregation = 7; } message Exception { diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference new file mode 100644 index 00000000000..8ad00273bbf --- /dev/null +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference @@ -0,0 +1,355 @@ +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9] + ], + + "rows": 10, + + "rows_before_aggregation_at_least": 10 +} + + + + + + i + Int32 + + + + + + 0 + + + 1 + + + 2 + + + 3 + + + 4 + + + 5 + + + 6 + + + 7 + + + 8 + + + 9 + + + 10 + 10 + +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [10], + [11], + [12] + ], + + "rows": 3, + + "rows_before_aggregation_at_least": 3 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0] + ], + + "rows": 1, + + "rows_before_limit_at_least": 20, + + "rows_before_aggregation_at_least": 20 +} +{ + "meta": + [ + { + "name": "max(i)", + "type": "Int32" + } + ], + + "data": + [ + [19] + ], + + "rows": 1, + + "rows_before_aggregation_at_least": 20 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9], + [10], + [11], + [12], + [13], + [14], + [15], + [16], + [17], + [18], + [19], + [20], + [21], + [22], + [23], + [24], + [25], + [26], + [27], + [28], + [29] + ], + + "rows": 30, + + "rows_before_limit_at_least": 60, + + "rows_before_aggregation_at_least": 60 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9], + [10], + [11], + [12], + [13], + [14], + [15], + [16], + [17], + [18], + [19] + ], + + "rows": 20, + + "rows_before_limit_at_least": 40, + + "rows_before_aggregation_at_least": 40 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9], + [10], + [11], + [12], + [13], + [14], + [15], + [16], + [17], + [18], + [19], + [20], + [21], + [22], + [23], + [24], + [25], + [26], + [27], + [28], + [29] + ], + + "rows": 30, + + "rows_before_limit_at_least": 30, + + "rows_before_aggregation_at_least": 60 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9], + [10], + [11], + [12], + [13], + [14], + [15], + [16], + [17], + [18], + [19] + ], + + "rows": 20, + + "rows_before_limit_at_least": 20, + + "rows_before_aggregation_at_least": 40 +} +{ + "meta": + [ + { + "name": "max(i)", + "type": "Int32" + } + ], + + "data": + [ + [19] + ], + + "rows": 1, + + "rows_before_limit_at_least": 1, + + "rows_before_aggregation_at_least": 40 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [4], + [3], + [2], + [5], + [1], + [6], + [7], + [9], + [8] + ], + + "rows": 10, + + "rows_before_limit_at_least": 10, + + "rows_before_aggregation_at_least": 20 +} diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql new file mode 100644 index 00000000000..6a3759c11e8 --- /dev/null +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql @@ -0,0 +1,31 @@ +-- Tags: no-parallel, no-random-merge-tree-settings + +drop table if exists test; + +create table test (i int) engine MergeTree order by tuple(); + +insert into test select arrayJoin(range(10000)); + +set rows_before_aggregation = 1, output_format_write_statistics = 0, max_block_size = 100; + +select * from test where i < 10 group by i order by i FORMAT JSONCompact; +select * from test where i < 10 group by i order by i FORMAT XML; + +select * from test group by i having i in (10, 11, 12) order by i FORMAT JSONCompact; + +select * from test where i < 20 group by i order by i limit 1 FORMAT JSONCompact; + +select max(i) from test where i < 20 FORMAT JSONCompact; + +set prefer_localhost_replica = 0; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 30 group by i order by i FORMAT JSONCompact; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 group by i order by i FORMAT JSONCompact; + +set prefer_localhost_replica = 1; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 30 group by i order by i FORMAT JSONCompact; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 group by i order by i FORMAT JSONCompact; +select max(i) from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 FORMAT JSONCompact; + +select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i FORMAT JSONCompact; + +drop table if exists test; \ No newline at end of file From 92ce1368b3aa2056e05a89ad013d7fa818f9888d Mon Sep 17 00:00:00 2001 From: morning-color Date: Thu, 4 Jul 2024 18:03:40 +0800 Subject: [PATCH 126/644] Rename RowsBeforeLimitCounter to RowsBeforeStepCounter --- src/Processors/Formats/IOutputFormat.cpp | 4 +-- src/Processors/Formats/IOutputFormat.h | 4 +-- src/Processors/IProcessor.h | 6 ++-- src/Processors/LimitTransform.h | 4 +-- src/Processors/OffsetTransform.h | 4 +-- src/Processors/RowsBeforeLimitCounter.h | 36 ------------------- src/Processors/RowsBeforeStepCounter.h | 36 +++++++++++++++++++ src/Processors/Sources/RemoteSource.h | 2 +- .../Transforms/AggregatingTransform.h | 2 +- .../Transforms/PartialSortingTransform.h | 4 +-- src/QueryPipeline/QueryPipeline.cpp | 4 +-- src/QueryPipeline/QueryPipelineBuilder.cpp | 2 +- ...74_exact_rows_before_aggregation.reference | 27 +++++++++++--- .../03174_exact_rows_before_aggregation.sql | 9 +++-- 14 files changed, 81 insertions(+), 63 deletions(-) delete mode 100644 src/Processors/RowsBeforeLimitCounter.h create mode 100644 src/Processors/RowsBeforeStepCounter.h diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index 7eaecfab3dc..97628778adb 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -69,9 +69,9 @@ void IOutputFormat::work() if (finished && !finalized) { - if (rows_before_limit_counter && rows_before_limit_counter->hasAppliedLimit()) + if (rows_before_limit_counter && rows_before_limit_counter->hasAppliedStep()) setRowsBeforeLimit(rows_before_limit_counter->get()); - if (rows_before_aggregation_counter && rows_before_aggregation_counter->hasAppliedLimit()) + if (rows_before_aggregation_counter && rows_before_aggregation_counter->hasAppliedStep()) setRowsBeforeAggregation(rows_before_aggregation_counter->get()); finalize(); if (auto_flush) diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index 1119797e7ff..721e02f1c1d 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -1,9 +1,9 @@ #pragma once #include -#include -#include #include +#include +#include #include namespace DB diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index a06958fed73..ccdd6308de5 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -21,10 +21,10 @@ class IQueryPlanStep; struct StorageLimits; using StorageLimitsList = std::list; -class RowsBeforeLimitCounter; -using RowsBeforeLimitCounterPtr = std::shared_ptr; +class RowsBeforeStepCounter; +using RowsBeforeLimitCounterPtr = std::shared_ptr; -using RowsBeforeAggregationCounterPtr = std::shared_ptr; +using RowsBeforeAggregationCounterPtr = std::shared_ptr; class IProcessor; using ProcessorPtr = std::shared_ptr; diff --git a/src/Processors/LimitTransform.h b/src/Processors/LimitTransform.h index 33ff968985f..515203f6829 100644 --- a/src/Processors/LimitTransform.h +++ b/src/Processors/LimitTransform.h @@ -1,8 +1,8 @@ #pragma once -#include -#include #include +#include +#include namespace DB { diff --git a/src/Processors/OffsetTransform.h b/src/Processors/OffsetTransform.h index 79a7d15fe0b..7ef16518540 100644 --- a/src/Processors/OffsetTransform.h +++ b/src/Processors/OffsetTransform.h @@ -1,8 +1,8 @@ #pragma once -#include -#include #include +#include +#include namespace DB { diff --git a/src/Processors/RowsBeforeLimitCounter.h b/src/Processors/RowsBeforeLimitCounter.h deleted file mode 100644 index 5aa867ffc43..00000000000 --- a/src/Processors/RowsBeforeLimitCounter.h +++ /dev/null @@ -1,36 +0,0 @@ -#pragma once -#include -#include - -namespace DB -{ - -/// This class helps to calculate rows_before_limit_at_least and rows_before_aggregation_at_least. -class RowsBeforeLimitCounter -{ -public: - void add(uint64_t rows) - { - setAppliedLimit(); - rows_before_limit.fetch_add(rows, std::memory_order_release); - } - - void set(uint64_t rows) - { - setAppliedLimit(); - rows_before_limit.store(rows, std::memory_order_release); - } - - uint64_t get() const { return rows_before_limit.load(std::memory_order_acquire); } - - void setAppliedLimit() { has_applied_limit.store(true, std::memory_order_release); } - bool hasAppliedLimit() const { return has_applied_limit.load(std::memory_order_acquire); } - -private: - std::atomic rows_before_limit = 0; - std::atomic_bool has_applied_limit = false; -}; - -using RowsBeforeLimitCounterPtr = std::shared_ptr; - -} diff --git a/src/Processors/RowsBeforeStepCounter.h b/src/Processors/RowsBeforeStepCounter.h new file mode 100644 index 00000000000..d9912bfa076 --- /dev/null +++ b/src/Processors/RowsBeforeStepCounter.h @@ -0,0 +1,36 @@ +#pragma once +#include +#include + +namespace DB +{ + +/// This class helps to calculate rows_before_limit_at_least and rows_before_aggregation_at_least. +class RowsBeforeStepCounter +{ +public: + void add(uint64_t rows) + { + setAppliedStep(); + rows_before_step.fetch_add(rows, std::memory_order_release); + } + + void set(uint64_t rows) + { + setAppliedStep(); + rows_before_step.store(rows, std::memory_order_release); + } + + uint64_t get() const { return rows_before_step.load(std::memory_order_acquire); } + + void setAppliedStep() { has_applied_step.store(true, std::memory_order_release); } + bool hasAppliedStep() const { return has_applied_step.load(std::memory_order_acquire); } + +private: + std::atomic rows_before_step = 0; + std::atomic_bool has_applied_step = false; +}; + +using RowsBeforeLimitCounterPtr = std::shared_ptr; + +} diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index c2f4d6842bb..bbc563ec5fe 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 53939ea6a99..9f9638175f0 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Processors/Transforms/PartialSortingTransform.h b/src/Processors/Transforms/PartialSortingTransform.h index 8f25c93037f..abb4b290322 100644 --- a/src/Processors/Transforms/PartialSortingTransform.h +++ b/src/Processors/Transforms/PartialSortingTransform.h @@ -1,7 +1,7 @@ #pragma once -#include -#include #include +#include +#include #include namespace DB diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 0e3eec21d50..acccbed2585 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -262,7 +262,7 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) if (!processors.empty()) { - rows_before_limit_at_least = std::make_shared(); + rows_before_limit_at_least = std::make_shared(); for (auto & processor : processors) processor->setRowsBeforeLimitCounter(rows_before_limit_at_least); @@ -278,7 +278,7 @@ static void initRowsBeforeAggregation(std::shared_ptr processors, IO { if (!processors->empty()) { - RowsBeforeAggregationCounterPtr rows_before_aggregation_at_least = std::make_shared(); + RowsBeforeAggregationCounterPtr rows_before_aggregation_at_least = std::make_shared(); for (auto & processor : *processors) { if (auto transform = std::dynamic_pointer_cast(processor)) diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 803d1686ad7..d276fed60a2 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference index 8ad00273bbf..36db9721599 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference @@ -101,12 +101,29 @@ "data": [ - [0] + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9], + [10], + [11], + [12], + [13], + [14], + [15], + [16], + [17], + [18], + [19] ], - "rows": 1, - - "rows_before_limit_at_least": 20, + "rows": 20, "rows_before_aggregation_at_least": 20 } @@ -126,6 +143,8 @@ "rows": 1, + "rows_before_limit_at_least": 1, + "rows_before_aggregation_at_least": 20 } { diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql index 6a3759c11e8..8ccbce42706 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql @@ -6,16 +6,15 @@ create table test (i int) engine MergeTree order by tuple(); insert into test select arrayJoin(range(10000)); -set rows_before_aggregation = 1, output_format_write_statistics = 0, max_block_size = 100; +set rows_before_aggregation = 1, exact_rows_before_limit = 1, output_format_write_statistics = 0, max_block_size = 100; select * from test where i < 10 group by i order by i FORMAT JSONCompact; select * from test where i < 10 group by i order by i FORMAT XML; select * from test group by i having i in (10, 11, 12) order by i FORMAT JSONCompact; -select * from test where i < 20 group by i order by i limit 1 FORMAT JSONCompact; - -select max(i) from test where i < 20 FORMAT JSONCompact; +select * from test where i < 20 group by i order by i FORMAT JSONCompact; +select max(i) from test where i < 20 limit 1 FORMAT JSONCompact; set prefer_localhost_replica = 0; select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 30 group by i order by i FORMAT JSONCompact; @@ -26,6 +25,6 @@ select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 group by i order by i FORMAT JSONCompact; select max(i) from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 FORMAT JSONCompact; -select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i FORMAT JSONCompact; +select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i limit 10 FORMAT JSONCompact; drop table if exists test; \ No newline at end of file From 355f144cda1838acce8f89f54e3ae84300263ea3 Mon Sep 17 00:00:00 2001 From: morning-color Date: Thu, 4 Jul 2024 18:11:26 +0800 Subject: [PATCH 127/644] Fix rename bug --- src/Processors/Formats/IOutputFormat.h | 4 ++-- src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index 721e02f1c1d..40ac1317618 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -36,13 +36,13 @@ public: void setAutoFlush() { auto_flush = true; } /// Value for rows_before_limit_at_least field. - virtual void setRowsBeforeLimit(size_t /*rows*/) { } + virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) { } /// Counter to calculate rows_before_limit_at_least in processors pipeline. void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_counter.swap(counter); } /// Value for rows_before_aggregation_at_least field. - virtual void setRowsBeforeAggregation(size_t /*rows*/) { } + virtual void setRowsBeforeAggregation(size_t /*rows_before_limit*/) { } /// Counter to calculate rows_before_aggregation_at_least in processors pipeline. void setRowsBeforeAggregationCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_aggregation_counter.swap(counter); } diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp index 4bd6684dff4..5d6db17aaa2 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -82,7 +82,7 @@ TemplateBlockOutputFormat::ResultsetPart TemplateBlockOutputFormat::stringToResu return ResultsetPart::ExtremesMax; else if (part == "rows") return ResultsetPart::Rows; - else if (part == "rows") + else if (part == "rows_before_limit") return ResultsetPart::RowsBeforeLimit; else if (part == "time") return ResultsetPart::TimeElapsed; @@ -165,7 +165,7 @@ void TemplateBlockOutputFormat::finalizeImpl() break; case ResultsetPart::RowsBeforeLimit: if (!statistics.applied_limit) - format.throwInvalidFormat("Cannot print rows for this request", i); + format.throwInvalidFormat("Cannot print rows_before_limit for this request", i); writeValue(statistics.rows_before_limit, format.escaping_rules[i]); break; case ResultsetPart::TimeElapsed: From 5f42e1518277e54c34fce274b5cf46cd5069cb2a Mon Sep 17 00:00:00 2001 From: morning-color Date: Thu, 4 Jul 2024 19:58:22 +0800 Subject: [PATCH 128/644] Fix Bug --- src/QueryPipeline/QueryPipeline.cpp | 14 ++++++++++---- .../03174_exact_rows_before_aggregation.sql | 2 +- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index acccbed2585..311c8a60531 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -276,17 +276,23 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) } static void initRowsBeforeAggregation(std::shared_ptr processors, IOutputFormat * output_format) { + bool has_aggregation = false; + if (!processors->empty()) { RowsBeforeAggregationCounterPtr rows_before_aggregation_at_least = std::make_shared(); - for (auto & processor : *processors) + for (auto processor : *processors) { if (auto transform = std::dynamic_pointer_cast(processor)) + { transform->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); - if (auto remote = std::dynamic_pointer_cast(processor)) - remote->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); + has_aggregation = true; + } + if (typeid_cast(processor.get()) || typeid_cast(processor.get())) + processor->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); } - rows_before_aggregation_at_least->add(0); + if (has_aggregation) + rows_before_aggregation_at_least->add(0); output_format->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); } } diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql index 8ccbce42706..31b817e8a65 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql @@ -27,4 +27,4 @@ select max(i) from cluster(test_cluster_two_shards, currentDatabase(), test) whe select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i limit 10 FORMAT JSONCompact; -drop table if exists test; \ No newline at end of file +drop table if exists test; From 942f7d7532059cf931242ce5c94a39ea0344b50b Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 5 Jul 2024 12:44:31 +0000 Subject: [PATCH 129/644] fixes after review --- programs/obfuscator/Obfuscator.cpp | 4 +- src/Core/Settings.h | 6 +- src/Core/SettingsChangesHistory.cpp | 6 +- src/Formats/ReadSchemaUtils.cpp | 23 +- src/Formats/ReadSchemaUtils.h | 6 +- .../DataLakes/IStorageDataLake.h | 3 +- .../ObjectStorage/ReadBufferIterator.cpp | 2 +- .../ObjectStorage/ReadBufferIterator.h | 2 +- .../ObjectStorage/StorageObjectStorage.cpp | 60 ++--- .../ObjectStorage/StorageObjectStorage.h | 6 + .../StorageObjectStorageCluster.cpp | 3 +- .../StorageObjectStorageSource.cpp | 13 +- src/Storages/ObjectStorage/Utils.cpp | 7 +- src/Storages/ObjectStorage/Utils.h | 1 + .../StorageObjectStorageQueue.cpp | 3 +- src/Storages/StorageFile.cpp | 53 ++--- src/Storages/StorageURL.cpp | 24 +- src/Storages/VirtualColumnUtils.cpp | 63 +++-- src/Storages/VirtualColumnUtils.h | 9 +- src/TableFunctions/TableFunctionFormat.cpp | 10 +- .../TableFunctionObjectStorage.cpp | 3 +- .../__init__.py | 0 .../configs/cluster_azure.xml | 39 ---- .../configs/cluster_hdfs.xml | 33 --- .../configs/disable_profilers_azure.xml | 9 - .../configs/macro_hdfs.xml | 5 - .../configs/named_collections_azure.xml | 14 -- .../configs/schema_cache_azure.xml | 3 - .../configs/schema_cache_hdfs.xml | 3 - .../configs/users_azure.xml | 9 - .../test_azure.py | 219 ------------------ .../test_hdfs.py | 87 ------- .../test_storage_azure_blob_storage/test.py | 110 +++++++++ tests/integration/test_storage_hdfs/test.py | 49 ++++ 34 files changed, 304 insertions(+), 583 deletions(-) delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/__init__.py delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 688ae1a1143..11e85bc1302 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1307,7 +1307,9 @@ try throw ErrnoException(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Input must be seekable file (it will be read twice)"); SingleReadBufferIterator read_buffer_iterator(std::move(file)); - schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, context_const); + + std::string sample_string; + schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, sample_string, context_const); } else { diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8399d3925db..65b93b893b6 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -935,6 +935,7 @@ class IColumn; M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ M(Bool, allow_deprecated_error_prone_window_functions, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)", 0) \ M(Bool, allow_deprecated_snowflake_conversion_functions, false, "Enables deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake.", 0) \ + M(Bool, use_hive_partitioning, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines.", 0)\ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. @@ -1106,11 +1107,6 @@ class IColumn; M(Bool, input_format_tsv_skip_trailing_empty_lines, false, "Skip trailing empty lines in TSV format", 0) \ M(Bool, input_format_custom_skip_trailing_empty_lines, false, "Skip trailing empty lines in CustomSeparated format", 0) \ M(Bool, input_format_tsv_crlf_end_of_line, false, "If it is set true, file function will read TSV format with \\r\\n instead of \\n.", 0) \ - M(Bool, file_hive_partitioning, false, "Allows to use hive partitioning for file format", 0)\ - M(Bool, url_hive_partitioning, false, "Allows to use hive partitioning for url format", 0)\ - M(Bool, s3_hive_partitioning, false, "Allows to use hive partitioning for s3 format", 0)\ - M(Bool, azure_blob_storage_hive_partitioning, false, "Allows to use hive partitioning for AzureBlobStorage format", 0)\ - M(Bool, hdfs_hive_partitioning, false, "Allows to use hive partitioning for hdfs format", 0)\ \ M(Bool, input_format_native_allow_types_conversion, true, "Allow data types conversion in Native input format", 0) \ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 607f9b6d858..b676cd85ce6 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,11 +59,7 @@ static std::initializer_list readSchemaFromFormatImpl( std::optional format_name, const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, + std::string & sample_path, const ContextPtr & context) try { @@ -143,6 +144,10 @@ try { iterator_data = read_buffer_iterator.next(); + /// Extracting the File path for hive-style partitioning + if (sample_path.empty()) + sample_path = read_buffer_iterator.getLastFilePath(); + /// Read buffer iterator can determine the data format if it's unknown. /// For example by scanning schema cache or by finding new file with format extension. if (!format_name && iterator_data.format_name) @@ -163,7 +168,7 @@ try return {*iterator_data.cached_columns, *format_name}; } - schemas_for_union_mode.emplace_back(iterator_data.cached_columns->getAll(), read_buffer_iterator.getLastFileName()); + schemas_for_union_mode.emplace_back(iterator_data.cached_columns->getAll(), read_buffer_iterator.getLastFilePath()); continue; } @@ -249,7 +254,7 @@ try if (!names_and_types.empty()) read_buffer_iterator.setSchemaToLastFile(ColumnsDescription(names_and_types)); - schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFileName()); + schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFilePath()); } catch (...) { @@ -410,7 +415,7 @@ try throw Exception(ErrorCodes::CANNOT_DETECT_FORMAT, "The data format cannot be detected by the contents of the files. You can specify the format manually"); read_buffer_iterator.setSchemaToLastFile(ColumnsDescription(names_and_types)); - schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFileName()); + schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFilePath()); } if (format_name && mode == SchemaInferenceMode::DEFAULT) @@ -526,9 +531,9 @@ try } catch (Exception & e) { - auto file_name = read_buffer_iterator.getLastFileName(); - if (!file_name.empty()) - e.addMessage(fmt::format("(in file/uri {})", file_name)); + auto file_path = read_buffer_iterator.getLastFilePath(); + if (!file_path.empty()) + e.addMessage(fmt::format("(in file/uri {})", file_path)); throw; } @@ -536,17 +541,19 @@ ColumnsDescription readSchemaFromFormat( const String & format_name, const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, + std::string & sample_path, const ContextPtr & context) { - return readSchemaFromFormatImpl(format_name, format_settings, read_buffer_iterator, context).first; + return readSchemaFromFormatImpl(format_name, format_settings, read_buffer_iterator, sample_path, context).first; } std::pair detectFormatAndReadSchema( const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, + std::string & sample_path, const ContextPtr & context) { - return readSchemaFromFormatImpl(std::nullopt, format_settings, read_buffer_iterator, context); + return readSchemaFromFormatImpl(std::nullopt, format_settings, read_buffer_iterator, sample_path, context); } SchemaCache::Key getKeyForSchemaCache( diff --git a/src/Formats/ReadSchemaUtils.h b/src/Formats/ReadSchemaUtils.h index bb5e068f696..6c562a06bf0 100644 --- a/src/Formats/ReadSchemaUtils.h +++ b/src/Formats/ReadSchemaUtils.h @@ -56,8 +56,8 @@ struct IReadBufferIterator /// Set auto detected format name. virtual void setFormatName(const String & /*format_name*/) {} - /// Get last processed file name for better exception messages. - virtual String getLastFileName() const { return ""; } + /// Get last processed file path for better exception messages. + virtual String getLastFilePath() const { return ""; } /// Return true if method recreateLastReadBuffer is implemented. virtual bool supportsLastReadBufferRecreation() const { return false; } @@ -122,6 +122,7 @@ ColumnsDescription readSchemaFromFormat( const String & format_name, const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, + std::string & sample_path, const ContextPtr & context); /// Try to detect the format of the data and it's schema. @@ -131,6 +132,7 @@ ColumnsDescription readSchemaFromFormat( std::pair detectFormatAndReadSchema( const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, + std::string & sample_path, const ContextPtr & context); SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional & format_settings, const ContextPtr & context); diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index 83865c47eb8..5c40cda442b 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -89,8 +89,9 @@ public: { ConfigurationPtr configuration = base_configuration->clone(); configuration->setPaths(metadata->getDataFiles()); + std::string sample_string; return Storage::resolveSchemaFromData( - object_storage_, configuration, format_settings_, local_context); + object_storage_, configuration, format_settings_, sample_string, local_context); } } diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index 78cdc442f64..a47049791ae 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -131,7 +131,7 @@ void ReadBufferIterator::setFormatName(const String & format_name) format = format_name; } -String ReadBufferIterator::getLastFileName() const +String ReadBufferIterator::getLastFilePath() const { if (current_object_info) return current_object_info->getPath(); diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.h b/src/Storages/ObjectStorage/ReadBufferIterator.h index 6eeb52ec2ed..b81aebb7b07 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.h +++ b/src/Storages/ObjectStorage/ReadBufferIterator.h @@ -33,7 +33,7 @@ public: void setResultingSchema(const ColumnsDescription & columns) override; - String getLastFileName() const override; + String getLastFilePath() const override; void setFormatName(const String & format_name) override; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index ae7c211330c..717f48983f3 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -33,17 +33,22 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - -bool checkIfHiveSettingEnabled(const ContextPtr & context, const std::string & storage_type_name) +std::string StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata, ContextPtr context) { - if (storage_type_name == "s3") - return context->getSettings().s3_hive_partitioning; - else if (storage_type_name == "hdfs") - return context->getSettings().hdfs_hive_partitioning; - else if (storage_type_name == "azure") - return context->getSettings().azure_blob_storage_hive_partitioning; - else - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_name); + auto file_iterator = StorageObjectStorageSource::createFileIterator( + configuration, + object_storage, + distributed_processing, + context, + {}, // predicate + metadata.getColumns().getAll(), // virtual_columns + nullptr, // read_keys + {} // file_progress_callback + ); + + if (auto file = file_iterator->next(0)) + return file->getPath(); + return ""; } StorageObjectStorage::StorageObjectStorage( @@ -66,7 +71,9 @@ StorageObjectStorage::StorageObjectStorage( , log(getLogger(fmt::format("Storage{}({})", configuration->getEngineName(), table_id_.getFullTableName()))) { ColumnsDescription columns{columns_}; - resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, context); + + std::string sample_path; + resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, sample_path, context); configuration->check(context); StorageInMemoryMetadata metadata; @@ -74,23 +81,13 @@ StorageObjectStorage::StorageObjectStorage( metadata.setConstraints(constraints_); metadata.setComment(comment); - auto file_iterator = StorageObjectStorageSource::createFileIterator( - configuration, - object_storage, - distributed_processing_, - context, - {}, // predicate - metadata.getColumns().getAll(), // virtual_columns - nullptr, // read_keys - {} // file_progress_callback - ); + + if (sample_path.empty() && context->getSettings().use_hive_partitioning) + sample_path = getPathSample(metadata, context); + else if (!context->getSettings().use_hive_partitioning) + sample_path = ""; - Strings paths; - - if (checkIfHiveSettingEnabled(context, configuration->getTypeName())) - if (auto file = file_iterator->next(0)) - paths = {file->getPath()}; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), paths)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), sample_path)); setInMemoryMetadata(metadata); } @@ -386,33 +383,36 @@ ColumnsDescription StorageObjectStorage::resolveSchemaFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, + std::string & sample_path, const ContextPtr & context) { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); - return readSchemaFromFormat(configuration->format, format_settings, *iterator, context); + return readSchemaFromFormat(configuration->format, format_settings, *iterator, sample_path, context); } std::string StorageObjectStorage::resolveFormatFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, + std::string & sample_path, const ContextPtr & context) { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); - return detectFormatAndReadSchema(format_settings, *iterator, context).second; + return detectFormatAndReadSchema(format_settings, *iterator, sample_path, context).second; } std::pair StorageObjectStorage::resolveSchemaAndFormatFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, + std::string & sample_path, const ContextPtr & context) { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); - auto [columns, format] = detectFormatAndReadSchema(format_settings, *iterator, context); + auto [columns, format] = detectFormatAndReadSchema(format_settings, *iterator, sample_path, context); configuration->format = format; return std::pair(columns, format); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index cf8ec113653..dd7ec31c970 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -42,6 +42,7 @@ public: size_t list_object_keys_size; bool throw_on_zero_files_match; bool ignore_non_existent_file; + bool use_hive_partitioning; }; StorageObjectStorage( @@ -100,23 +101,28 @@ public: const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, + std::string & sample_path, const ContextPtr & context); static std::string resolveFormatFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, + std::string & sample_path, const ContextPtr & context); static std::pair resolveSchemaAndFormatFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, + std::string & sample_path, const ContextPtr & context); protected: virtual void updateConfiguration(ContextPtr local_context); + std::string getPathSample(StorageInMemoryMetadata metadata, ContextPtr context); + static std::unique_ptr createReadBufferIterator( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 78f568d8ae2..0dc4b845a47 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -33,7 +33,8 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( , object_storage(object_storage_) { ColumnsDescription columns{columns_}; - resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, {}, context_); + std::string sample_path; + resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, {}, sample_path, context_); configuration->check(context_); StorageInMemoryMetadata metadata; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index afb23961312..ecb3ff9d856 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -197,16 +197,6 @@ Chunk StorageObjectStorageSource::generate() const auto & filename = object_info->getFileName(); chassert(object_info->metadata); - auto hive_map = VirtualColumnUtils::parsePartitionMapFromPath(object_info->getPath()); - bool contains_virtual_column = std::any_of(hive_map.begin(), hive_map.end(), - [&](const auto& pair) - { - return read_from_format_info.requested_virtual_columns.contains(pair.first); - }); - - if (!contains_virtual_column) - hive_map.clear(); // If we cannot find any virtual column in requested, we don't add any of them to chunk - VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( chunk, read_from_format_info.requested_virtual_columns, { @@ -214,8 +204,7 @@ Chunk StorageObjectStorageSource::generate() .size = object_info->metadata->size_bytes, .filename = &filename, .last_modified = object_info->metadata->last_modified, - .hive_partitioning_map = hive_map - }); + }, object_info->getPath()); return chunk; } diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index e49e14d2a0c..73410d959e0 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -49,19 +49,20 @@ void resolveSchemaAndFormat( ObjectStoragePtr object_storage, const StorageObjectStorage::ConfigurationPtr & configuration, std::optional format_settings, + std::string & sample_path, const ContextPtr & context) { if (columns.empty()) { if (format == "auto") std::tie(columns, format) = - StorageObjectStorage::resolveSchemaAndFormatFromData(object_storage, configuration, format_settings, context); + StorageObjectStorage::resolveSchemaAndFormatFromData(object_storage, configuration, format_settings, sample_path, context); else - columns = StorageObjectStorage::resolveSchemaFromData(object_storage, configuration, format_settings, context); + columns = StorageObjectStorage::resolveSchemaFromData(object_storage, configuration, format_settings, sample_path, context); } else if (format == "auto") { - format = StorageObjectStorage::resolveFormatFromData(object_storage, configuration, format_settings, context); + format = StorageObjectStorage::resolveFormatFromData(object_storage, configuration, format_settings, sample_path, context); } if (!columns.hasOnlyOrdinary()) diff --git a/src/Storages/ObjectStorage/Utils.h b/src/Storages/ObjectStorage/Utils.h index 2077999df41..7ee14f50979 100644 --- a/src/Storages/ObjectStorage/Utils.h +++ b/src/Storages/ObjectStorage/Utils.h @@ -19,6 +19,7 @@ void resolveSchemaAndFormat( ObjectStoragePtr object_storage, const StorageObjectStorage::ConfigurationPtr & configuration, std::optional format_settings, + std::string & sample_path, const ContextPtr & context); } diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index 95265cde9ea..c12cdddeec7 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -160,7 +160,8 @@ StorageObjectStorageQueue::StorageObjectStorageQueue( configuration->check(context_); ColumnsDescription columns{columns_}; - resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, context_); + std::string sample_path; + resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, sample_path, context_); configuration->check(context_); StorageInMemoryMetadata storage_metadata; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 0c32f29cb34..9751d596fff 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -502,7 +502,7 @@ namespace StorageFile::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); } - String getLastFileName() const override + String getLastFilePath() const override { if (current_index != 0) return paths[current_index - 1]; @@ -777,7 +777,7 @@ namespace format = format_name; } - String getLastFileName() const override + String getLastFilePath() const override { return last_read_file_path; } @@ -880,10 +880,11 @@ std::pair StorageFile::getTableStructureAndFormatFro auto read_buffer_iterator = SingleReadBufferIterator(std::move(read_buf)); ColumnsDescription columns; + std::string sample_path; if (format) - columns = readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context); + columns = readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context); else - std::tie(columns, format) = detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); + std::tie(columns, format) = detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); peekable_read_buffer_from_fd = read_buffer_iterator.releaseBuffer(); if (peekable_read_buffer_from_fd) @@ -928,20 +929,21 @@ std::pair StorageFile::getTableStructureAndFormatFro } + std::string sample_path; if (archive_info) { ReadBufferFromArchiveIterator read_buffer_iterator(*archive_info, format, format_settings, context); if (format) - return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context), *format}; + return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context), *format}; - return detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); + return detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); } ReadBufferFromFileIterator read_buffer_iterator(paths, format, compression_method, format_settings, context); if (format) - return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context), *format}; + return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context), *format}; - return detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); + return detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); } ColumnsDescription StorageFile::getTableStructureFromFile( @@ -1097,10 +1099,10 @@ void StorageFile::setStorageMetadata(CommonArguments args) storage_metadata.setComment(args.comment); setInMemoryMetadata(storage_metadata); - Strings paths_for_virtuals; - if (args.getContext()->getSettingsRef().file_hive_partitioning) - paths_for_virtuals = paths; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), paths_for_virtuals)); + std::string path_for_virtuals; + if (args.getContext()->getSettingsRef().use_hive_partitioning && !paths.empty()) + path_for_virtuals = paths[0]; + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), path_for_virtuals, format_settings.value_or(FormatSettings{}))); } @@ -1442,14 +1444,9 @@ Chunk StorageFileSource::generate() chunk_size = input_format->getApproxBytesReadForChunk(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - std::map hive_map; - if (getContext()->getSettingsRef().file_hive_partitioning) - { - hive_map = VirtualColumnUtils::parsePartitionMapFromPath(current_path); - - for (const auto& item : hive_map) - requested_virtual_columns.push_back(NameAndTypePair(item.first, std::make_shared())); - } + std::string hive_partitioning_path; + if (getContext()->getSettingsRef().use_hive_partitioning) + hive_partitioning_path = current_path; /// Enrich with virtual columns. VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( @@ -1459,8 +1456,7 @@ Chunk StorageFileSource::generate() .size = current_file_size, .filename = (filename_override.has_value() ? &filename_override.value() : nullptr), .last_modified = current_file_last_modified, - .hive_partitioning_map = hive_map - }); + }, hive_partitioning_path); return chunk; } @@ -1636,16 +1632,6 @@ void ReadFromFile::createIterator(const ActionsDAG::Node * predicate) storage->distributed_processing); } -void addPartitionColumnsToInfoHeader(Strings paths, ReadFromFormatInfo & info) -{ - for (const auto& path : paths) - { - auto map = VirtualColumnUtils::parsePartitionMapFromPath(path); - for (const auto& item : map) - info.source_header.insertUnique(ColumnWithTypeAndName(std::make_shared(), item.first)); - } -} - void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { createIterator(nullptr); @@ -1665,9 +1651,6 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui paths = storage->paths; } - if (getContext()->getSettingsRef().file_hive_partitioning) - addPartitionColumnsToInfoHeader(paths, info); - if (max_num_streams > files_to_read) num_streams = files_to_read; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index f6374701fc2..59c5465a381 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -153,10 +153,10 @@ IStorageURLBase::IStorageURLBase( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - Strings uri_for_partitioning; - if (context_->getSettingsRef().url_hive_partitioning) - uri_for_partitioning = {uri}; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), uri_for_partitioning)); + std::string uri_for_partitioning; + if (context_->getSettingsRef().use_hive_partitioning) + uri_for_partitioning = uri; + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), uri_for_partitioning, format_settings.value_or(FormatSettings{}))); } @@ -415,9 +415,9 @@ Chunk StorageURLSource::generate() size_t chunk_size = 0; if (input_format) chunk_size = input_format->getApproxBytesReadForChunk(); - std::map hive_map; - if (getContext()->getSettingsRef().url_hive_partitioning) - hive_map = VirtualColumnUtils::parsePartitionMapFromPath(curr_uri.getPath()); + std::string hive_partitioning_path; + if (getContext()->getSettingsRef().use_hive_partitioning) + hive_partitioning_path = curr_uri.getPath(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( @@ -425,8 +425,7 @@ Chunk StorageURLSource::generate() { .path = curr_uri.getPath(), .size = current_file_size, - .hive_partitioning_map = hive_map - }); + }, hive_partitioning_path); return chunk; } @@ -859,7 +858,7 @@ namespace format = format_name; } - String getLastFileName() const override { return current_url_option; } + String getLastFilePath() const override { return current_url_option; } bool supportsLastReadBufferRecreation() const override { return true; } @@ -960,9 +959,10 @@ std::pair IStorageURLBase::getTableStructureAndForma urls_to_check = {uri}; ReadBufferIterator read_buffer_iterator(urls_to_check, format, compression_method, headers, format_settings, context); + std::string sample_path; if (format) - return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context), *format}; - return detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); + return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context), *format}; + return detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); } ColumnsDescription IStorageURLBase::getTableStructureFromData( diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 0b79e3b7a16..379b14d8e51 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include "Functions/FunctionsLogical.h" #include "Functions/IFunction.h" #include "Functions/IFunctionAdaptors.h" @@ -115,22 +116,19 @@ NameSet getVirtualNamesForFileLikeStorage() return {"_path", "_file", "_size", "_time"}; } -Strings parseVirtualColumnNameFromPath(const std::string & path) +std::map parseFromPath(const std::string& path) { std::string pattern = "/([^/]+)=([^/]+)"; - // Map to store the key-value pairs - std::map key_values; - re2::StringPiece input_piece(path); - std::string key; - Strings result; - while (RE2::FindAndConsume(&input_piece, pattern, &key)) - result.push_back(key); - return result; + std::map key_values; + std::string key, value; + while (RE2::FindAndConsume(&input_piece, pattern, &key, &value)) + key_values["_" + key] = value; + return key_values; } -VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, Strings paths) +VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, std::string path, FormatSettings settings) { VirtualColumnsDescription desc; @@ -147,11 +145,13 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription add_virtual("_size", makeNullable(std::make_shared())); add_virtual("_time", makeNullable(std::make_shared())); - for (const auto& path : paths) + auto map = parseFromPath(path); + for (const auto& item : map) { - auto names = parseVirtualColumnNameFromPath(path); - for (const auto& name : names) - add_virtual("_" + name, std::make_shared(std::make_shared())); + auto type = tryInferDataTypeForSingleField(item.second, settings); + if (type == nullptr) + type = std::make_shared(); + add_virtual(item.first, std::make_shared(type)); } return desc; @@ -213,25 +213,11 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const return block.getByName("_idx").column; } -std::map parsePartitionMapFromPath(const std::string & path) -{ - std::string pattern = "/([^/]+)=([^/]+)"; // Regex to capture key=value pairs - // Map to store the key-value pairs - std::map key_values; - - re2::StringPiece input_piece(path); - std::string key; - std::string value; - while (RE2::FindAndConsume(&input_piece, pattern, &key, &value)) - key_values["_" + key] = value; - - return key_values; -} - void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values) + VirtualsForFileLikeStorage virtual_values, const std::string & hive_partitioning_path) { + auto hive_map = parseFromPath(hive_partitioning_path); for (const auto & virtual_column : requested_virtual_columns) { if (virtual_column.name == "_path") @@ -265,13 +251,22 @@ void addRequestedFileLikeStorageVirtualsToChunk( else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } - else + else if (!hive_map.empty()) { - auto it = virtual_values.hive_partitioning_map.find(virtual_column.getNameInStorage()); - if (it != virtual_values.hive_partitioning_map.end()) + bool contains_virtual_column = std::any_of(hive_map.begin(), hive_map.end(), + [&](const auto& pair) + { + return requested_virtual_columns.contains(pair.first); + }); + + if (!contains_virtual_column) + hive_map.clear(); // If we cannot find any virtual column in requested, we don't add any of them to chunk + + auto it = hive_map.find(virtual_column.getNameInStorage()); + if (it != hive_map.end()) { chunk.addColumn(virtual_column.getTypeInStorage()->createColumnConst(chunk.getNumRows(), it->second)->convertToFullColumnIfConst()); - virtual_values.hive_partitioning_map.erase(it); + hive_map.erase(it); } } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index f9b49cc48ed..72922be60bd 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -49,7 +50,7 @@ auto extractSingleValueFromBlock(const Block & block, const String & name) } NameSet getVirtualNamesForFileLikeStorage(); -VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, Strings paths = {}); +VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, std::string path = "", FormatSettings settings = FormatSettings()); ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); @@ -76,15 +77,13 @@ struct VirtualsForFileLikeStorage std::optional size { std::nullopt }; const String * filename { nullptr }; std::optional last_modified { std::nullopt }; - std::map hive_partitioning_map {}; - }; -std::map parsePartitionMapFromPath(const std::string & path); +std::map parseFromPath(const std::string& path); void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values); + VirtualsForFileLikeStorage virtual_values, const std::string & hive_partitioning_path = ""); } } diff --git a/src/TableFunctions/TableFunctionFormat.cpp b/src/TableFunctions/TableFunctionFormat.cpp index ad2a142a140..66152cb0c91 100644 --- a/src/TableFunctions/TableFunctionFormat.cpp +++ b/src/TableFunctions/TableFunctionFormat.cpp @@ -85,9 +85,10 @@ ColumnsDescription TableFunctionFormat::getActualTableStructure(ContextPtr conte if (structure == "auto") { SingleReadBufferIterator read_buffer_iterator(std::make_unique(data)); + std::string sample_path; if (format == "auto") - return detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, context).first; - return readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, context); + return detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, sample_path, context).first; + return readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, sample_path, context); } return parseColumnsListFromString(structure, context); } @@ -131,11 +132,12 @@ StoragePtr TableFunctionFormat::executeImpl(const ASTPtr & /*ast_function*/, Con String format_name = format; if (structure == "auto") { + std::string sample_path; SingleReadBufferIterator read_buffer_iterator(std::make_unique(data)); if (format_name == "auto") - std::tie(columns, format_name) = detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, context); + std::tie(columns, format_name) = detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, sample_path, context); else - columns = readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, context); + columns = readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, sample_path, context); } else { diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 550d9cc799b..39392a4c44c 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -84,7 +84,8 @@ ColumnsDescription TableFunctionObjectStorage< context->checkAccess(getSourceAccessType()); ColumnsDescription columns; auto storage = getObjectStorage(context, !is_insert_query); - resolveSchemaAndFormat(columns, configuration->format, storage, configuration, std::nullopt, context); + std::string sample_path; + resolveSchemaAndFormat(columns, configuration->format, storage, configuration, std::nullopt, sample_path, context); return columns; } else diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/__init__.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml deleted file mode 100644 index ffa4673c9ee..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml +++ /dev/null @@ -1,39 +0,0 @@ - - - - - - node_0 - 9000 - - - node_1 - 9000 - - - node_2 - 9000 - - - - - - - - node_0 - 9000 - - - - - node_1 - 19000 - - - - - - - simple_cluster - - \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml deleted file mode 100644 index b99b21ea40b..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml +++ /dev/null @@ -1,33 +0,0 @@ - - - - - - node1 - 9000 - - - - - node1 - 19000 - - - - - - - - 127.0.0.1 - 9000 - - - - - 127.0.0.2 - 9000 - - - - - diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml deleted file mode 100644 index a39badbf8ec..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - - - 0 - 0 - - - diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml deleted file mode 100644 index c2e11b47a5e..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml +++ /dev/null @@ -1,5 +0,0 @@ - - - test_cluster_two_shards - - \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml deleted file mode 100644 index bd7f9ff97f1..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml +++ /dev/null @@ -1,14 +0,0 @@ - - - - cont - test_simple_write_named.csv - key UInt64, data String - CSV - - - devstoreaccount1 - Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== - - - diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml deleted file mode 100644 index e2168ecd06d..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 2 - \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml deleted file mode 100644 index 37639649b5f..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 2 - \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml deleted file mode 100644 index 4b6ba057ecb..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - - - default - 1 - - - diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py deleted file mode 100644 index 0be697821f0..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py +++ /dev/null @@ -1,219 +0,0 @@ -#!/usr/bin/env python3 - -import pytest -import time - -from helpers.cluster import ClickHouseCluster, is_arm -import re - -from azure.storage.blob import BlobServiceClient -from helpers.cluster import ClickHouseCluster, ClickHouseInstance - -if is_arm(): - pytestmark = pytest.mark.skip - - -@pytest.fixture(scope="module") -def cluster(): - try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "node", - main_configs=[ - "configs/named_collections_azure.xml", - "configs/schema_cache_azure.xml", - ], - user_configs=[ - "configs/disable_profilers_azure.xml", - "configs/users_azure.xml", - ], - with_azurite=True, - ) - cluster.start() - container_client = cluster.blob_service_client.get_container_client("cont") - container_client.create_container() - yield cluster - finally: - cluster.shutdown() - - -def azure_query( - node, query, expect_error=False, try_num=10, settings={}, query_on_retry=None -): - for i in range(try_num): - try: - if expect_error: - return node.query_and_get_error(query, settings=settings) - else: - return node.query(query, settings=settings) - except Exception as ex: - retriable_errors = [ - "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", - "DB::Exception: Azure::Core::Http::TransportException: Connection closed before getting full response or response is less than expected", - "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", - "DB::Exception: Azure::Core::Http::TransportException: Error while polling for socket ready read", - "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", - "Azure::Core::Http::TransportException, e.what() = Connection closed before getting full response or response is less than expected", - "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", - "Azure::Core::Http::TransportException, e.what() = Error while polling for socket ready read", - ] - retry = False - for error in retriable_errors: - if error in str(ex): - retry = True - print(f"Try num: {i}. Having retriable error: {ex}") - time.sleep(i) - break - if not retry or i == try_num - 1: - raise Exception(ex) - if query_on_retry is not None: - node.query(query_on_retry) - continue - - -def get_azure_file_content(filename, port): - container_name = "cont" - connection_string = ( - f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" - f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" - f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" - ) - blob_service_client = BlobServiceClient.from_connection_string( - str(connection_string) - ) - container_client = blob_service_client.get_container_client(container_name) - blob_client = container_client.get_blob_client(filename) - download_stream = blob_client.download_blob() - return download_stream.readall().decode("utf-8") - - -@pytest.fixture(autouse=True, scope="function") -def delete_all_files(cluster): - port = cluster.env_variables["AZURITE_PORT"] - connection_string = ( - f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" - f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" - f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" - ) - blob_service_client = BlobServiceClient.from_connection_string(connection_string) - containers = blob_service_client.list_containers() - for container in containers: - container_client = blob_service_client.get_container_client(container) - blob_list = container_client.list_blobs() - for blob in blob_list: - print(blob) - blob_client = container_client.get_blob_client(blob) - blob_client.delete_blob() - - assert len(list(container_client.list_blobs())) == 0 - - yield - - -def test_azure_partitioning_with_one_parameter(cluster): - # type: (ClickHouseCluster) -> None - node = cluster.instances["node"] # type: ClickHouseInstance - table_format = "column1 String, column2 String" - values = f"('Elizabeth', 'Gordon')" - path = "a/column1=Elizabeth/sample.csv" - - azure_query( - node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", - ) - - query = ( - f"SELECT column1, column2, _file, _path, _column1 FROM azureBlobStorage(azure_conf2, " - f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}')" - ) - assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} - ).splitlines() == [ - "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format( - bucket="cont", max_path=path - ) - ] - - query = ( - f"SELECT column2 FROM azureBlobStorage(azure_conf2, " - f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" - ) - assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} - ).splitlines() == ["Gordon"] - - -def test_azure_partitioning_with_two_parameters(cluster): - # type: (ClickHouseCluster) -> None - node = cluster.instances["node"] # type: ClickHouseInstance - table_format = "column1 String, column2 String" - values_1 = f"('Elizabeth', 'Gordon')" - values_2 = f"('Emilia', 'Gregor')" - path = "a/column1=Elizabeth/column2=Gordon/sample.csv" - - azure_query( - node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", - ) - - query = ( - f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, " - f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" - ) - assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} - ).splitlines() == [ - "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth\tGordon".format( - bucket="cont", max_path=path - ) - ] - - query = ( - f"SELECT column1 FROM azureBlobStorage(azure_conf2, " - f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2;" - ) - assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} - ).splitlines() == ["Elizabeth"] - - query = ( - f"SELECT column1 FROM azureBlobStorage(azure_conf2, " - f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2 AND column1=_column1;" - ) - assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} - ).splitlines() == ["Elizabeth"] - - -def test_azure_partitioning_without_setting(cluster): - # type: (ClickHouseCluster) -> None - node = cluster.instances["node"] # type: ClickHouseInstance - table_format = "column1 String, column2 String" - values_1 = f"('Elizabeth', 'Gordon')" - values_2 = f"('Emilia', 'Gregor')" - path = "a/column1=Elizabeth/column2=Gordon/sample.csv" - - azure_query( - node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", - ) - - query = ( - f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, " - f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" - ) - pattern = re.compile( - r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL - ) - - with pytest.raises(Exception, match=pattern): - azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 0}) diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py deleted file mode 100644 index 4667d18688a..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py +++ /dev/null @@ -1,87 +0,0 @@ -#!/usr/bin/env python3 - -import pytest - -from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster, is_arm -import re - -from helpers.cluster import ClickHouseCluster - -if is_arm(): - pytestmark = pytest.mark.skip - -cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance( - "node1", - main_configs=[ - "configs/macro_hdfs.xml", - "configs/schema_cache_hdfs.xml", - "configs/cluster_hdfs.xml", - ], - with_hdfs=True, -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_hdfs_partitioning_with_one_parameter(started_cluster): - hdfs_api = started_cluster.hdfs_api - hdfs_api.write_data(f"/column0=Elizabeth/parquet_1", f"Elizabeth\tGordon\n") - assert hdfs_api.read_data(f"/column0=Elizabeth/parquet_1") == f"Elizabeth\tGordon\n" - - r = node1.query( - "SELECT _column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", - settings={"hdfs_hive_partitioning": 1}, - ) - assert r == f"Elizabeth\n" - - -def test_hdfs_partitioning_with_two_parameters(started_cluster): - hdfs_api = started_cluster.hdfs_api - hdfs_api.write_data( - f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n" - ) - assert ( - hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") - == f"Elizabeth\tGordon\n" - ) - - r = node1.query( - "SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", - settings={"hdfs_hive_partitioning": 1}, - ) - assert r == f"Gordon\n" - - -def test_hdfs_partitioning_without_setting(started_cluster): - hdfs_api = started_cluster.hdfs_api - hdfs_api.write_data( - f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n" - ) - assert ( - hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") - == f"Elizabeth\tGordon\n" - ) - pattern = re.compile( - r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL - ) - - with pytest.raises(QueryRuntimeException, match=pattern): - node1.query( - f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", - settings={"hdfs_hive_partitioning": 0}, - ) - - -if __name__ == "__main__": - cluster.start() - input("Cluster created, press any key to destroy...") - cluster.shutdown() diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 20b004a7605..893df6d23aa 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -5,6 +5,7 @@ import json import logging import os import io +import re import random import threading import time @@ -1462,3 +1463,112 @@ def test_insert_create_new_file(cluster): assert TSV(res) == TSV( "test_create_new_file.csv\t1\ntest_create_new_file.1.csv\t2\n" ) + + +def test_hive_partitioning_with_one_parameter(cluster): + # type: (ClickHouseCluster) -> None + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 String, column2 String" + values = f"('Elizabeth', 'Gordon')" + path = "a/column1=Elizabeth/sample.csv" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + ) + + query = ( + f"SELECT column1, column2, _file, _path, _column1 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}')" + ) + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == [ + "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format( + bucket="cont", max_path=path + ) + ] + + query = ( + f"SELECT column2 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" + ) + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == ["Gordon"] + + +def test_hive_partitioning_with_two_parameters(cluster): + # type: (ClickHouseCluster) -> None + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 String, column2 String" + values_1 = f"('Elizabeth', 'Gordon')" + values_2 = f"('Emilia', 'Gregor')" + path = "a/column1=Elizabeth/column2=Gordon/sample.csv" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + ) + + query = ( + f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" + ) + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == [ + "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth\tGordon".format( + bucket="cont", max_path=path + ) + ] + + query = ( + f"SELECT column1 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2;" + ) + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == ["Elizabeth"] + + query = ( + f"SELECT column1 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2 AND column1=_column1;" + ) + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == ["Elizabeth"] + + +def test_hive_partitioning_without_setting(cluster): + # type: (ClickHouseCluster) -> None + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 String, column2 String" + values_1 = f"('Elizabeth', 'Gordon')" + values_2 = f"('Emilia', 'Gregor')" + path = "a/column1=Elizabeth/column2=Gordon/sample.csv" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + ) + + query = ( + f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" + ) + pattern = re.compile( + r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL + ) + + with pytest.raises(Exception, match=pattern): + azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 0}) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 47d8f44c0b7..8071b520a4f 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -3,6 +3,7 @@ import os import pytest import time from helpers.cluster import ClickHouseCluster, is_arm +from helpers.client import QueryRuntimeException from helpers.test_tools import TSV from pyhdfs import HdfsClient @@ -1180,6 +1181,54 @@ def test_respect_object_existence_on_partitioned_write(started_cluster): assert int(result) == 44 +def test_hive_partitioning_with_one_parameter(started_cluster): + hdfs_api = started_cluster.hdfs_api + hdfs_api.write_data(f"/column0=Elizabeth/parquet_1", f"Elizabeth\tGordon\n") + assert hdfs_api.read_data(f"/column0=Elizabeth/parquet_1") == f"Elizabeth\tGordon\n" + + r = node1.query( + "SELECT _column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", + settings={"hdfs_hive_partitioning": 1}, + ) + assert r == f"Elizabeth\n" + + +def test_hive_partitioning_with_two_parameters(started_cluster): + hdfs_api = started_cluster.hdfs_api + hdfs_api.write_data( + f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n" + ) + assert ( + hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") + == f"Elizabeth\tGordon\n" + ) + + r = node1.query( + "SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", + settings={"hdfs_hive_partitioning": 1}, + ) + assert r == f"Gordon\n" + + +def test_hive_partitioning_without_setting(started_cluster): + hdfs_api = started_cluster.hdfs_api + hdfs_api.write_data( + f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n" + ) + assert ( + hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") + == f"Elizabeth\tGordon\n" + ) + pattern = re.compile( + r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL + ) + + with pytest.raises(QueryRuntimeException, match=pattern): + node1.query( + f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", + settings={"hdfs_hive_partitioning": 0}, + ) + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 3fb50aa5d8560f3cd5f4fc9b35bfa47a60d2ca80 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 5 Jul 2024 13:28:47 +0000 Subject: [PATCH 130/644] style fix --- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 1 - tests/integration/test_storage_hdfs/test.py | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 717f48983f3..4b5b514e67d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -81,7 +81,6 @@ StorageObjectStorage::StorageObjectStorage( metadata.setConstraints(constraints_); metadata.setComment(comment); - if (sample_path.empty() && context->getSettings().use_hive_partitioning) sample_path = getPathSample(metadata, context); else if (!context->getSettings().use_hive_partitioning) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 8071b520a4f..da46756841d 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1229,6 +1229,7 @@ def test_hive_partitioning_without_setting(started_cluster): settings={"hdfs_hive_partitioning": 0}, ) + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 6c4c17f119fb95d66f894a92ce8c91fa6664ff5b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 5 Jul 2024 16:44:26 +0200 Subject: [PATCH 131/644] remove use_hive_partitioning from query settings --- src/Storages/ObjectStorage/StorageObjectStorage.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 3fbfc3aacd7..f97d2620fe5 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -43,7 +43,6 @@ public: size_t list_object_keys_size; bool throw_on_zero_files_match; bool ignore_non_existent_file; - bool use_hive_partitioning; }; StorageObjectStorage( From 9064bb1b8389e301f45bc78a5365665292f51c6e Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 5 Jul 2024 19:36:59 +0200 Subject: [PATCH 132/644] fix settings in tests --- .../test_storage_azure_blob_storage/test.py | 12 ++++++------ tests/integration/test_storage_hdfs/test.py | 6 +++--- .../0_stateless/03203_hive_style_partitioning.sh | 12 ++++++------ 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f2a1f9e35a9..6966abfee4f 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -1484,7 +1484,7 @@ def test_hive_partitioning_with_one_parameter(cluster): f"blob_path='{path}', format='CSV', structure='{table_format}')" ) assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} + node, query, settings={"use_hive_partitioning": 1} ).splitlines() == [ "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format( bucket="cont", max_path=path @@ -1497,7 +1497,7 @@ def test_hive_partitioning_with_one_parameter(cluster): f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" ) assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} + node, query, settings={"use_hive_partitioning": 1} ).splitlines() == ["Gordon"] @@ -1521,7 +1521,7 @@ def test_hive_partitioning_with_two_parameters(cluster): f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" ) assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} + node, query, settings={"use_hive_partitioning": 1} ).splitlines() == [ "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth\tGordon".format( bucket="cont", max_path=path @@ -1534,7 +1534,7 @@ def test_hive_partitioning_with_two_parameters(cluster): f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2;" ) assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} + node, query, settings={"use_hive_partitioning": 1} ).splitlines() == ["Elizabeth"] query = ( @@ -1543,7 +1543,7 @@ def test_hive_partitioning_with_two_parameters(cluster): f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2 AND column1=_column1;" ) assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} + node, query, settings={"use_hive_partitioning": 1} ).splitlines() == ["Elizabeth"] @@ -1571,4 +1571,4 @@ def test_hive_partitioning_without_setting(cluster): ) with pytest.raises(Exception, match=pattern): - azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 0}) + azure_query(node, query, settings={"use_hive_partitioning": 0}) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index da46756841d..aa3efb8ba4a 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1188,7 +1188,7 @@ def test_hive_partitioning_with_one_parameter(started_cluster): r = node1.query( "SELECT _column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", - settings={"hdfs_hive_partitioning": 1}, + settings={"use_hive_partitioning": 1}, ) assert r == f"Elizabeth\n" @@ -1205,7 +1205,7 @@ def test_hive_partitioning_with_two_parameters(started_cluster): r = node1.query( "SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", - settings={"hdfs_hive_partitioning": 1}, + settings={"use_hive_partitioning": 1}, ) assert r == f"Gordon\n" @@ -1226,7 +1226,7 @@ def test_hive_partitioning_without_setting(started_cluster): with pytest.raises(QueryRuntimeException, match=pattern): node1.query( f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", - settings={"hdfs_hive_partitioning": 0}, + settings={"use_hive_partitioning": 0}, ) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 98c039f3454..544fd17ffff 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -9,7 +9,7 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" $CLICKHOUSE_LOCAL -n -q """ -set file_hive_partitioning = 1; +set use_hive_partitioning = 1; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -31,7 +31,7 @@ SELECT *, _non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_exi SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" $CLICKHOUSE_LOCAL -n -q """ -set file_hive_partitioning = 0; +set use_hive_partitioning = 0; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" @@ -41,7 +41,7 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE URL PARTITIONING'" $CLICKHOUSE_LOCAL -n -q """ -set url_hive_partitioning = 1; +set use_hive_partitioning = 1; SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -63,7 +63,7 @@ SELECT *, _non_existing_column FROM url('http://localhost:11111/test/partitionin SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" $CLICKHOUSE_LOCAL -n -q """ -set url_hive_partitioning = 0; +set use_hive_partitioning = 0; SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" @@ -73,7 +73,7 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3 PARTITIONING'" $CLICKHOUSE_LOCAL -n -q """ -set s3_hive_partitioning = 1; +set use_hive_partitioning = 1; SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -96,7 +96,7 @@ SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=*/s """ $CLICKHOUSE_LOCAL -n -q """ -set s3_hive_partitioning = 0; +set use_hive_partitioning = 0; SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" From ebb10d7f8fe16e533593178a1778632c00a3c1b7 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 6 Jul 2024 02:12:01 +0000 Subject: [PATCH 133/644] add rebuild option in projection and LWD --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- src/Core/SettingsEnums.cpp | 3 +- src/Core/SettingsEnums.h | 1 + src/Interpreters/InterpreterDeleteQuery.cpp | 56 +++++++++++++++---- ...61_lightweight_delete_projection.reference | 3 + .../03161_lightweight_delete_projection.sql | 27 +++++++++ 7 files changed, 80 insertions(+), 14 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4a343d864db..bd691fe0dee 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -612,7 +612,7 @@ class IColumn; M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) ALIAS(allow_experimental_lightweight_delete) \ M(UInt64, lightweight_deletes_sync, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes", 0) \ - M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete.", 0) \ + M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection of this table then do lightweight delete, or do lightweight delete then rebuild projections.", 0) \ M(Bool, apply_deleted_mask, true, "Enables filtering out rows deleted with lightweight DELETE. If disabled, a query will be able to read those rows. This is useful for debugging and \"undelete\" scenarios", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 6af6b4b15aa..951dd4d74f3 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -61,7 +61,7 @@ static std::initializer_listgetSettingsRef().lightweight_mutation_projection_mode; - if (mode == LightweightMutationProjectionMode::THROW) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "DELETE query is not supported for table {} as it has projections. " - "User should drop all the projections manually before running the query", - table->getStorageID().getFullTableName()); - } - else if (mode == LightweightMutationProjectionMode::DROP) + + auto dropOrClearProjections = [&](bool isDrop) { std::vector all_projections = metadata_snapshot->projections.getAllRegisteredNames(); - context->setSetting("mutations_sync", Field(context->getSettingsRef().lightweight_deletes_sync)); - /// Drop projections first so that lightweight delete can be performed. for (const auto & projection : all_projections) { String alter_query = "ALTER TABLE " + table->getStorageID().getFullTableName() + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) - + " DROP PROJECTION IF EXISTS " + projection; + + (isDrop ? " DROP" : " CLEAR") +" PROJECTION " + projection; ParserAlterQuery parser; ASTPtr alter_ast = parseQuery( @@ -151,6 +143,48 @@ BlockIO InterpreterDeleteQuery::execute() InterpreterAlterQuery alter_interpreter(alter_ast, context); alter_interpreter.execute(); } + + return all_projections; + }; + + if (mode == LightweightMutationProjectionMode::THROW) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "DELETE query is not supported for table {} as it has projections. " + "User should drop all the projections manually before running the query", + table->getStorageID().getFullTableName()); + } + else if (mode == LightweightMutationProjectionMode::DROP) + { + dropOrClearProjections(true); + } + else if (mode == LightweightMutationProjectionMode::REBUILD) + { + std::vector all_projections{dropOrClearProjections(false)}; + BlockIO res = lightweightDelete(); + + for (const auto & projection : all_projections) + { + String alter_query = + "ALTER TABLE " + table->getStorageID().getFullTableName() + + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) + + " MATERIALIZE PROJECTION " + projection; + + ParserAlterQuery parser; + ASTPtr alter_ast = parseQuery( + parser, + alter_query.data(), + alter_query.data() + alter_query.size(), + "ALTER query", + 0, + DBMS_DEFAULT_MAX_PARSER_DEPTH, + DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); + + InterpreterAlterQuery alter_interpreter(alter_ast, context); + alter_interpreter.execute(); + } + + return res; } else { diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index c5a6cbab0bc..307d3cb53fc 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -1,2 +1,5 @@ 1231 John 33 8888 Alice 50 +6666 Ksenia 48 +8888 Alice 50 +p users 3 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index b189388e356..fb32646b46a 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -19,6 +19,8 @@ DELETE FROM users WHERE uid = 8888 SETTINGS lightweight_mutation_projection_mode DELETE FROM users WHERE uid = 6666 SETTINGS lightweight_mutation_projection_mode = 'drop'; +SYSTEM FLUSH LOGS; + -- expecting no projection SELECT name, @@ -29,3 +31,28 @@ WHERE (database = currentDatabase()) AND (`table` = 'users'); SELECT * FROM users ORDER BY uid; DROP TABLE users; + +CREATE TABLE users ( + uid Int16, + name String, + age Int16, + projection p (select * order by age) +) ENGINE = MergeTree order by uid; + +INSERT INTO users VALUES (1231, 'John', 33), (6666, 'Ksenia', 48), (8888, 'Alice', 50); + +DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'rebuild'; + +SELECT * FROM users ORDER BY uid; + +SYSTEM FLUSH LOGS; + +-- expecting projection p with 3 rows is active +SELECT + name, + `table`, + rows, +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND active = 1; + +DROP TABLE users; \ No newline at end of file From 6e5e680797f5b2147455826e4e223c27be5039a6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 6 Jul 2024 05:42:21 +0000 Subject: [PATCH 134/644] bump libprotobuf-mutator, fix build --- contrib/libprotobuf-mutator | 2 +- src/AggregateFunctions/fuzzers/CMakeLists.txt | 2 +- ..._function_state_deserialization_fuzzer.cpp | 24 +------------------ src/Core/fuzzers/CMakeLists.txt | 2 +- src/Core/fuzzers/names_and_types_fuzzer.cpp | 22 ----------------- src/DataTypes/fuzzers/CMakeLists.txt | 2 +- .../data_type_deserialization_fuzzer.cpp | 22 ----------------- src/Formats/fuzzers/CMakeLists.txt | 2 +- src/Formats/fuzzers/format_fuzzer.cpp | 21 ---------------- src/Storages/fuzzers/CMakeLists.txt | 2 +- .../fuzzers/columns_description_fuzzer.cpp | 21 ---------------- 11 files changed, 7 insertions(+), 115 deletions(-) diff --git a/contrib/libprotobuf-mutator b/contrib/libprotobuf-mutator index a304ec48dcf..b922c8ab900 160000 --- a/contrib/libprotobuf-mutator +++ b/contrib/libprotobuf-mutator @@ -1 +1 @@ -Subproject commit a304ec48dcf15d942607032151f7e9ee504b5dcf +Subproject commit b922c8ab9004ef9944982e4f165e2747b13223fa diff --git a/src/AggregateFunctions/fuzzers/CMakeLists.txt b/src/AggregateFunctions/fuzzers/CMakeLists.txt index 907a275b4b3..1ce0c52feb7 100644 --- a/src/AggregateFunctions/fuzzers/CMakeLists.txt +++ b/src/AggregateFunctions/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable(aggregate_function_state_deserialization_fuzzer aggregate_function_state_deserialization_fuzzer.cpp ${SRCS}) -target_link_libraries(aggregate_function_state_deserialization_fuzzer PRIVATE dbms clickhouse_aggregate_functions) +target_link_libraries(aggregate_function_state_deserialization_fuzzer PRIVATE clickhouse_functions clickhouse_aggregate_functions) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index a956d9906bc..31fc93e4288 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -12,33 +12,11 @@ #include -#include - +#include #include #include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - -class IFunctionBase; -using FunctionBasePtr = std::shared_ptr; - -FunctionBasePtr createFunctionBaseCast( - ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for aggregate_function_state_deserialization_fuzzer"); -} - -} - - extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/Core/fuzzers/CMakeLists.txt b/src/Core/fuzzers/CMakeLists.txt index 51db6fa0b53..61d6b9629eb 100644 --- a/src/Core/fuzzers/CMakeLists.txt +++ b/src/Core/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (names_and_types_fuzzer names_and_types_fuzzer.cpp) -target_link_libraries (names_and_types_fuzzer PRIVATE dbms) +target_link_libraries (names_and_types_fuzzer PRIVATE clickhouse_functions) diff --git a/src/Core/fuzzers/names_and_types_fuzzer.cpp b/src/Core/fuzzers/names_and_types_fuzzer.cpp index 74debedf2a3..6fdd8703014 100644 --- a/src/Core/fuzzers/names_and_types_fuzzer.cpp +++ b/src/Core/fuzzers/names_and_types_fuzzer.cpp @@ -1,29 +1,7 @@ -#include -#include #include #include -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - -class IFunctionBase; -using FunctionBasePtr = std::shared_ptr; - -FunctionBasePtr createFunctionBaseCast( - ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for names_and_types_fuzzer"); -} - -} - - extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/DataTypes/fuzzers/CMakeLists.txt b/src/DataTypes/fuzzers/CMakeLists.txt index 939bf5f5e3f..e54ef0a860c 100644 --- a/src/DataTypes/fuzzers/CMakeLists.txt +++ b/src/DataTypes/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable(data_type_deserialization_fuzzer data_type_deserialization_fuzzer.cpp ${SRCS}) -target_link_libraries(data_type_deserialization_fuzzer PRIVATE dbms clickhouse_aggregate_functions) +target_link_libraries(data_type_deserialization_fuzzer PRIVATE clickhouse_functions clickhouse_aggregate_functions) diff --git a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp index 7d9a0513d18..0ae325871fb 100644 --- a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp +++ b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp @@ -8,33 +8,11 @@ #include #include -#include - #include #include -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - -class IFunctionBase; -using FunctionBasePtr = std::shared_ptr; - -FunctionBasePtr createFunctionBaseCast( - ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for data_type_deserialization_fuzzer"); -} - -} - - extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/Formats/fuzzers/CMakeLists.txt b/src/Formats/fuzzers/CMakeLists.txt index 38009aeec1d..b8a7e78b6e2 100644 --- a/src/Formats/fuzzers/CMakeLists.txt +++ b/src/Formats/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable(format_fuzzer format_fuzzer.cpp ${SRCS}) -target_link_libraries(format_fuzzer PRIVATE dbms clickhouse_aggregate_functions) +target_link_libraries(format_fuzzer PRIVATE clickhouse_functions clickhouse_aggregate_functions) diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp index 2c1ec65e54d..27f7d7b292f 100644 --- a/src/Formats/fuzzers/format_fuzzer.cpp +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -3,7 +3,6 @@ #include #include -#include #include #include @@ -21,26 +20,6 @@ #include -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - -class IFunctionBase; -using FunctionBasePtr = std::shared_ptr; - -FunctionBasePtr createFunctionBaseCast( - ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for format_fuzzer"); -} - -} - - extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/Storages/fuzzers/CMakeLists.txt b/src/Storages/fuzzers/CMakeLists.txt index 719b9b77cd9..7bee2da2e26 100644 --- a/src/Storages/fuzzers/CMakeLists.txt +++ b/src/Storages/fuzzers/CMakeLists.txt @@ -4,4 +4,4 @@ clickhouse_add_executable (mergetree_checksum_fuzzer mergetree_checksum_fuzzer.c target_link_libraries (mergetree_checksum_fuzzer PRIVATE dbms) clickhouse_add_executable (columns_description_fuzzer columns_description_fuzzer.cpp) -target_link_libraries (columns_description_fuzzer PRIVATE dbms) +target_link_libraries (columns_description_fuzzer PRIVATE clickhouse_functions) diff --git a/src/Storages/fuzzers/columns_description_fuzzer.cpp b/src/Storages/fuzzers/columns_description_fuzzer.cpp index ac285ea50f7..e10e0cc52f5 100644 --- a/src/Storages/fuzzers/columns_description_fuzzer.cpp +++ b/src/Storages/fuzzers/columns_description_fuzzer.cpp @@ -1,28 +1,7 @@ -#include #include #include -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - -class IFunctionBase; -using FunctionBasePtr = std::shared_ptr; - -FunctionBasePtr createFunctionBaseCast( - ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for columns_description_fuzzer"); -} - -} - - extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try From c13371166c9a9545155c9274d36c3b91a1ade5cb Mon Sep 17 00:00:00 2001 From: yariks5s Date: Sun, 7 Jul 2024 22:25:15 +0000 Subject: [PATCH 135/644] add import re to tests --- tests/integration/test_storage_hdfs/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index da46756841d..9a166cba2ab 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -2,6 +2,7 @@ import os import pytest import time +import re from helpers.cluster import ClickHouseCluster, is_arm from helpers.client import QueryRuntimeException from helpers.test_tools import TSV From a3c4cbfce257f171e66e04598ac9eae548c3836f Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 8 Jul 2024 13:57:54 +0000 Subject: [PATCH 136/644] clang-tidy, fix cp with minio --- docker/test/stateless/setup_minio.sh | 2 +- .../03203_hive_style_partitioning.sh | 52 +++++++++---------- utils/keeper-bench/Runner.cpp | 3 +- 3 files changed, 29 insertions(+), 28 deletions(-) diff --git a/docker/test/stateless/setup_minio.sh b/docker/test/stateless/setup_minio.sh index 2b9433edd20..aacb9d88a45 100755 --- a/docker/test/stateless/setup_minio.sh +++ b/docker/test/stateless/setup_minio.sh @@ -101,7 +101,7 @@ upload_data() { # shellcheck disable=SC2045 for file in $(ls "${data_path}"); do echo "${file}"; - ./mc cp "${data_path}"/"${file}" clickminio/test/"${file}"; + ./mc cp --recursive "${data_path}"/"${file}" clickminio/test/"${file}"; done } diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 544fd17ffff..334bfef4f02 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -43,29 +43,29 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE URL PARTITIONING'" $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _non_existing_column FROM url('http://localhost:11111/test/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" +SELECT *, _non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" @@ -75,28 +75,28 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3 PARTITIONING'" $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _non_existing_column FROM s3('http://localhost:11111/test/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0; +SELECT *, _non_existing_column FROM s3('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = _column0; """ $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index 587e015b340..f8a0e37d1a9 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -544,7 +544,8 @@ struct ZooKeeperRequestFromLogReader file_read_buf = DB::wrapReadBufferWithCompressionMethod(std::move(file_read_buf), compression_method); DB::SingleReadBufferIterator read_buffer_iterator(std::move(file_read_buf)); - auto [columns_description, format] = DB::detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); + std::string sample_path; + auto [columns_description, format] = DB::detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); DB::ColumnsWithTypeAndName columns; columns.reserve(columns_description.size()); From 3cc6a133c64861f4493849905950abb5cc1fbaac Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 9 Jul 2024 12:38:16 +0200 Subject: [PATCH 137/644] Update setup_minio.sh --- docker/test/stateless/setup_minio.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/setup_minio.sh b/docker/test/stateless/setup_minio.sh index aacb9d88a45..0e344cbb9c4 100755 --- a/docker/test/stateless/setup_minio.sh +++ b/docker/test/stateless/setup_minio.sh @@ -101,7 +101,7 @@ upload_data() { # shellcheck disable=SC2045 for file in $(ls "${data_path}"); do echo "${file}"; - ./mc cp --recursive "${data_path}"/"${file}" clickminio/test/"${file}"; + ./mc cp --recursive "${data_path}"/ clickminio/test/; done } @@ -148,4 +148,4 @@ main() { setup_aws_credentials } -main "$@" \ No newline at end of file +main "$@" From 362bf4befcd55de5f49e2665c7c7f9483a700dc8 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 9 Jul 2024 17:30:08 +0200 Subject: [PATCH 138/644] Update setup_minio.sh --- docker/test/stateless/setup_minio.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/setup_minio.sh b/docker/test/stateless/setup_minio.sh index 0e344cbb9c4..8bd75f16321 100755 --- a/docker/test/stateless/setup_minio.sh +++ b/docker/test/stateless/setup_minio.sh @@ -101,7 +101,7 @@ upload_data() { # shellcheck disable=SC2045 for file in $(ls "${data_path}"); do echo "${file}"; - ./mc cp --recursive "${data_path}"/ clickminio/test/; + ./mc cp "${data_path}"/"${file}" clickminio/test/"${file}"; done } From 1761102b3a071143b40039ee1e83666ebffa88fb Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 9 Jul 2024 17:31:41 +0200 Subject: [PATCH 139/644] fix path --- .../column1=Gordon/sample.parquet | Bin .../column1=Schmidt/sample.parquet | Bin .../sample.parquet | Bin 3 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/data_minio/hive_partitioning/{coumn0=Elizabeth => column0=Elizabeth}/column1=Gordon/sample.parquet (100%) rename tests/queries/0_stateless/data_minio/hive_partitioning/{coumn0=Elizabeth => column0=Elizabeth}/column1=Schmidt/sample.parquet (100%) rename tests/queries/0_stateless/data_minio/hive_partitioning/{coumn0=Elizabeth => column0=Elizabeth}/sample.parquet (100%) diff --git a/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Gordon/sample.parquet b/tests/queries/0_stateless/data_minio/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet similarity index 100% rename from tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Gordon/sample.parquet rename to tests/queries/0_stateless/data_minio/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet diff --git a/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Schmidt/sample.parquet b/tests/queries/0_stateless/data_minio/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet similarity index 100% rename from tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Schmidt/sample.parquet rename to tests/queries/0_stateless/data_minio/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet diff --git a/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/sample.parquet b/tests/queries/0_stateless/data_minio/hive_partitioning/column0=Elizabeth/sample.parquet similarity index 100% rename from tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/sample.parquet rename to tests/queries/0_stateless/data_minio/hive_partitioning/column0=Elizabeth/sample.parquet From 9db80a6e2d14c6341c7afc66aeaf6998c98f9f8a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 9 Jul 2024 17:47:05 +0000 Subject: [PATCH 140/644] more testing with chunked --- programs/benchmark/Benchmark.cpp | 4 ++-- src/Client/ConnectionParameters.cpp | 4 ++-- src/Client/ConnectionParameters.h | 4 ++-- src/Dictionaries/ClickHouseDictionarySource.cpp | 8 ++++---- src/Interpreters/Cluster.cpp | 4 ++-- src/Interpreters/Cluster.h | 4 ++-- 6 files changed, 14 insertions(+), 14 deletions(-) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 251761e0bad..0a7faf5ec01 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -666,8 +666,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) Strings hosts = options.count("host") ? options["host"].as() : Strings({"localhost"}); - String proto_send_chunked {"notchunked_optional"}; - String proto_recv_chunked {"notchunked_optional"}; + String proto_send_chunked {"chunked"}; + String proto_recv_chunked {"chunked"}; if (options.count("proto_caps")) { diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index 4bca65083c4..50af589dba3 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -107,8 +107,8 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati } } - proto_send_chunked = config.getString("proto_caps.send", "notchunked_optional"); - proto_recv_chunked = config.getString("proto_caps.recv", "notchunked_optional"); + proto_send_chunked = config.getString("proto_caps.send", "chunked"); + proto_recv_chunked = config.getString("proto_caps.recv", "chunked"); quota_key = config.getString("quota_key", ""); diff --git a/src/Client/ConnectionParameters.h b/src/Client/ConnectionParameters.h index 71057a2b543..ef4df17143e 100644 --- a/src/Client/ConnectionParameters.h +++ b/src/Client/ConnectionParameters.h @@ -20,8 +20,8 @@ struct ConnectionParameters std::string default_database; std::string user; std::string password; - std::string proto_send_chunked = "notchunked_optional"; - std::string proto_recv_chunked = "notchunked_optional"; + std::string proto_send_chunked = "chunked"; + std::string proto_recv_chunked = "chunked"; std::string quota_key; SSHKey ssh_private_key; std::string jwt; diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 3b096da92c6..14c6aac24f6 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -236,8 +236,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .host = host, .user = named_collection->getAnyOrDefault({"user", "username"}, "default"), .password = named_collection->getOrDefault("password", ""), - .proto_send_chunked = named_collection->getOrDefault("proto_send_chunked", "notchunked_optional"), - .proto_recv_chunked = named_collection->getOrDefault("proto_recv_chunked", "notchunked_optional"), + .proto_send_chunked = named_collection->getOrDefault("proto_send_chunked", "chunked"), + .proto_recv_chunked = named_collection->getOrDefault("proto_recv_chunked", "chunked"), .quota_key = named_collection->getOrDefault("quota_key", ""), .db = named_collection->getAnyOrDefault({"db", "database"}, default_database), .table = named_collection->getOrDefault("table", ""), @@ -262,8 +262,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .host = host, .user = config.getString(settings_config_prefix + ".user", "default"), .password = config.getString(settings_config_prefix + ".password", ""), - .proto_send_chunked = config.getString(settings_config_prefix + ".proto_caps.send", "notchunked_optional"), - .proto_recv_chunked = config.getString(settings_config_prefix + ".proto_caps.recv", "notchunked_optional"), + .proto_send_chunked = config.getString(settings_config_prefix + ".proto_caps.send", "chunked"), + .proto_recv_chunked = config.getString(settings_config_prefix + ".proto_caps.recv", "chunked"), .quota_key = config.getString(settings_config_prefix + ".quota_key", ""), .db = config.getString(settings_config_prefix + ".db", default_database), .table = config.getString(settings_config_prefix + ".table", ""), diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 1d7ccd484d0..9b227fcc1fc 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -113,8 +113,8 @@ Cluster::Address::Address( secure = ConfigHelper::getBool(config, config_prefix + ".secure", false, /* empty_as */true) ? Protocol::Secure::Enable : Protocol::Secure::Disable; priority = Priority{config.getInt(config_prefix + ".priority", 1)}; - proto_send_chunked = config.getString(config_prefix + ".proto_caps.send", "notchunked_optional"); - proto_recv_chunked = config.getString(config_prefix + ".proto_caps.recv", "notchunked_optional"); + proto_send_chunked = config.getString(config_prefix + ".proto_caps.send", "chunked"); + proto_recv_chunked = config.getString(config_prefix + ".proto_caps.recv", "chunked"); const char * port_type = secure == Protocol::Secure::Enable ? "tcp_port_secure" : "tcp_port"; auto default_port = config.getInt(port_type, 0); diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index f3146ac0134..009ef15df6c 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -114,8 +114,8 @@ public: UInt16 port{0}; String user; String password; - String proto_send_chunked = "notchunked_optional"; - String proto_recv_chunked = "notchunked_optional"; + String proto_send_chunked = "chunked"; + String proto_recv_chunked = "chunked"; String quota_key; /// For inter-server authorization From d7f08ffdb74b4fce89eff3133e36a5f50fc4ef0b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 9 Jul 2024 21:01:37 +0200 Subject: [PATCH 141/644] Update setup_minio.sh --- docker/test/stateless/setup_minio.sh | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/docker/test/stateless/setup_minio.sh b/docker/test/stateless/setup_minio.sh index 8bd75f16321..49837fdb1ac 100755 --- a/docker/test/stateless/setup_minio.sh +++ b/docker/test/stateless/setup_minio.sh @@ -99,10 +99,7 @@ upload_data() { # iterating over globs will cause redundant file variable to be # a path to a file, not a filename # shellcheck disable=SC2045 - for file in $(ls "${data_path}"); do - echo "${file}"; - ./mc cp "${data_path}"/"${file}" clickminio/test/"${file}"; - done + ./mc cp --recursive "${data_path}"/ clickminio/test/ } setup_aws_credentials() { From 2794b7bf84faf91cfb92d4a8fb76bb3a8183de44 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 10 Jul 2024 00:20:11 +0000 Subject: [PATCH 142/644] defaults to notchunked, add docs to server's and client's configs --- programs/benchmark/Benchmark.cpp | 4 ++-- programs/client/clickhouse-client.xml | 15 +++++++++++++++ programs/server/config.xml | 15 +++++++++++++++ src/Client/ConnectionParameters.cpp | 4 ++-- src/Client/ConnectionParameters.h | 4 ++-- src/Dictionaries/ClickHouseDictionarySource.cpp | 8 ++++---- src/Interpreters/Cluster.cpp | 4 ++-- src/Interpreters/Cluster.h | 4 ++-- src/Server/TCPHandler.cpp | 8 ++++---- 9 files changed, 48 insertions(+), 18 deletions(-) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 0a7faf5ec01..36f774a3c12 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -666,8 +666,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) Strings hosts = options.count("host") ? options["host"].as() : Strings({"localhost"}); - String proto_send_chunked {"chunked"}; - String proto_recv_chunked {"chunked"}; + String proto_send_chunked {"notchunked"}; + String proto_recv_chunked {"notchunked"}; if (options.count("proto_caps")) { diff --git a/programs/client/clickhouse-client.xml b/programs/client/clickhouse-client.xml index d0deb818c1e..376e64906e2 100644 --- a/programs/client/clickhouse-client.xml +++ b/programs/client/clickhouse-client.xml @@ -37,6 +37,21 @@ {display_name} \e[1;31m:)\e[0m + + + 9000 + + + diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index 50af589dba3..4d0a9ffa08c 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -107,8 +107,8 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati } } - proto_send_chunked = config.getString("proto_caps.send", "chunked"); - proto_recv_chunked = config.getString("proto_caps.recv", "chunked"); + proto_send_chunked = config.getString("proto_caps.send", "notchunked"); + proto_recv_chunked = config.getString("proto_caps.recv", "notchunked"); quota_key = config.getString("quota_key", ""); diff --git a/src/Client/ConnectionParameters.h b/src/Client/ConnectionParameters.h index ef4df17143e..382bfe34a3d 100644 --- a/src/Client/ConnectionParameters.h +++ b/src/Client/ConnectionParameters.h @@ -20,8 +20,8 @@ struct ConnectionParameters std::string default_database; std::string user; std::string password; - std::string proto_send_chunked = "chunked"; - std::string proto_recv_chunked = "chunked"; + std::string proto_send_chunked = "notchunked"; + std::string proto_recv_chunked = "notchunked"; std::string quota_key; SSHKey ssh_private_key; std::string jwt; diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 14c6aac24f6..b36d53a6159 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -236,8 +236,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .host = host, .user = named_collection->getAnyOrDefault({"user", "username"}, "default"), .password = named_collection->getOrDefault("password", ""), - .proto_send_chunked = named_collection->getOrDefault("proto_send_chunked", "chunked"), - .proto_recv_chunked = named_collection->getOrDefault("proto_recv_chunked", "chunked"), + .proto_send_chunked = named_collection->getOrDefault("proto_send_chunked", "notchunked"), + .proto_recv_chunked = named_collection->getOrDefault("proto_recv_chunked", "notchunked"), .quota_key = named_collection->getOrDefault("quota_key", ""), .db = named_collection->getAnyOrDefault({"db", "database"}, default_database), .table = named_collection->getOrDefault("table", ""), @@ -262,8 +262,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .host = host, .user = config.getString(settings_config_prefix + ".user", "default"), .password = config.getString(settings_config_prefix + ".password", ""), - .proto_send_chunked = config.getString(settings_config_prefix + ".proto_caps.send", "chunked"), - .proto_recv_chunked = config.getString(settings_config_prefix + ".proto_caps.recv", "chunked"), + .proto_send_chunked = config.getString(settings_config_prefix + ".proto_caps.send", "notchunked"), + .proto_recv_chunked = config.getString(settings_config_prefix + ".proto_caps.recv", "notchunked"), .quota_key = config.getString(settings_config_prefix + ".quota_key", ""), .db = config.getString(settings_config_prefix + ".db", default_database), .table = config.getString(settings_config_prefix + ".table", ""), diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 9b227fcc1fc..dd9e35834eb 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -113,8 +113,8 @@ Cluster::Address::Address( secure = ConfigHelper::getBool(config, config_prefix + ".secure", false, /* empty_as */true) ? Protocol::Secure::Enable : Protocol::Secure::Disable; priority = Priority{config.getInt(config_prefix + ".priority", 1)}; - proto_send_chunked = config.getString(config_prefix + ".proto_caps.send", "chunked"); - proto_recv_chunked = config.getString(config_prefix + ".proto_caps.recv", "chunked"); + proto_send_chunked = config.getString(config_prefix + ".proto_caps.send", "notchunked"); + proto_recv_chunked = config.getString(config_prefix + ".proto_caps.recv", "notchunked"); const char * port_type = secure == Protocol::Secure::Enable ? "tcp_port_secure" : "tcp_port"; auto default_port = config.getInt(port_type, 0); diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index 009ef15df6c..c69d77668ab 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -114,8 +114,8 @@ public: UInt16 port{0}; String user; String password; - String proto_send_chunked = "chunked"; - String proto_recv_chunked = "chunked"; + String proto_send_chunked = "notchunked"; + String proto_recv_chunked = "notchunked"; String quota_key; /// For inter-server authorization diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 40fd3848455..9c5e5e9c572 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -310,8 +310,8 @@ void TCPHandler::runImpl() return chunked_srv; }; - bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "chunked_optional"), proto_recv_chunked_cl, "send"); - bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "chunked_optional"), proto_send_chunked_cl, "recv"); + bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "notchunked"), proto_recv_chunked_cl, "send"); + bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "notchunked"), proto_send_chunked_cl, "recv"); if (out_chunked) out->enableChunked(); @@ -1660,8 +1660,8 @@ void TCPHandler::sendHello() writeVarUInt(VERSION_PATCH, *out); if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) { - writeStringBinary(server.config().getString("proto_caps.send", "chunked"), *out); - writeStringBinary(server.config().getString("proto_caps.recv", "chunked"), *out); + writeStringBinary(server.config().getString("proto_caps.send", "notchunked"), *out); + writeStringBinary(server.config().getString("proto_caps.recv", "notchunked"), *out); } if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES) { From 6cd6319ba70945e7ae50447772c57d61e488e72e Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Jul 2024 15:32:28 +0000 Subject: [PATCH 143/644] Properly convert boolean literals in query tree --- src/Interpreters/convertFieldToType.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 184c263dbdb..d87d4a73e37 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -214,6 +214,10 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID } else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String) { + /// Bool is not represented in which_type, so we need to type it separately + if (isInt64OrUInt64orBoolFieldType(src.getType()) && type.getName() == "Bool") + return bool(src.safeGet()); + if (which_type.isUInt8()) return convertNumericType(src, type); if (which_type.isUInt16()) return convertNumericType(src, type); if (which_type.isUInt32()) return convertNumericType(src, type); From 294eaaeabd60140d4f4f6ae892b015d6fe4f551c Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Jul 2024 15:43:06 +0000 Subject: [PATCH 144/644] Test bool in gtest_transform_query_for_external_database --- .../gtest_transform_query_for_external_database.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 6765e112bb9..5a63c118e2d 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -74,6 +75,7 @@ private: {"a", std::make_shared()}, {"b", std::make_shared()}, {"foo", std::make_shared()}, + {"is_value", DataTypeFactory::instance().get("Bool")}, }), TableWithColumnNamesAndTypes( createDBAndTable("table2"), @@ -411,6 +413,14 @@ TEST(TransformQueryForExternalDatabase, Analyzer) R"(SELECT "column" FROM "test"."table")"); check(state, 1, {"column", "apply_id", "apply_type", "apply_status", "create_time", "field", "value", "a", "b", "foo"}, - "SELECT * FROM table WHERE (column) IN (1)", + "SELECT * EXCEPT (is_value) FROM table WHERE (column) IN (1)", R"(SELECT "column", "apply_id", "apply_type", "apply_status", "create_time", "field", "value", "a", "b", "foo" FROM "test"."table" WHERE "column" IN (1))"); + + check(state, 1, {"is_value"}, + "SELECT is_value FROM table WHERE is_value = true", + R"(SELECT "is_value" FROM "test"."table" WHERE "is_value" = true)"); + + check(state, 1, {"is_value"}, + "SELECT is_value FROM table WHERE is_value = 1", + R"(SELECT "is_value" FROM "test"."table" WHERE "is_value" = 1)"); } From 46fbc23e093a85f431ac8afdc33f52fe267506c0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Jul 2024 17:11:21 +0000 Subject: [PATCH 145/644] update 02952_conjunction_optimization.reference --- .../0_stateless/02952_conjunction_optimization.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02952_conjunction_optimization.reference b/tests/queries/0_stateless/02952_conjunction_optimization.reference index eeadfaae21d..8af0abefd3a 100644 --- a/tests/queries/0_stateless/02952_conjunction_optimization.reference +++ b/tests/queries/0_stateless/02952_conjunction_optimization.reference @@ -32,7 +32,7 @@ QUERY id: 0 FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: Bool ARGUMENTS LIST id: 6, nodes: 2 - CONSTANT id: 7, constant_value: UInt64_1, constant_value_type: Bool + CONSTANT id: 7, constant_value: Bool_1, constant_value_type: Bool FUNCTION id: 8, function_name: notIn, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 9, nodes: 2 From b8944abe0ec73ca386e851ce96d5f7ddaf3d254e Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Wed, 10 Jul 2024 19:01:52 +0800 Subject: [PATCH 146/644] refactor: avoid unneed calculation in SeriesPeriodDetect Signed-off-by: Ruihang Xia --- src/Functions/seriesPeriodDetectFFT.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Functions/seriesPeriodDetectFFT.cpp b/src/Functions/seriesPeriodDetectFFT.cpp index 471354235d5..ecf8398bbd5 100644 --- a/src/Functions/seriesPeriodDetectFFT.cpp +++ b/src/Functions/seriesPeriodDetectFFT.cpp @@ -153,12 +153,8 @@ public: return true; } - std::vector xfreq(spec_len); double step = 0.5 / (spec_len - 1); - for (size_t i = 0; i < spec_len; ++i) - xfreq[i] = i * step; - - auto freq = xfreq[idx]; + auto freq = idx * step; period = std::round(1 / freq); return true; From 7b58722c07e8feb6acca5f0762411a55b8c58915 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 10 Jul 2024 13:15:44 +0200 Subject: [PATCH 147/644] Update setup_minio.sh --- docker/test/stateless/setup_minio.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/setup_minio.sh b/docker/test/stateless/setup_minio.sh index 49837fdb1ac..02e3d117de2 100755 --- a/docker/test/stateless/setup_minio.sh +++ b/docker/test/stateless/setup_minio.sh @@ -99,7 +99,9 @@ upload_data() { # iterating over globs will cause redundant file variable to be # a path to a file, not a filename # shellcheck disable=SC2045 - ./mc cp --recursive "${data_path}"/ clickminio/test/ + if [ -d "${data_path}" ]; then + ./mc cp --recursive "${data_path}"/ clickminio/test/ + fi } setup_aws_credentials() { From a751719a33e2691426bdb057eaf509a74e84753d Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 10 Jul 2024 17:44:06 +0000 Subject: [PATCH 148/644] fixes due to review --- docs/en/operations/settings/settings.md | 6 +++++ docs/en/sql-reference/table-functions/file.md | 17 ++++++++++++ docs/en/sql-reference/table-functions/hdfs.md | 17 ++++++++++++ docs/en/sql-reference/table-functions/s3.md | 17 ++++++++++++ programs/obfuscator/Obfuscator.cpp | 3 +-- src/Formats/ReadSchemaUtils.cpp | 11 ++------ src/Formats/ReadSchemaUtils.h | 2 -- src/Storages/Hive/StorageHive.cpp | 2 +- .../DataLakes/IStorageDataLake.h | 4 +-- .../ObjectStorage/StorageObjectStorage.cpp | 14 ++++++---- .../StorageObjectStorageCluster.cpp | 2 +- .../StorageObjectStorageSource.cpp | 11 +++++--- .../StorageObjectStorageSource.h | 3 ++- .../StorageObjectStorageQueue.cpp | 2 +- src/Storages/StorageFile.cpp | 20 +++++++------- src/Storages/StorageFileCluster.cpp | 2 +- src/Storages/StorageURL.cpp | 23 +++++++++++----- src/Storages/StorageURLCluster.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 25 +++++++++++------- src/Storages/VirtualColumnUtils.h | 9 +++++-- src/TableFunctions/TableFunctionFormat.cpp | 10 +++---- .../03203_hive_style_partitioning.reference | 5 +++- .../03203_hive_style_partitioning.sh | 11 +++++--- .../array=[1,2,3]/float=42.42/sample.parquet | Bin 0 -> 1308 bytes .../number=42/date=2020-01-01/sample.parquet | Bin 0 -> 1308 bytes 25 files changed, 152 insertions(+), 66 deletions(-) create mode 100644 tests/queries/0_stateless/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet create mode 100644 tests/queries/0_stateless/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 1d74a63b972..e100e0f27f7 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5591,3 +5591,9 @@ Default value: `10000000`. Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached. Default value: `1GiB`. + +## use_hive_partitioning + +Allows the usage of Hive-style partitioning in queries. When enabled, ClickHouse interprets and maintains table partitions in a way that is consistent with the Hive partitioning scheme, which is commonly used in Hadoop ecosystems. + +Default value: `0`. diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 3a3162dad9a..88af3663552 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -198,6 +198,23 @@ SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt3 - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. +## Hive-style partitioning {#hive-style-patitioning} + +When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. + +**Example** + +Use virtual column, created with Hive-style partitioning + +``` sql +SET use_hive_patitioning = 1; +SELECT _specified_column from file('/specified_column=specified_data/file.txt'); +``` + +``` reference +specified_data +``` + ## Settings {#settings} - [engine_file_empty_if_not_exists](/docs/en/operations/settings/settings.md#engine-file-empty_if-not-exists) - allows to select empty data from a file that doesn't exist. Disabled by default. diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 28cba5ccc6a..beb1ad12532 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -99,6 +99,23 @@ FROM hdfs('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name Strin - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. +## Hive-style partitioning {#hive-style-patitioning} + +When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. + +**Example** + +Use virtual column, created with Hive-style partitioning + +``` sql +SET use_hive_patitioning = 1; +SELECT _specified_column from HDFS('hdfs://hdfs1:9000/specified_column=specified_data/file.txt'); +``` + +``` reference +specified_data +``` + ## Storage Settings {#storage-settings} - [hdfs_truncate_on_insert](/docs/en/operations/settings/settings.md#hdfs_truncate_on_insert) - allows to truncate file before insert into it. Disabled by default. diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 35e5d86034c..45c4caa1a13 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -274,6 +274,23 @@ FROM s3( - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. In case of archive shows uncompressed file size of the file inside the archive. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. +## Hive-style partitioning {#hive-style-patitioning} + +When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. + +**Example** + +Use virtual column, created with Hive-style partitioning + +``` sql +SET use_hive_patitioning = 1; +SELECT _specified_column from HDFS('hdfs://hdfs1:9000/specified_column=specified_data/file.txt'); +``` + +``` reference +specified_data +``` + ## Storage Settings {#storage-settings} - [s3_truncate_on_insert](/docs/en/operations/settings/settings.md#s3_truncate_on_insert) - allows to truncate file before insert into it. Disabled by default. diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 11e85bc1302..4c3981c1d01 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1308,8 +1308,7 @@ try SingleReadBufferIterator read_buffer_iterator(std::move(file)); - std::string sample_string; - schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, sample_string, context_const); + schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, context_const); } else { diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 1e70840f91f..1920459c378 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -94,7 +94,6 @@ std::pair readSchemaFromFormatImpl( std::optional format_name, const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, - std::string & sample_path, const ContextPtr & context) try { @@ -144,10 +143,6 @@ try { iterator_data = read_buffer_iterator.next(); - /// Extracting the File path for hive-style partitioning - if (sample_path.empty()) - sample_path = read_buffer_iterator.getLastFilePath(); - /// Read buffer iterator can determine the data format if it's unknown. /// For example by scanning schema cache or by finding new file with format extension. if (!format_name && iterator_data.format_name) @@ -541,19 +536,17 @@ ColumnsDescription readSchemaFromFormat( const String & format_name, const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, - std::string & sample_path, const ContextPtr & context) { - return readSchemaFromFormatImpl(format_name, format_settings, read_buffer_iterator, sample_path, context).first; + return readSchemaFromFormatImpl(format_name, format_settings, read_buffer_iterator, context).first; } std::pair detectFormatAndReadSchema( const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, - std::string & sample_path, const ContextPtr & context) { - return readSchemaFromFormatImpl(std::nullopt, format_settings, read_buffer_iterator, sample_path, context); + return readSchemaFromFormatImpl(std::nullopt, format_settings, read_buffer_iterator, context); } SchemaCache::Key getKeyForSchemaCache( diff --git a/src/Formats/ReadSchemaUtils.h b/src/Formats/ReadSchemaUtils.h index 6c562a06bf0..7168e7f0817 100644 --- a/src/Formats/ReadSchemaUtils.h +++ b/src/Formats/ReadSchemaUtils.h @@ -122,7 +122,6 @@ ColumnsDescription readSchemaFromFormat( const String & format_name, const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, - std::string & sample_path, const ContextPtr & context); /// Try to detect the format of the data and it's schema. @@ -132,7 +131,6 @@ ColumnsDescription readSchemaFromFormat( std::pair detectFormatAndReadSchema( const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, - std::string & sample_path, const ContextPtr & context); SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional & format_settings, const ContextPtr & context); diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 28d8128e052..255dadea387 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -445,7 +445,7 @@ StorageHive::StorageHive( storage_metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, storage_metadata.columns, getContext()); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), getContext())); } void StorageHive::lazyInitialize() diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index a651be6017f..ec8e740b1c9 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -89,9 +89,9 @@ public: { ConfigurationPtr configuration = base_configuration->clone(); configuration->setPaths(metadata->getDataFiles()); - std::string sample_string; + std::string sample_path; return Storage::resolveSchemaFromData( - object_storage_, configuration, format_settings_, sample_string, local_context); + object_storage_, configuration, format_settings_, sample_path, local_context); } } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 8ab8b6b6881..48e9118e321 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -43,7 +43,8 @@ std::string StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata {}, // predicate metadata.getColumns().getAll(), // virtual_columns nullptr, // read_keys - {} // file_progress_callback + {}, // file_progress_callback + true // override_settings_for_hive_partitioning ); if (auto file = file_iterator->next(0)) @@ -86,7 +87,7 @@ StorageObjectStorage::StorageObjectStorage( else if (!context->getSettings().use_hive_partitioning) sample_path = ""; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), sample_path)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context, sample_path)); setInMemoryMetadata(metadata); } @@ -396,7 +397,8 @@ ColumnsDescription StorageObjectStorage::resolveSchemaFromData( { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); - return readSchemaFromFormat(configuration->format, format_settings, *iterator, sample_path, context); + sample_path = iterator->getLastFilePath(); + return readSchemaFromFormat(configuration->format, format_settings, *iterator, context); } std::string StorageObjectStorage::resolveFormatFromData( @@ -408,7 +410,8 @@ std::string StorageObjectStorage::resolveFormatFromData( { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); - return detectFormatAndReadSchema(format_settings, *iterator, sample_path, context).second; + sample_path = iterator->getLastFilePath(); + return detectFormatAndReadSchema(format_settings, *iterator, context).second; } std::pair StorageObjectStorage::resolveSchemaAndFormatFromData( @@ -420,7 +423,8 @@ std::pair StorageObjectStorage::resolveSchemaAn { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); - auto [columns, format] = detectFormatAndReadSchema(format_settings, *iterator, sample_path, context); + sample_path = iterator->getLastFilePath(); + auto [columns, format] = detectFormatAndReadSchema(format_settings, *iterator, context); configuration->format = format; return std::pair(columns, format); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 0dc4b845a47..92327b4cde0 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -41,7 +41,7 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( metadata.setColumns(columns); metadata.setConstraints(constraints_); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns())); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context_)); setInMemoryMetadata(metadata); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 88ae0a2319c..e5c9318de5d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -105,7 +105,8 @@ std::shared_ptr StorageObjectStorageSourc const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, - std::function file_progress_callback) + std::function file_progress_callback, + bool override_settings_for_hive_partitioning) { if (distributed_processing) return std::make_shared( @@ -122,11 +123,14 @@ std::shared_ptr StorageObjectStorageSourc std::unique_ptr iterator; if (configuration->isPathWithGlobs()) { + bool throw_on_zero_files_match = settings.throw_on_zero_files_match; + if (override_settings_for_hive_partitioning) + throw_on_zero_files_match = false; /// Iterate through disclosed globs and make a source for each file iterator = std::make_unique( object_storage, configuration, predicate, virtual_columns, local_context, is_archive ? nullptr : read_keys, settings.list_object_keys_size, - settings.throw_on_zero_files_match, file_progress_callback); + throw_on_zero_files_match, file_progress_callback); } else { @@ -204,7 +208,8 @@ Chunk StorageObjectStorageSource::generate() .size = object_info->isArchive() ? object_info->fileSizeInArchive() : object_info->metadata->size_bytes, .filename = &filename, .last_modified = object_info->metadata->last_modified, - }, object_info->getPath()); + .hive_partitioning_path = object_info->getPath(), + }); const auto & partition_columns = configuration->getPartitionColumns(); if (!partition_columns.empty() && chunk_size && chunk.hasColumns()) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 271b38fa75c..a99bb068372 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -58,7 +58,8 @@ public: const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, - std::function file_progress_callback = {}); + std::function file_progress_callback = {}, + bool override_settings_for_hive_partitioning = false); static std::string getUniqueStoragePathIdentifier( const Configuration & configuration, diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index 98f8cdc7e7a..a9239e3ad06 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -168,7 +168,7 @@ StorageObjectStorageQueue::StorageObjectStorageQueue( storage_metadata.setColumns(columns); storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_)); setInMemoryMetadata(storage_metadata); LOG_INFO(log, "Using zookeeper path: {}", zk_path.string()); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 9751d596fff..e6b9137444e 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -52,6 +52,7 @@ #include #include #include +#include "Formats/FormatSettings.h" #include #include @@ -880,11 +881,10 @@ std::pair StorageFile::getTableStructureAndFormatFro auto read_buffer_iterator = SingleReadBufferIterator(std::move(read_buf)); ColumnsDescription columns; - std::string sample_path; if (format) - columns = readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context); + columns = readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context); else - std::tie(columns, format) = detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); + std::tie(columns, format) = detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); peekable_read_buffer_from_fd = read_buffer_iterator.releaseBuffer(); if (peekable_read_buffer_from_fd) @@ -929,21 +929,20 @@ std::pair StorageFile::getTableStructureAndFormatFro } - std::string sample_path; if (archive_info) { ReadBufferFromArchiveIterator read_buffer_iterator(*archive_info, format, format_settings, context); if (format) - return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context), *format}; + return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context), *format}; - return detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); + return detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); } ReadBufferFromFileIterator read_buffer_iterator(paths, format, compression_method, format_settings, context); if (format) - return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context), *format}; + return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context), *format}; - return detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); + return detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); } ColumnsDescription StorageFile::getTableStructureFromFile( @@ -1102,7 +1101,7 @@ void StorageFile::setStorageMetadata(CommonArguments args) std::string path_for_virtuals; if (args.getContext()->getSettingsRef().use_hive_partitioning && !paths.empty()) path_for_virtuals = paths[0]; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), path_for_virtuals, format_settings.value_or(FormatSettings{}))); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), args.getContext(), path_for_virtuals, format_settings.value_or(FormatSettings{}))); } @@ -1456,7 +1455,8 @@ Chunk StorageFileSource::generate() .size = current_file_size, .filename = (filename_override.has_value() ? &filename_override.value() : nullptr), .last_modified = current_file_last_modified, - }, hive_partitioning_path); + .hive_partitioning_path = hive_partitioning_path, + }); return chunk; } diff --git a/src/Storages/StorageFileCluster.cpp b/src/Storages/StorageFileCluster.cpp index d43e242f70c..f7684182e79 100644 --- a/src/Storages/StorageFileCluster.cpp +++ b/src/Storages/StorageFileCluster.cpp @@ -61,7 +61,7 @@ StorageFileCluster::StorageFileCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context)); } void StorageFileCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 59c5465a381..5da42638b87 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -99,6 +99,17 @@ static ConnectionTimeouts getHTTPTimeouts(ContextPtr context) return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings().keep_alive_timeout); } +String getSampleURI(String uri, ContextPtr context) +{ + if (urlWithGlobs(uri)) + { + auto uris = parseRemoteDescription(uri, 0, uri.size(), ',', context->getSettingsRef().glob_expansion_max_elements); + if (!uris.empty()) + return uris[0]; + } + return uri; +} + IStorageURLBase::IStorageURLBase( const String & uri_, const ContextPtr & context_, @@ -155,8 +166,8 @@ IStorageURLBase::IStorageURLBase( std::string uri_for_partitioning; if (context_->getSettingsRef().use_hive_partitioning) - uri_for_partitioning = uri; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), uri_for_partitioning, format_settings.value_or(FormatSettings{}))); + uri_for_partitioning = getSampleURI(uri, context_); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_, uri_for_partitioning, format_settings.value_or(FormatSettings{}))); } @@ -425,7 +436,8 @@ Chunk StorageURLSource::generate() { .path = curr_uri.getPath(), .size = current_file_size, - }, hive_partitioning_path); + .hive_partitioning_path = hive_partitioning_path, + }); return chunk; } @@ -959,10 +971,9 @@ std::pair IStorageURLBase::getTableStructureAndForma urls_to_check = {uri}; ReadBufferIterator read_buffer_iterator(urls_to_check, format, compression_method, headers, format_settings, context); - std::string sample_path; if (format) - return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context), *format}; - return detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); + return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context), *format}; + return detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); } ColumnsDescription IStorageURLBase::getTableStructureFromData( diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 2e7c63d0097..592bd71f546 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -75,7 +75,7 @@ StorageURLCluster::StorageURLCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context)); } void StorageURLCluster::updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 379b14d8e51..b7669c65992 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -39,10 +39,13 @@ #include #include #include +#include +#include #include "Functions/FunctionsLogical.h" #include "Functions/IFunction.h" #include "Functions/IFunctionAdaptors.h" #include "Functions/indexHint.h" +#include #include #include #include @@ -116,7 +119,7 @@ NameSet getVirtualNamesForFileLikeStorage() return {"_path", "_file", "_size", "_time"}; } -std::map parseFromPath(const std::string& path) +std::map parseHivePartitioningKeysAndValues(const std::string& path) { std::string pattern = "/([^/]+)=([^/]+)"; re2::StringPiece input_piece(path); @@ -128,7 +131,7 @@ std::map parseFromPath(const std::string& path) return key_values; } -VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, std::string path, FormatSettings settings) +VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, const ContextPtr & context, std::string path, std::optional format_settings_) { VirtualColumnsDescription desc; @@ -145,13 +148,17 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription add_virtual("_size", makeNullable(std::make_shared())); add_virtual("_time", makeNullable(std::make_shared())); - auto map = parseFromPath(path); - for (const auto& item : map) + auto map = parseHivePartitioningKeysAndValues(path); + for (auto& item : map) { - auto type = tryInferDataTypeForSingleField(item.second, settings); + auto format_settings = format_settings_ ? *format_settings_ : getFormatSettings(context); + auto type = tryInferDataTypeByEscapingRule(item.second, format_settings, FormatSettings::EscapingRule::Raw); if (type == nullptr) type = std::make_shared(); - add_virtual(item.first, std::make_shared(type)); + if (type->canBeInsideLowCardinality()) + add_virtual(item.first, std::make_shared(type)); + else + add_virtual(item.first, type); } return desc; @@ -215,9 +222,9 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values, const std::string & hive_partitioning_path) + VirtualsForFileLikeStorage virtual_values) { - auto hive_map = parseFromPath(hive_partitioning_path); + auto hive_map = parseHivePartitioningKeysAndValues(virtual_values.hive_partitioning_path); for (const auto & virtual_column : requested_virtual_columns) { if (virtual_column.name == "_path") @@ -265,7 +272,7 @@ void addRequestedFileLikeStorageVirtualsToChunk( auto it = hive_map.find(virtual_column.getNameInStorage()); if (it != hive_map.end()) { - chunk.addColumn(virtual_column.getTypeInStorage()->createColumnConst(chunk.getNumRows(), it->second)->convertToFullColumnIfConst()); + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), convertFieldToType(Field(it->second), *virtual_column.type))->convertToFullColumnIfConst()); hive_map.erase(it); } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 72922be60bd..594253a32c1 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -50,7 +50,11 @@ auto extractSingleValueFromBlock(const Block & block, const String & name) } NameSet getVirtualNamesForFileLikeStorage(); -VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, std::string path = "", FormatSettings settings = FormatSettings()); +VirtualColumnsDescription getVirtualsForFileLikeStorage( + const ColumnsDescription & storage_columns, + const ContextPtr & context, + std::string sample_path = "", + std::optional format_settings_ = std::nullopt); ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); @@ -77,13 +81,14 @@ struct VirtualsForFileLikeStorage std::optional size { std::nullopt }; const String * filename { nullptr }; std::optional last_modified { std::nullopt }; + const String & hive_partitioning_path = ""; }; std::map parseFromPath(const std::string& path); void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values, const std::string & hive_partitioning_path = ""); + VirtualsForFileLikeStorage virtual_values); } } diff --git a/src/TableFunctions/TableFunctionFormat.cpp b/src/TableFunctions/TableFunctionFormat.cpp index 66152cb0c91..ad2a142a140 100644 --- a/src/TableFunctions/TableFunctionFormat.cpp +++ b/src/TableFunctions/TableFunctionFormat.cpp @@ -85,10 +85,9 @@ ColumnsDescription TableFunctionFormat::getActualTableStructure(ContextPtr conte if (structure == "auto") { SingleReadBufferIterator read_buffer_iterator(std::make_unique(data)); - std::string sample_path; if (format == "auto") - return detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, sample_path, context).first; - return readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, sample_path, context); + return detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, context).first; + return readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, context); } return parseColumnsListFromString(structure, context); } @@ -132,12 +131,11 @@ StoragePtr TableFunctionFormat::executeImpl(const ASTPtr & /*ast_function*/, Con String format_name = format; if (structure == "auto") { - std::string sample_path; SingleReadBufferIterator read_buffer_iterator(std::make_unique(data)); if (format_name == "auto") - std::tie(columns, format_name) = detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, sample_path, context); + std::tie(columns, format_name) = detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, context); else - columns = readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, sample_path, context); + columns = readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, context); } else { diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index 6ef1fcdf652..e0f46caf1c8 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -60,7 +60,10 @@ Stanley Gibson Elizabeth Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth -Elizabeth Gordon Elizabeth +42 2020-01-01 +[1,2,3] 42.42 +Array(Int64) LowCardinality(Float64) +101 1 TESTING THE S3 PARTITIONING first last Elizabeth diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 334bfef4f02..9d805b39b8a 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -28,7 +28,13 @@ SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=E SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; SELECT *, _non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = _column0; + +SELECT _number, _date FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') LIMIT 1; +SELECT _array, _float FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1; +SELECT toTypeName(_array), toTypeName(_float) FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1; +SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') WHERE _number = 42; +""" $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; @@ -59,8 +65,7 @@ SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/colum SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" +SELECT *, _non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;""" $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; diff --git a/tests/queries/0_stateless/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 From 1f33eb32b0c80b9dde27a8d7aa9ad26c271aceae Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 11 Jul 2024 03:02:15 +0000 Subject: [PATCH 149/644] try to drop projection correctly --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- src/Core/SettingsEnums.cpp | 3 +- src/Core/SettingsEnums.h | 3 +- src/Interpreters/InterpreterDeleteQuery.cpp | 61 +------------------ src/Interpreters/MutationsInterpreter.cpp | 6 +- src/Storages/MergeTree/MutateTask.cpp | 7 ++- ...61_lightweight_delete_projection.reference | 5 -- .../03161_lightweight_delete_projection.sql | 33 +--------- 9 files changed, 18 insertions(+), 104 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c884f8f80c4..f7b44ea775c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -588,7 +588,7 @@ class IColumn; M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) ALIAS(allow_experimental_lightweight_delete) \ M(UInt64, lightweight_deletes_sync, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes", 0) \ - M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection of this table then do lightweight delete, or do lightweight delete then rebuild projections.", 0) \ + M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop projections of this table's relevant parts.", 0) \ M(Bool, apply_deleted_mask, true, "Enables filtering out rows deleted with lightweight DELETE. If disabled, a query will be able to read those rows. This is useful for debugging and \"undelete\" scenarios", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 5174cf82c2e..194292a467e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -70,7 +70,7 @@ static std::initializer_listgetSettingsRef().lightweight_mutation_projection_mode; - auto dropOrClearProjections = [&](bool isDrop) - { - std::vector all_projections = metadata_snapshot->projections.getAllRegisteredNames(); - - /// Drop projections first so that lightweight delete can be performed. - for (const auto & projection : all_projections) - { - String alter_query = - "ALTER TABLE " + table->getStorageID().getFullTableName() - + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) - + (isDrop ? " DROP" : " CLEAR") +" PROJECTION " + projection; - - ParserAlterQuery parser; - ASTPtr alter_ast = parseQuery( - parser, - alter_query.data(), - alter_query.data() + alter_query.size(), - "ALTER query", - 0, - DBMS_DEFAULT_MAX_PARSER_DEPTH, - DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); - - InterpreterAlterQuery alter_interpreter(alter_ast, context); - alter_interpreter.execute(); - } - - return all_projections; - }; - if (mode == LightweightMutationProjectionMode::THROW) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, @@ -156,43 +127,13 @@ BlockIO InterpreterDeleteQuery::execute() } else if (mode == LightweightMutationProjectionMode::DROP) { - dropOrClearProjections(true); - } - else if (mode == LightweightMutationProjectionMode::REBUILD) - { - std::vector all_projections{dropOrClearProjections(false)}; - BlockIO res = lightweightDelete(); - - for (const auto & projection : all_projections) - { - String alter_query = - "ALTER TABLE " + table->getStorageID().getFullTableName() - + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) - + " MATERIALIZE PROJECTION " + projection; - - ParserAlterQuery parser; - ASTPtr alter_ast = parseQuery( - parser, - alter_query.data(), - alter_query.data() + alter_query.size(), - "ALTER query", - 0, - DBMS_DEFAULT_MAX_PARSER_DEPTH, - DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); - - InterpreterAlterQuery alter_interpreter(alter_ast, context); - alter_interpreter.execute(); - } - - return res; + return lightweightDelete(); } else { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unrecognized lightweight_mutation_projection_mode, only throw and drop are allowed."); } - - return lightweightDelete(); } throw Exception(ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 6d3a4f30b34..ace285bcfc9 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -487,7 +487,11 @@ static void validateUpdateColumns( if (column_name == RowExistsColumn::name) { if (!source.supportsLightweightDelete()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); + { + // if (!source.getStorage()->isMergeTree() + // || context->getSettingsRef().lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW) + // throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); + } } else if (virtual_columns.tryGet(column_name)) { diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a552ee89aee..8ca987eb1f8 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1042,6 +1042,8 @@ struct MutationContext /// Whether we need to count lightweight delete rows in this mutation bool count_lightweight_deleted_rows; + + bool lightweight_mutation_mode; }; using MutationContextPtr = std::shared_ptr; @@ -1571,7 +1573,7 @@ private: } else { - if (ctx->source_part->checksums.has(projection.getDirectoryName())) + if (!ctx->lightweight_mutation_mode && ctx->source_part->checksums.has(projection.getDirectoryName())) entries_to_hardlink.insert(projection.getDirectoryName()); } } @@ -2255,7 +2257,8 @@ bool MutateTask::prepare() if (ctx->mutating_pipeline_builder.initialized()) ctx->execute_ttl_type = MutationHelpers::shouldExecuteTTL(ctx->metadata_snapshot, ctx->interpreter->getColumnDependencies()); - if (ctx->data->getSettings()->exclude_deleted_rows_for_part_size_in_merge && ctx->updated_header.has(RowExistsColumn::name)) + ctx->lightweight_mutation_mode = ctx->updated_header.has(RowExistsColumn::name); + if (ctx->data->getSettings()->exclude_deleted_rows_for_part_size_in_merge && ctx->lightweight_mutation_mode) { /// This mutation contains lightweight delete and we need to count the deleted rows, /// Reset existing_rows_count of new data part to 0 and it will be updated while writing _row_exists column diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index 307d3cb53fc..e69de29bb2d 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -1,5 +0,0 @@ -1231 John 33 -8888 Alice 50 -6666 Ksenia 48 -8888 Alice 50 -p users 3 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index fb32646b46a..4e674fa0cfd 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -10,14 +10,12 @@ CREATE TABLE users ( ) ENGINE = MergeTree order by uid; INSERT INTO users VALUES (1231, 'John', 33); -INSERT INTO users VALUES (6666, 'Ksenia', 48); -INSERT INTO users VALUES (8888, 'Alice', 50); DELETE FROM users WHERE 1; -- { serverError NOT_IMPLEMENTED } -DELETE FROM users WHERE uid = 8888 SETTINGS lightweight_mutation_projection_mode = 'throw'; -- { serverError NOT_IMPLEMENTED } +DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'throw'; -- { serverError NOT_IMPLEMENTED } -DELETE FROM users WHERE uid = 6666 SETTINGS lightweight_mutation_projection_mode = 'drop'; +DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'drop'; SYSTEM FLUSH LOGS; @@ -26,33 +24,8 @@ SELECT name, `table` FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users'); +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); SELECT * FROM users ORDER BY uid; -DROP TABLE users; - -CREATE TABLE users ( - uid Int16, - name String, - age Int16, - projection p (select * order by age) -) ENGINE = MergeTree order by uid; - -INSERT INTO users VALUES (1231, 'John', 33), (6666, 'Ksenia', 48), (8888, 'Alice', 50); - -DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'rebuild'; - -SELECT * FROM users ORDER BY uid; - -SYSTEM FLUSH LOGS; - --- expecting projection p with 3 rows is active -SELECT - name, - `table`, - rows, -FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND active = 1; - DROP TABLE users; \ No newline at end of file From fa65e374dcb66c2e927f52ea521e9a8586feef65 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 11 Jul 2024 11:57:33 +0200 Subject: [PATCH 150/644] fix docs --- docs/en/sql-reference/table-functions/file.md | 4 ++-- docs/en/sql-reference/table-functions/hdfs.md | 4 ++-- docs/en/sql-reference/table-functions/s3.md | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index d21f523ab8e..838a7ab61de 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -206,7 +206,7 @@ SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt3 - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. -## Hive-style partitioning {#hive-style-patitioning} +## Hive-style partitioning {#hive-style-partitioning} When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. @@ -215,7 +215,7 @@ When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtu Use virtual column, created with Hive-style partitioning ``` sql -SET use_hive_patitioning = 1; +SET use_hive_partitioning = 1; SELECT _specified_column from file('/specified_column=specified_data/file.txt'); ``` diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index beb1ad12532..fc84c431066 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -99,7 +99,7 @@ FROM hdfs('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name Strin - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. -## Hive-style partitioning {#hive-style-patitioning} +## Hive-style partitioning {#hive-style-partitioning} When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. @@ -108,7 +108,7 @@ When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtu Use virtual column, created with Hive-style partitioning ``` sql -SET use_hive_patitioning = 1; +SET use_hive_partitioning = 1; SELECT _specified_column from HDFS('hdfs://hdfs1:9000/specified_column=specified_data/file.txt'); ``` diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 45c4caa1a13..15074a77475 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -274,7 +274,7 @@ FROM s3( - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. In case of archive shows uncompressed file size of the file inside the archive. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. -## Hive-style partitioning {#hive-style-patitioning} +## Hive-style partitioning {#hive-style-partitioning} When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. @@ -283,7 +283,7 @@ When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtu Use virtual column, created with Hive-style partitioning ``` sql -SET use_hive_patitioning = 1; +SET use_hive_partitioning = 1; SELECT _specified_column from HDFS('hdfs://hdfs1:9000/specified_column=specified_data/file.txt'); ``` From eb085ea585d10f077d1ce66ee3f663ca016d24e8 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 11 Jul 2024 13:06:29 +0000 Subject: [PATCH 151/644] fix --- src/Interpreters/MutationsInterpreter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index ace285bcfc9..c2341463041 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -488,7 +488,7 @@ static void validateUpdateColumns( { if (!source.supportsLightweightDelete()) { - // if (!source.getStorage()->isMergeTree() + // if (!source.getStorage()->isMergeTree() // || context->getSettingsRef().lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW) // throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); } From df104abcc60366df3946a23c52c2e67a92dcb545 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 11 Jul 2024 17:20:19 +0200 Subject: [PATCH 152/644] try to fix tests --- .../03203_hive_style_partitioning.reference | 8 ++++---- .../03203_hive_style_partitioning.sh | 18 +++++++++--------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index e0f46caf1c8..0e6b6052946 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -29,6 +29,10 @@ Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth Elizabeth Gordon Elizabeth +42 2020-01-01 +[1,2,3] 42.42 +Array(Int64) LowCardinality(Float64) +101 1 TESTING THE URL PARTITIONING first last Elizabeth @@ -60,10 +64,6 @@ Stanley Gibson Elizabeth Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth -42 2020-01-01 -[1,2,3] 42.42 -Array(Int64) LowCardinality(Float64) -101 1 TESTING THE S3 PARTITIONING first last Elizabeth diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 9d805b39b8a..e74f24bfd80 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -5,10 +5,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" +$CLICKHOUSE_CLIENT -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" -$CLICKHOUSE_LOCAL -n -q """ +$CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 1; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -36,17 +36,17 @@ SELECT toTypeName(_array), toTypeName(_float) FROM file('$CURDIR/data_hive/parti SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') WHERE _number = 42; """ -$CLICKHOUSE_LOCAL -n -q """ +$CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 0; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" -$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE URL PARTITIONING'" +$CLICKHOUSE_CLIENT -q "SELECT 'TESTING THE URL PARTITIONING'" -$CLICKHOUSE_LOCAL -n -q """ +$CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 1; SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -67,17 +67,17 @@ SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/colum SELECT *, _non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;""" -$CLICKHOUSE_LOCAL -n -q """ +$CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 0; SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" -$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3 PARTITIONING'" +$CLICKHOUSE_CLIENT -q "SELECT 'TESTING THE S3 PARTITIONING'" -$CLICKHOUSE_LOCAL -n -q """ +$CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 1; SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -100,7 +100,7 @@ SELECT *, _non_existing_column FROM s3('http://localhost:11111/test/hive_partiti SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = _column0; """ -$CLICKHOUSE_LOCAL -n -q """ +$CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 0; SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; From 9257c4aac299836dc3b1e215c8fd8ba9b190d3b4 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 11 Jul 2024 15:31:51 +0000 Subject: [PATCH 153/644] change support lightweight delete condition --- src/Interpreters/InterpreterDeleteQuery.cpp | 5 +++-- src/Interpreters/MutationsInterpreter.cpp | 6 +----- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 +--- 3 files changed, 5 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 0f081c522dd..a7d0264f0b0 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -60,6 +60,7 @@ BlockIO InterpreterDeleteQuery::execute() auto table_lock = table->lockForShare(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = table->getInMemoryMetadataPtr(); + bool hasProjection = table->hasProjection(); auto lightweightDelete = [&]() { @@ -107,13 +108,13 @@ BlockIO InterpreterDeleteQuery::execute() table->mutate(mutation_commands, getContext()); return {}; } - else if (table->supportsLightweightDelete()) + else if (!hasProjection && table->supportsLightweightDelete()) { return lightweightDelete(); } else { - if (table->hasProjection()) + if (hasProjection) { auto context = Context::createCopy(getContext()); auto mode = context->getSettingsRef().lightweight_mutation_projection_mode; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index c2341463041..6d3a4f30b34 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -487,11 +487,7 @@ static void validateUpdateColumns( if (column_name == RowExistsColumn::name) { if (!source.supportsLightweightDelete()) - { - // if (!source.getStorage()->isMergeTree() - // || context->getSettingsRef().lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW) - // throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); - } + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); } else if (virtual_columns.tryGet(column_name)) { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index c2e0e778220..0ef8bcfc681 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1641,11 +1641,9 @@ void IMergeTreeDataPart::loadColumns(bool require) } -/// Project part / part with project parts / compact part doesn't support LWD. bool IMergeTreeDataPart::supportLightweightDeleteMutate() const { - return (part_type == MergeTreeDataPartType::Wide || part_type == MergeTreeDataPartType::Compact) && - parent_part == nullptr && projection_parts.empty(); + return (part_type == MergeTreeDataPartType::Wide || part_type == MergeTreeDataPartType::Compact); } bool IMergeTreeDataPart::hasLightweightDelete() const From 4f11dbc7f372d46769da4ab3af6db83b7967faa0 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 11 Jul 2024 18:25:33 +0000 Subject: [PATCH 154/644] fix with wide part --- src/Storages/MergeTree/MutateTask.cpp | 11 +++--- .../03161_lightweight_delete_projection.sql | 36 ++++++++++++++++++- 2 files changed, 41 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 8ca987eb1f8..57784067720 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1043,7 +1043,7 @@ struct MutationContext /// Whether we need to count lightweight delete rows in this mutation bool count_lightweight_deleted_rows; - bool lightweight_mutation_mode; + bool lightweight_delete_mode; }; using MutationContextPtr = std::shared_ptr; @@ -1573,7 +1573,7 @@ private: } else { - if (!ctx->lightweight_mutation_mode && ctx->source_part->checksums.has(projection.getDirectoryName())) + if (!ctx->lightweight_delete_mode && ctx->source_part->checksums.has(projection.getDirectoryName())) entries_to_hardlink.insert(projection.getDirectoryName()); } } @@ -1843,7 +1843,8 @@ private: hardlinked_files.insert(it->name()); } } - else if (!endsWith(it->name(), ".tmp_proj")) // ignore projection tmp merge dir + /// Ignore projection tmp merge dir, and under lightweight delete mode ignore projection files. + else if (!endsWith(it->name(), ".tmp_proj") && !ctx->lightweight_delete_mode) { // it's a projection part directory ctx->new_data_part->getDataPartStorage().createProjection(destination); @@ -2257,8 +2258,8 @@ bool MutateTask::prepare() if (ctx->mutating_pipeline_builder.initialized()) ctx->execute_ttl_type = MutationHelpers::shouldExecuteTTL(ctx->metadata_snapshot, ctx->interpreter->getColumnDependencies()); - ctx->lightweight_mutation_mode = ctx->updated_header.has(RowExistsColumn::name); - if (ctx->data->getSettings()->exclude_deleted_rows_for_part_size_in_merge && ctx->lightweight_mutation_mode) + ctx->lightweight_delete_mode = ctx->updated_header.has(RowExistsColumn::name); + if (ctx->data->getSettings()->exclude_deleted_rows_for_part_size_in_merge && ctx->lightweight_delete_mode) { /// This mutation contains lightweight delete and we need to count the deleted rows, /// Reset existing_rows_count of new data part to 0 and it will be updated while writing _row_exists column diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 4e674fa0cfd..bfeb0127fa4 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -1,13 +1,47 @@ DROP TABLE IF EXISTS users; +-- compact part CREATE TABLE users ( uid Int16, name String, age Int16, projection p1 (select count(), age group by age), projection p2 (select age, name group by age, name) -) ENGINE = MergeTree order by uid; +) ENGINE = MergeTree order by uid +SETTINGS min_bytes_for_wide_part = 10485760; + +INSERT INTO users VALUES (1231, 'John', 33); + +DELETE FROM users WHERE 1; -- { serverError NOT_IMPLEMENTED } + +DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'throw'; -- { serverError NOT_IMPLEMENTED } + +DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'drop'; + +SYSTEM FLUSH LOGS; + +-- expecting no projection +SELECT + name, + `table` +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); + +SELECT * FROM users ORDER BY uid; + +DROP TABLE users; + + +-- wide part +CREATE TABLE users ( + uid Int16, + name String, + age Int16, + projection p1 (select count(), age group by age), + projection p2 (select age, name group by age, name) +) ENGINE = MergeTree order by uid +SETTINGS min_bytes_for_wide_part = 0; INSERT INTO users VALUES (1231, 'John', 33); From df9211c345e8bcfc53ed392a351e6320991240d1 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 11 Jul 2024 18:32:38 +0000 Subject: [PATCH 155/644] fix --- src/Storages/MergeTree/MutateTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 57784067720..2adcb49d6a3 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1844,7 +1844,7 @@ private: } } /// Ignore projection tmp merge dir, and under lightweight delete mode ignore projection files. - else if (!endsWith(it->name(), ".tmp_proj") && !ctx->lightweight_delete_mode) + else if (!endsWith(it->name(), ".tmp_proj") && !ctx->lightweight_delete_mode) { // it's a projection part directory ctx->new_data_part->getDataPartStorage().createProjection(destination); From 10dd4a9fe66914899ec5e5c89e5b9cc24096f64c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 11 Jul 2024 21:16:47 +0200 Subject: [PATCH 156/644] debugging tests --- .../0_stateless/03203_hive_style_partitioning.sh | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index e74f24bfd80..14b4a116596 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -5,10 +5,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" +$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" -$CLICKHOUSE_CLIENT -n -q """ +$CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -36,17 +36,17 @@ SELECT toTypeName(_array), toTypeName(_float) FROM file('$CURDIR/data_hive/parti SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') WHERE _number = 42; """ -$CLICKHOUSE_CLIENT -n -q """ +$CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" -$CLICKHOUSE_CLIENT -q "SELECT 'TESTING THE URL PARTITIONING'" +$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE URL PARTITIONING'" -$CLICKHOUSE_CLIENT -n -q """ +$CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -67,14 +67,14 @@ SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/colum SELECT *, _non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;""" -$CLICKHOUSE_CLIENT -n -q """ +$CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" -$CLICKHOUSE_CLIENT -q "SELECT 'TESTING THE S3 PARTITIONING'" +$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3 PARTITIONING'" $CLICKHOUSE_CLIENT -n -q """ From 119777cd7346a32f45588d069bf6a89efe091867 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 12 Jul 2024 12:03:25 +0200 Subject: [PATCH 157/644] update reference --- .../0_stateless/03203_hive_style_partitioning.reference | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index 0e6b6052946..d187f4cdd2c 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -29,9 +29,9 @@ Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth Elizabeth Gordon Elizabeth -42 2020-01-01 -[1,2,3] 42.42 -Array(Int64) LowCardinality(Float64) +42 2020-01-01 +[1,2,3] 42.42 +Array(Int64) LowCardinality(Float64) 101 1 TESTING THE URL PARTITIONING From 0988e1deadf34736f126e9eb3a2162d3abbe1314 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 12 Jul 2024 14:59:43 +0200 Subject: [PATCH 158/644] update tests --- .../queries/0_stateless/03203_hive_style_partitioning.reference | 2 +- tests/queries/0_stateless/03203_hive_style_partitioning.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index d187f4cdd2c..be43048dd01 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -96,4 +96,4 @@ Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth Elizabeth Gordon Elizabeth -1 +OK diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 14b4a116596..58a74a3ca8f 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -104,4 +104,4 @@ $CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 0; SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; -""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" +""" 2>&1 | grep -F -q "UNKNOWN_IDENTIFIER" && echo "OK" || echo "FAIL"; From 9c6a49b6d474836ee894ddaaa02ebb982370d25c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 12 Jul 2024 14:30:46 +0000 Subject: [PATCH 159/644] fix WriteBufferFromPocoSocketChunked::finalizeImpl() --- src/IO/WriteBufferFromPocoSocketChunked.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.cpp b/src/IO/WriteBufferFromPocoSocketChunked.cpp index 98c5126c24b..9da46ee2d10 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.cpp +++ b/src/IO/WriteBufferFromPocoSocketChunked.cpp @@ -202,7 +202,7 @@ void WriteBufferFromPocoSocketChunked::nextImpl() void WriteBufferFromPocoSocketChunked::finalizeImpl() { - if (offset() == sizeof(*chunk_size_ptr)) + if (chunked && offset() == sizeof(*chunk_size_ptr)) pos -= sizeof(*chunk_size_ptr); WriteBufferFromPocoSocket::finalizeImpl(); } From 201f813516e1283a4d0528bf71753e8291526ccf Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 13 Jul 2024 02:37:09 +0000 Subject: [PATCH 160/644] add prep for rebuild --- .../MergeTree/MergeMutateSelectedEntry.h | 5 +- .../MergeTree/MergeTreeMutationEntry.cpp | 4 +- .../MergeTree/MergeTreeMutationEntry.h | 6 ++- .../MergeTree/MutatePlainMergeTreeTask.cpp | 2 + src/Storages/MergeTree/MutateTask.cpp | 52 +++++++++++++------ src/Storages/StorageMergeTree.cpp | 11 +++- 6 files changed, 60 insertions(+), 20 deletions(-) diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index c420cbca12b..116c7d26552 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -40,12 +40,15 @@ struct MergeMutateSelectedEntry CurrentlyMergingPartsTaggerPtr tagger; MutationCommandsConstPtr commands; MergeTreeTransactionPtr txn; + Field lightweight_delete_projection_mode; MergeMutateSelectedEntry(FutureMergedMutatedPartPtr future_part_, CurrentlyMergingPartsTaggerPtr tagger_, - MutationCommandsConstPtr commands_, const MergeTreeTransactionPtr & txn_ = NO_TRANSACTION_PTR) + MutationCommandsConstPtr commands_, const MergeTreeTransactionPtr & txn_ = NO_TRANSACTION_PTR, + const Field & lightweight_delete_projection_mode_ = LightweightMutationProjectionMode::THROW) : future_part(future_part_) , tagger(std::move(tagger_)) , commands(commands_) , txn(txn_) + , lightweight_delete_projection_mode(lightweight_delete_projection_mode_) {} }; diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 4dbccb91620..06f4875d120 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -48,7 +48,8 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) } MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number, - const TransactionID & tid_, const WriteSettings & settings) + const TransactionID & tid_, const WriteSettings & settings, + const Field & lightweight_delete_projection_mode_) : create_time(time(nullptr)) , commands(std::move(commands_)) , disk(std::move(disk_)) @@ -56,6 +57,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP , file_name("tmp_mutation_" + toString(tmp_number) + ".txt") , is_temp(true) , tid(tid_) + , lightweight_delete_projection_mode(lightweight_delete_projection_mode_) { try { diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index 04297f2852a..cbc7e2d4274 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -36,9 +36,13 @@ struct MergeTreeMutationEntry /// or UnknownCSN if it's not committed (yet) or RolledBackCSN if it's rolled back or PrehistoricCSN if there is no transaction. CSN csn = Tx::UnknownCSN; + /// From query context. + Field lightweight_delete_projection_mode; + /// Create a new entry and write it to a temporary file. MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number, - const TransactionID & tid_, const WriteSettings & settings); + const TransactionID & tid_, const WriteSettings & settings, + const Field & lightweight_delete_projection_mode_); MergeTreeMutationEntry(const MergeTreeMutationEntry &) = delete; MergeTreeMutationEntry(MergeTreeMutationEntry &&) = default; diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 20f387137e7..1bf337973ff 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -140,6 +140,8 @@ ContextMutablePtr MutatePlainMergeTreeTask::createTaskContext() const auto queryId = getQueryId(); context->setCurrentQueryId(queryId); context->setBackgroundOperationTypeForContext(ClientInfo::BackgroundOperationType::MUTATION); + if (merge_mutate_entry) + context->setSetting("lightweight_mutation_projection_mode", merge_mutate_entry->lightweight_delete_projection_mode); return context; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 2adcb49d6a3..ed603abd9c3 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -656,7 +656,9 @@ static NameSet collectFilesToSkip( const std::set & indices_to_recalc, const String & mrk_extension, const std::set & projections_to_recalc, - const std::set & stats_to_recalc) + const std::set & stats_to_recalc, + const StorageMetadataPtr & metadata_snapshot, + bool lightweight_delete_mode) { NameSet files_to_skip = source_part->getFileNamesWithoutChecksums(); @@ -680,8 +682,16 @@ static NameSet collectFilesToSkip( } } - for (const auto & projection : projections_to_recalc) - files_to_skip.insert(projection->getDirectoryName()); + if (lightweight_delete_mode) + { + for (const auto & projection : metadata_snapshot->getProjections()) + files_to_skip.insert(projection.getDirectoryName()); + } + else + { + for (const auto & projection : projections_to_recalc) + files_to_skip.insert(projection->getDirectoryName()); + } for (const auto & stat : stats_to_recalc) files_to_skip.insert(stat->getFileName() + STATS_FILE_SUFFIX); @@ -1042,8 +1052,6 @@ struct MutationContext /// Whether we need to count lightweight delete rows in this mutation bool count_lightweight_deleted_rows; - - bool lightweight_delete_mode; }; using MutationContextPtr = std::shared_ptr; @@ -1573,7 +1581,7 @@ private: } else { - if (!ctx->lightweight_delete_mode && ctx->source_part->checksums.has(projection.getDirectoryName())) + if (!ctx->updated_header.has(RowExistsColumn::name) && ctx->source_part->checksums.has(projection.getDirectoryName())) entries_to_hardlink.insert(projection.getDirectoryName()); } } @@ -1843,8 +1851,7 @@ private: hardlinked_files.insert(it->name()); } } - /// Ignore projection tmp merge dir, and under lightweight delete mode ignore projection files. - else if (!endsWith(it->name(), ".tmp_proj") && !ctx->lightweight_delete_mode) + else if (!endsWith(it->name(), ".tmp_proj")) // ignore projection tmp merge dir { // it's a projection part directory ctx->new_data_part->getDataPartStorage().createProjection(destination); @@ -2193,6 +2200,7 @@ bool MutateTask::prepare() context_for_reading->setSetting("allow_asynchronous_read_from_io_pool_for_merge_tree", false); context_for_reading->setSetting("max_streams_for_merge_tree_reading", Field(0)); context_for_reading->setSetting("read_from_filesystem_cache_if_exists_otherwise_bypass_cache", 1); + context_for_reading->setSetting("lightweight_mutation_projection_mode", Field(ctx->context->getSettingsRef().lightweight_mutation_projection_mode)); MutationHelpers::splitAndModifyMutationCommands( ctx->source_part, ctx->metadata_snapshot, @@ -2217,6 +2225,15 @@ bool MutateTask::prepare() ctx->mutating_pipeline_builder = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); ctx->progress_callback = MergeProgressCallback((*ctx->mutate_entry)->ptr(), ctx->watch_prev_elapsed, *ctx->stage_progress); + + // ctx->updated_header.has(RowExistsColumn::name); + // for (const auto & projection : ctx->metadata_snapshot->getProjections()) + // { + // if (!ctx->source_part->hasProjection(projection.name)) + // continue; + + // ctx->materialized_projections.insert(projection.name); + // } } auto single_disk_volume = std::make_shared("volume_" + ctx->future_part->name, ctx->space_reservation->getDisk(), 0); @@ -2258,8 +2275,8 @@ bool MutateTask::prepare() if (ctx->mutating_pipeline_builder.initialized()) ctx->execute_ttl_type = MutationHelpers::shouldExecuteTTL(ctx->metadata_snapshot, ctx->interpreter->getColumnDependencies()); - ctx->lightweight_delete_mode = ctx->updated_header.has(RowExistsColumn::name); - if (ctx->data->getSettings()->exclude_deleted_rows_for_part_size_in_merge && ctx->lightweight_delete_mode) + bool lightweight_delete_mode = ctx->updated_header.has(RowExistsColumn::name); + if (ctx->data->getSettings()->exclude_deleted_rows_for_part_size_in_merge && lightweight_delete_mode) { /// This mutation contains lightweight delete and we need to count the deleted rows, /// Reset existing_rows_count of new data part to 0 and it will be updated while writing _row_exists column @@ -2296,10 +2313,13 @@ bool MutateTask::prepare() ctx->context, ctx->materialized_indices); - ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate( - ctx->source_part, - ctx->metadata_snapshot, - ctx->materialized_projections); + if (!lightweight_delete_mode) + { + ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate( + ctx->source_part, + ctx->metadata_snapshot, + ctx->materialized_projections); + } ctx->stats_to_recalc = MutationHelpers::getStatisticsToRecalculate(ctx->metadata_snapshot, ctx->materialized_statistics); @@ -2310,7 +2330,9 @@ bool MutateTask::prepare() ctx->indices_to_recalc, ctx->mrk_extension, ctx->projections_to_recalc, - ctx->stats_to_recalc); + ctx->stats_to_recalc, + ctx->metadata_snapshot, + lightweight_delete_mode); ctx->files_to_rename = MutationHelpers::collectFilesForRenames( ctx->source_part, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 611289ffd78..063e3b7f064 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -517,7 +517,8 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context { std::lock_guard lock(currently_processing_in_background_mutex); - MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings()); + MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings(), + Field(query_context->getSettingsRef().lightweight_mutation_projection_mode)); version = increment.get(); entry.commit(version); String mutation_id = entry.file_name; @@ -1282,12 +1283,18 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( auto commands = std::make_shared(); size_t current_ast_elements = 0; auto last_mutation_to_apply = mutations_end_it; + + /// Trying to grab it from query context. + Field lightweight_delete_projection_mode = LightweightMutationProjectionMode::THROW; + for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { /// Do not squash mutations from different transactions to be able to commit/rollback them independently. if (first_mutation_tid != it->second.tid) break; + lightweight_delete_projection_mode = it->second.lightweight_delete_projection_mode; + size_t commands_size = 0; MutationCommands commands_for_size_validation; for (const auto & command : it->second.commands) @@ -1364,7 +1371,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( future_part->part_format = part->getFormat(); tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}, false), *this, metadata_snapshot, true); - return std::make_shared(future_part, std::move(tagger), commands, txn); + return std::make_shared(future_part, std::move(tagger), commands, txn, lightweight_delete_projection_mode); } } From 0fc14520c821f22b493d32657fede6be10832d60 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 13 Jul 2024 23:06:37 +0000 Subject: [PATCH 161/644] add server termination on exit --- programs/server/fuzzers/tcp_protocol_fuzzer.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/programs/server/fuzzers/tcp_protocol_fuzzer.cpp b/programs/server/fuzzers/tcp_protocol_fuzzer.cpp index 950ea09669a..7cebdc2ad65 100644 --- a/programs/server/fuzzers/tcp_protocol_fuzzer.cpp +++ b/programs/server/fuzzers/tcp_protocol_fuzzer.cpp @@ -10,6 +10,7 @@ #include #include +#include #include @@ -25,6 +26,12 @@ static int64_t port = 9000; using namespace std::chrono_literals; +void on_exit() +{ + BaseDaemon::terminate(); + main_app.wait(); +} + extern "C" int LLVMFuzzerInitialize(int * argc, char ***argv) { @@ -60,6 +67,8 @@ int LLVMFuzzerInitialize(int * argc, char ***argv) exit(-1); } + atexit(on_exit); + return 0; } From 3ccc2aed4c76eba20e0fc88768412bbfacafbb95 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 13 Jul 2024 23:44:13 +0000 Subject: [PATCH 162/644] add fuzzer_arguments to fuzzer runner --- docker/test/libfuzzer/run_libfuzzer.py | 7 +++++++ tests/fuzz/tcp_protocol_fuzzer.options | 4 ++++ 2 files changed, 11 insertions(+) create mode 100644 tests/fuzz/tcp_protocol_fuzzer.options diff --git a/docker/test/libfuzzer/run_libfuzzer.py b/docker/test/libfuzzer/run_libfuzzer.py index 5ed019490d5..cdd09dfa3be 100755 --- a/docker/test/libfuzzer/run_libfuzzer.py +++ b/docker/test/libfuzzer/run_libfuzzer.py @@ -20,6 +20,7 @@ def run_fuzzer(fuzzer: str): options_file = f"{fuzzer}.options" custom_libfuzzer_options = "" + fuzzer_arguments = "" with Path(options_file) as path: if path.exists() and path.is_file(): @@ -47,6 +48,12 @@ def run_fuzzer(fuzzer: str): for key, value in parser["libfuzzer"].items() ) + if parser.has_section("fuzzer_arguments"): + fuzzer_arguments = " ".join( + ("%s" % key) if value == "" else ("%s=%s" % (key, value)) + for key, value in parser["fuzzer_arguments"].items() + ) + cmd_line = f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {corpus_dir}" if custom_libfuzzer_options: cmd_line += f" {custom_libfuzzer_options}" diff --git a/tests/fuzz/tcp_protocol_fuzzer.options b/tests/fuzz/tcp_protocol_fuzzer.options new file mode 100644 index 00000000000..4885669d91d --- /dev/null +++ b/tests/fuzz/tcp_protocol_fuzzer.options @@ -0,0 +1,4 @@ +[fuzzer_arguments] +--log-file=tcp_protocol_fuzzer.log +--= +--logging.terminal=0 From d4116aeaeaeec3b17cd813d686a815476a794bed Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 15 Jul 2024 01:31:40 +0000 Subject: [PATCH 163/644] fix --- src/Core/SettingsEnums.h | 2 +- src/Storages/MergeTree/MergeMutateSelectedEntry.h | 4 ++-- src/Storages/MergeTree/MergeTreeMutationEntry.cpp | 2 +- src/Storages/MergeTree/MergeTreeMutationEntry.h | 4 ++-- src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 3 +-- src/Storages/StorageMergeTree.cpp | 4 ++-- 6 files changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 3611dfa72be..67fbce31be8 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -342,7 +342,7 @@ DECLARE_SETTING_ENUM(ParallelReplicasCustomKeyFilterType) enum class LightweightMutationProjectionMode : uint8_t { THROW, - DROP + DROP, }; DECLARE_SETTING_ENUM(LightweightMutationProjectionMode) diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index 116c7d26552..bf2d1a7f677 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -40,10 +40,10 @@ struct MergeMutateSelectedEntry CurrentlyMergingPartsTaggerPtr tagger; MutationCommandsConstPtr commands; MergeTreeTransactionPtr txn; - Field lightweight_delete_projection_mode; + LightweightMutationProjectionMode lightweight_delete_projection_mode; MergeMutateSelectedEntry(FutureMergedMutatedPartPtr future_part_, CurrentlyMergingPartsTaggerPtr tagger_, MutationCommandsConstPtr commands_, const MergeTreeTransactionPtr & txn_ = NO_TRANSACTION_PTR, - const Field & lightweight_delete_projection_mode_ = LightweightMutationProjectionMode::THROW) + const LightweightMutationProjectionMode & lightweight_delete_projection_mode_ = LightweightMutationProjectionMode::THROW) : future_part(future_part_) , tagger(std::move(tagger_)) , commands(commands_) diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 06f4875d120..d1bd8efa7a5 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -49,7 +49,7 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number, const TransactionID & tid_, const WriteSettings & settings, - const Field & lightweight_delete_projection_mode_) + const LightweightMutationProjectionMode & lightweight_delete_projection_mode_) : create_time(time(nullptr)) , commands(std::move(commands_)) , disk(std::move(disk_)) diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index cbc7e2d4274..3aca744aa15 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -37,12 +37,12 @@ struct MergeTreeMutationEntry CSN csn = Tx::UnknownCSN; /// From query context. - Field lightweight_delete_projection_mode; + LightweightMutationProjectionMode lightweight_delete_projection_mode; /// Create a new entry and write it to a temporary file. MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number, const TransactionID & tid_, const WriteSettings & settings, - const Field & lightweight_delete_projection_mode_); + const LightweightMutationProjectionMode & lightweight_delete_projection_mode_); MergeTreeMutationEntry(const MergeTreeMutationEntry &) = delete; MergeTreeMutationEntry(MergeTreeMutationEntry &&) = default; diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 1bf337973ff..666dbe7e61e 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -140,8 +140,7 @@ ContextMutablePtr MutatePlainMergeTreeTask::createTaskContext() const auto queryId = getQueryId(); context->setCurrentQueryId(queryId); context->setBackgroundOperationTypeForContext(ClientInfo::BackgroundOperationType::MUTATION); - if (merge_mutate_entry) - context->setSetting("lightweight_mutation_projection_mode", merge_mutate_entry->lightweight_delete_projection_mode); + context->setSetting("lightweight_mutation_projection_mode", merge_mutate_entry->lightweight_delete_projection_mode); return context; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 063e3b7f064..7f210779916 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -518,7 +518,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context std::lock_guard lock(currently_processing_in_background_mutex); MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings(), - Field(query_context->getSettingsRef().lightweight_mutation_projection_mode)); + query_context->getSettingsRef().lightweight_mutation_projection_mode); version = increment.get(); entry.commit(version); String mutation_id = entry.file_name; @@ -1285,7 +1285,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( auto last_mutation_to_apply = mutations_end_it; /// Trying to grab it from query context. - Field lightweight_delete_projection_mode = LightweightMutationProjectionMode::THROW; + LightweightMutationProjectionMode lightweight_delete_projection_mode = LightweightMutationProjectionMode::THROW; for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { From 3c09d585cde8068e1f57a1b2adfcdf8b126a8574 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 15 Jul 2024 02:14:58 +0000 Subject: [PATCH 164/644] fix --- src/Storages/MergeTree/MergeMutateSelectedEntry.h | 1 + src/Storages/MergeTree/MergeTreeMutationEntry.h | 1 + 2 files changed, 2 insertions(+) diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index bf2d1a7f677..f75d10d9ecb 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index 3aca744aa15..dbb17654ddd 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB From b5b944b4e6c5c7227c45e428888f1711c14d2f4f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 15 Jul 2024 10:15:32 +0200 Subject: [PATCH 165/644] Improve wording of docs based on review comments Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- docs/en/engines/table-engines/integrations/kafka.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index 8c9cd18d117..2f3c3bf62cd 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -257,7 +257,7 @@ If `allow_experimental_kafka_store_offsets_in_keeper` is enabled, then two more - `kafka_keeper_path` specifies the path to the table in ClickHouse Keeper - `kafka_replica_name` specifies the replica name in ClickHouse Keeper -Either both of the settings must be specified or neither of them. When both of them is specified, then a new, experimental Kafka engine will be used. The new engine doesn't depend on storing the committed offsets in Kafka,but stores them in ClickHouse Keeper. It still tries to commit the offsets to Kafka, but it only depends on those offsets when the table is created. In any other circumstances (table is restarted, or recovered after some error) the offsets stored in ClickHouse Keeper will be used to consume messages from. Apart from the committed offset, it also stores how many messages were consumed in the last batch, so if the insert fails, the same amount of messages will be consumed, thus enabling deduplication if necessary. +Either both of the settings must be specified or neither of them. When both of them are specified, then a new, experimental Kafka engine will be used. The new engine doesn't depend on storing the committed offsets in Kafka,but stores them in ClickHouse Keeper. It still tries to commit the offsets to Kafka, but it only depends on those offsets when the table is created. In any other circumstances (table is restarted, or recovered after some error) the offsets stored in ClickHouse Keeper will be used to consume messages from. Apart from the committed offset, it also stores how many messages were consumed in the last batch, so if the insert fails, the same amount of messages will be consumed, thus enabling deduplication if necessary. Example: @@ -285,8 +285,8 @@ SETTINGS allow_experimental_kafka_store_offsets_in_keeper=1; As the new engine is experimental, it is not production ready yet. There are few known limitations of the implementation: - The biggest limitation is the engine doesn't support direct reading from Kafka topic (insertion works, but reading doesn't), thus the direct `SELECT` queries will fail. - - Rapidly dropping and recreating the table or specifying the same ClickHouse Keeper path to different engines might cause issues. As best practice you can use the `{uuid}` to avoid clashing paths. - - To make repeatable reads possible messages cannot be consumed from multiple partitions on a single thread. On the other hand the Kafka consumers has to be polled regularly to keep them alive. As a result of these two we decided to only allow creating multiple consumer if `kafka_thread_per_consumer` is enabled, otherwise it is too complicated to avoid issues regarding polling consumers regularly. + - Rapidly dropping and recreating the table or specifying the same ClickHouse Keeper path to different engines might cause issues. As best practice you can use the `{uuid}` in `kafka_keeper_path` to avoid clashing paths. + - To make repeatable reads, messages cannot be consumed from multiple partitions on a single thread. On the other hand, the Kafka consumers have to be polled regularly to keep them alive. As a result of these two objectives, we decided to only allow creating multiple consumers if `kafka_thread_per_consumer` is enabled, otherwise it is too complicated to avoid issues regarding polling consumers regularly. - Consumers created by the new storage engine do not show up in [`system.kafka_consumers`](../../../operations/system-tables/kafka_consumers.md) table. **See Also** From 1ecfba837e41c865eb77e6e94eadb0440d7ba2c4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 15 Jul 2024 09:03:05 +0000 Subject: [PATCH 166/644] Rename experimental flag to `allow_experimental_kafka_offsets_storage_in_keeper` --- docs/en/engines/table-engines/integrations/kafka.md | 6 +++--- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- src/Storages/Kafka/StorageKafkaCommon.cpp | 4 ++-- tests/integration/test_storage_kafka/test.py | 2 +- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index 2f3c3bf62cd..389bb6c9029 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -253,7 +253,7 @@ The number of rows in one Kafka message depends on whether the format is row-bas ## Experimental engine to store committed offsets in ClickHouse Keeper -If `allow_experimental_kafka_store_offsets_in_keeper` is enabled, then two more settings can be specified to the Kafka table engine: +If `allow_experimental_kafka_offsets_storage_in_keeper` is enabled, then two more settings can be specified to the Kafka table engine: - `kafka_keeper_path` specifies the path to the table in ClickHouse Keeper - `kafka_replica_name` specifies the replica name in ClickHouse Keeper @@ -267,7 +267,7 @@ ENGINE = Kafka('localhost:19092', 'my-topic', 'my-consumer', 'JSONEachRow') SETTINGS kafka_keeper_path = '/clickhouse/{database}/experimental_kafka', kafka_replica_name = 'r1' -SETTINGS allow_experimental_kafka_store_offsets_in_keeper=1; +SETTINGS allow_experimental_kafka_offsets_storage_in_keeper=1; ``` Or to utilize the `uuid` and `replica` macros similarly to ReplicatedMergeTree: @@ -278,7 +278,7 @@ ENGINE = Kafka('localhost:19092', 'my-topic', 'my-consumer', 'JSONEachRow') SETTINGS kafka_keeper_path = '/clickhouse/{database}/{uuid}', kafka_replica_name = '{replica}' -SETTINGS allow_experimental_kafka_store_offsets_in_keeper=1; +SETTINGS allow_experimental_kafka_offsets_storage_in_keeper=1; ``` ### Known limitations diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 57a5216b870..d24ea494c73 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -687,7 +687,7 @@ class IColumn; M(UInt64, max_size_to_preallocate_for_joins, 100'000'000, "For how many elements it is allowed to preallocate space in all hash tables in total before join", 0) \ \ M(Bool, kafka_disable_num_consumers_limit, false, "Disable limit on kafka_num_consumers that depends on the number of available CPU cores", 0) \ - M(Bool, allow_experimental_kafka_store_offsets_in_keeper, false, "Allow experimental feature to store Kafka related offsets in ClickHouse Keeper. When enabled a ClickHouse Keeper path and replica name can be specified to the Kafka table engine. As a result instead of the regular Kafka engine, a new type of storage engine will be used that stores the committed offsets primarily in ClickHouse Keeper", 0) \ + M(Bool, allow_experimental_kafka_offsets_storage_in_keeper, false, "Allow experimental feature to store Kafka related offsets in ClickHouse Keeper. When enabled a ClickHouse Keeper path and replica name can be specified to the Kafka table engine. As a result instead of the regular Kafka engine, a new type of storage engine will be used that stores the committed offsets primarily in ClickHouse Keeper", 0) \ M(Bool, enable_software_prefetch_in_aggregation, true, "Enable use of software prefetch in aggregation", 0) \ M(Bool, allow_aggregate_partitions_independently, false, "Enable independent aggregation of partitions on separate threads when partition key suits group by key. Beneficial when number of partitions close to number of cores and partitions have roughly the same size", 0) \ M(Bool, force_aggregate_partitions_independently, false, "Force the use of optimization when it is applicable, but heuristics decided not to use it", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 6f96f693681..d612a4f9e08 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -69,7 +69,7 @@ static std::initializer_list( args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); - if (!args.getLocalContext()->getSettingsRef().allow_experimental_kafka_store_offsets_in_keeper && !args.query.attach) + if (!args.getLocalContext()->getSettingsRef().allow_experimental_kafka_offsets_storage_in_keeper && !args.query.attach) throw Exception( ErrorCodes::SUPPORT_IS_DISABLED, - "Storing the Kafka offsets in Keeper is experimental. Set `allow_experimental_kafka_store_offsets_in_keeper` setting " + "Storing the Kafka offsets in Keeper is experimental. Set `allow_experimental_kafka_offsets_storage_in_keeper` setting " "to enable it"); if (!has_keeper_path || !has_replica_name) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 81132a9a60f..ea8f6671d20 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -422,7 +422,7 @@ def generate_new_create_table_query( settings_string = create_settings_string(settings) query = f"""CREATE TABLE {database}.{table_name} ({columns_def}) ENGINE = Kafka('{brokers}', '{topic_list}', '{consumer_group}', '{format}', '{row_delimiter}') {settings_string} -SETTINGS allow_experimental_kafka_store_offsets_in_keeper=1""" +SETTINGS allow_experimental_kafka_offsets_storage_in_keeper=1""" logging.debug(f"Generated new create query: {query}") return query From 16c82501c6292835079822df7aa1f341a644f299 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 15 Jul 2024 11:54:04 +0000 Subject: [PATCH 167/644] Small review fixes --- src/Storages/Kafka/KafkaConsumer2.cpp | 4 +- src/Storages/Kafka/KafkaConsumer2.h | 7 +-- src/Storages/Kafka/StorageKafka.cpp | 36 ++++-------- src/Storages/Kafka/StorageKafka.h | 3 - src/Storages/Kafka/StorageKafka2.cpp | 72 ++++++++--------------- src/Storages/Kafka/StorageKafka2.h | 5 +- src/Storages/Kafka/StorageKafkaCommon.cpp | 21 ++++++- src/Storages/Kafka/StorageKafkaCommon.h | 8 ++- 8 files changed, 66 insertions(+), 90 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index d471c263653..51fc9bbe968 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -55,14 +55,14 @@ KafkaConsumer2::KafkaConsumer2( size_t max_batch_size, size_t poll_timeout_, const std::atomic & stopped_, - const Names & _topics) + const Names & topics_) : consumer(consumer_) , log(log_) , batch_size(max_batch_size) , poll_timeout(poll_timeout_) , stopped(stopped_) , current(messages.begin()) - , topics(_topics) + , topics(topics_) { // called (synchronously, during poll) when we enter the consumer group consumer->set_assignment_callback( diff --git a/src/Storages/Kafka/KafkaConsumer2.h b/src/Storages/Kafka/KafkaConsumer2.h index d7ec227d0bd..0df37434caf 100644 --- a/src/Storages/Kafka/KafkaConsumer2.h +++ b/src/Storages/Kafka/KafkaConsumer2.h @@ -71,18 +71,13 @@ public: using TopicPartitionCounts = std::vector; - // struct AssignmentChanges { - // TopicPartitions revoked_partitions; - // TopicPartitions new_partitions; - // }; - KafkaConsumer2( ConsumerPtr consumer_, LoggerPtr log_, size_t max_batch_size, size_t poll_timeout_, const std::atomic & stopped_, - const Names & _topics); + const Names & topics_); ~KafkaConsumer2(); diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 688b78ef78d..0e907187e11 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -21,19 +21,18 @@ #include #include #include +#include #include -#include #include #include #include #include #include +#include #include #include -#include #include #include -#include #include #include #include @@ -42,10 +41,10 @@ #include #include #include +#include #include #include #include -#include #include #include #include @@ -148,19 +147,22 @@ private: }; StorageKafka::StorageKafka( - const StorageID & table_id_, ContextPtr context_, - const ColumnsDescription & columns_, std::unique_ptr kafka_settings_, + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + std::unique_ptr kafka_settings_, const String & collection_name_) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , kafka_settings(std::move(kafka_settings_)) , macros_info{.table_id = table_id_} - , topics(parseTopics(getContext()->getMacros()->expand(kafka_settings->kafka_topic_list.value, macros_info))) + , topics(StorageKafkaUtils::parseTopics(getContext()->getMacros()->expand(kafka_settings->kafka_topic_list.value, macros_info))) , brokers(getContext()->getMacros()->expand(kafka_settings->kafka_broker_list.value, macros_info)) , group(getContext()->getMacros()->expand(kafka_settings->kafka_group_name.value, macros_info)) , client_id( - kafka_settings->kafka_client_id.value.empty() ? getDefaultClientId(table_id_) - : getContext()->getMacros()->expand(kafka_settings->kafka_client_id.value, macros_info)) + kafka_settings->kafka_client_id.value.empty() + ? StorageKafkaUtils::getDefaultClientId(table_id_) + : getContext()->getMacros()->expand(kafka_settings->kafka_client_id.value, macros_info)) , format_name(getContext()->getMacros()->expand(kafka_settings->kafka_format.value)) , max_rows_per_message(kafka_settings->kafka_max_rows_per_message.value) , schema_name(getContext()->getMacros()->expand(kafka_settings->kafka_schema.value, macros_info)) @@ -261,22 +263,6 @@ SettingsChanges StorageKafka::createSettingsAdjustments() return result; } -Names StorageKafka::parseTopics(String topic_list) -{ - Names result; - boost::split(result,topic_list,[](char c){ return c == ','; }); - for (String & topic : result) - { - boost::trim(topic); - } - return result; -} - -String StorageKafka::getDefaultClientId(const StorageID & table_id_) -{ - return fmt::format("{}-{}-{}-{}", VERSION_NAME, getFQDNOrHostName(), table_id_.database_name, table_id_.table_name); -} - void StorageKafka::read( QueryPlan & query_plan, const Names & column_names, diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index ef1781d5054..31a5744ee2a 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -167,9 +167,6 @@ private: size_t getMaxBlockSize() const; size_t getPollTimeoutMillisecond() const; - static Names parseTopics(String topic_list); - static String getDefaultClientId(const StorageID & table_id_); - bool streamToViews(); bool checkDependencies(const StorageID & table_id); diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 754c2dfa926..652ded635e4 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -33,10 +34,8 @@ #include #include #include -#include #include #include -#include "Common/config_version.h" #include #include #include @@ -45,12 +44,12 @@ #include #include #include +#include #include #include #include #include #include -#include "Storages/Kafka/KafkaConsumer2.h" #if USE_KRB5 # include @@ -115,18 +114,18 @@ StorageKafka2::StorageKafka2( , replica_path(keeper_path + "/replicas/" + kafka_settings_->kafka_replica_name.value) , kafka_settings(std::move(kafka_settings_)) , macros_info{.table_id = table_id_} - , topics(parseTopics(getContext()->getMacros()->expand(kafka_settings->kafka_topic_list.value, macros_info))) + , topics(StorageKafkaUtils::parseTopics(getContext()->getMacros()->expand(kafka_settings->kafka_topic_list.value, macros_info))) , brokers(getContext()->getMacros()->expand(kafka_settings->kafka_broker_list.value, macros_info)) , group(getContext()->getMacros()->expand(kafka_settings->kafka_group_name.value, macros_info)) , client_id( kafka_settings->kafka_client_id.value.empty() - ? getDefaultClientId(table_id_) + ? StorageKafkaUtils::getDefaultClientId(table_id_) : getContext()->getMacros()->expand(kafka_settings->kafka_client_id.value, macros_info)) , format_name(getContext()->getMacros()->expand(kafka_settings->kafka_format.value)) , max_rows_per_message(kafka_settings->kafka_max_rows_per_message.value) , schema_name(getContext()->getMacros()->expand(kafka_settings->kafka_schema.value, macros_info)) , num_consumers(kafka_settings->kafka_num_consumers.value) - , log(getLogger(String("StorageKafka2 ") + table_id_.getNameForLogs())) + , log(getLogger("StorageKafka2 (" + table_id_.getNameForLogs() + ")")) , semaphore(0, static_cast(num_consumers)) , settings_adjustments(createSettingsAdjustments()) , thread_per_consumer(kafka_settings->kafka_thread_per_consumer.value) @@ -134,7 +133,7 @@ StorageKafka2::StorageKafka2( , active_node_identifier(toString(ServerUUID::get())) { if (kafka_settings->kafka_num_consumers > 1 && !thread_per_consumer) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "With multiple consumer you have to use thread per consumer!"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "With multiple consumers, it is required to use `kafka_thread_per_consumer` setting"); if (kafka_settings->kafka_handle_error_mode == StreamingHandleErrorMode::STREAM) { @@ -186,15 +185,15 @@ VirtualColumnsDescription StorageKafka2::createVirtuals(StreamingHandleErrorMode } void StorageKafka2::partialShutdown() { + LOG_TRACE(log, "Cancelling streams"); for (auto & task : tasks) { - LOG_TRACE(log, "Cancelling streams"); task->stream_cancelled = true; } + LOG_TRACE(log, "Waiting for cleanup"); for (auto & task : tasks) { - LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); } is_active = false; @@ -202,14 +201,14 @@ void StorageKafka2::partialShutdown() bool StorageKafka2::activate() { - LOG_TEST(log, "activate task"); + LOG_TEST(log, "Activate task"); if (is_active && !getZooKeeper()->expired()) { LOG_TEST(log, "No need to activate"); return true; } - if (first_time) + if (first_activation_time) { LOG_DEBUG(log, "Activating replica"); assert(!is_active); @@ -252,12 +251,9 @@ bool StorageKafka2::activate() String is_active_path = fs::path(replica_path) / "is_active"; zookeeper->deleteEphemeralNodeIfContentMatches(is_active_path, active_node_identifier); - /// Simultaneously declare that this replica is active, and update the host. - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCreateRequest(is_active_path, active_node_identifier, zkutil::CreateMode::Ephemeral)); - try { + /// Simultaneously declare that this replica is active, and update the host. zookeeper->create(is_active_path, active_node_identifier, zkutil::CreateMode::Ephemeral); } catch (const Coordination::Exception & e) @@ -276,27 +272,21 @@ bool StorageKafka2::activate() return true; } - catch (...) + catch (const Coordination::Exception & e) { replica_is_active_node = nullptr; + LOG_ERROR(log, "Couldn't start replica: {}. {}", e.what(), DB::getCurrentExceptionMessage(true)); + return false; - try - { + } + catch (const Exception & e) + { + replica_is_active_node = nullptr; + if (e.code() != ErrorCodes::REPLICA_IS_ALREADY_ACTIVE) throw; - } - catch (const Coordination::Exception & e) - { - LOG_ERROR(log, "Couldn't start replica: {}. {}", e.what(), DB::getCurrentExceptionMessage(true)); - return false; - } - catch (const Exception & e) - { - if (e.code() != ErrorCodes::REPLICA_IS_ALREADY_ACTIVE) - throw; - LOG_ERROR(log, "Couldn't start replica: {}. {}", e.what(), DB::getCurrentExceptionMessage(true)); - return false; - } + LOG_ERROR(log, "Couldn't start replica: {}. {}", e.what(), DB::getCurrentExceptionMessage(true)); + return false; } }; @@ -315,8 +305,8 @@ bool StorageKafka2::activate() task->holder->activateAndSchedule(); } - if (first_time) - first_time = false; + if (first_activation_time) + first_activation_time = false; LOG_DEBUG(log, "Table activated successfully"); return true; @@ -357,20 +347,6 @@ SettingsChanges StorageKafka2::createSettingsAdjustments() return result; } -Names StorageKafka2::parseTopics(String topic_list) -{ - Names result; - boost::split(result, topic_list, [](char c) { return c == ','; }); - for (String & topic : result) - boost::trim(topic); - return result; -} - -String StorageKafka2::getDefaultClientId(const StorageID & table_id_) -{ - return fmt::format("{}-{}-{}-{}", VERSION_NAME, getFQDNOrHostName(), table_id_.database_name, table_id_.table_name); -} - Pipe StorageKafka2::read( const Names & /*column_names */, @@ -381,7 +357,7 @@ Pipe StorageKafka2::read( size_t /* max_block_size */, size_t /* num_streams */) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "You cannot read from the new Kafka storage!"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Direct read from the new Kafka storage is not implemented"); } diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index 0d6734ac0eb..e7c192bc3b0 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -162,7 +162,7 @@ private: zkutil::EphemeralNodeHolderPtr replica_is_active_node; BackgroundSchedulePool::TaskHolder activating_task; String active_node_identifier; - bool first_time = true; + bool first_activation_time = true; bool activate(); void partialShutdown(); @@ -194,9 +194,6 @@ private: size_t getMaxBlockSize() const; size_t getPollTimeoutMillisecond() const; - static Names parseTopics(String topic_list); - static String getDefaultClientId(const StorageID & table_id_); - enum class StallReason { NoAssignment, diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp index d570d9993b6..470e3445d03 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.cpp +++ b/src/Storages/Kafka/StorageKafkaCommon.cpp @@ -11,11 +11,13 @@ #include #include #include +#include #include #include #include #include #include +#include #include #include #include @@ -505,7 +507,7 @@ void registerStorageKafka(StorageFactory & factory) auto * settings_query = args.storage_def->settings; - chassert(settings_query != nullptr && "Unexpected settings query in StorageKafka"); + chassert(has_settings && "Unexpected settings query in StorageKafka"); settings_query->changes.setSetting("kafka_keeper_path", kafka_settings->kafka_keeper_path.value); settings_query->changes.setSetting("kafka_replica_name", kafka_settings->kafka_replica_name.value); @@ -539,6 +541,23 @@ void registerStorageKafka(StorageFactory & factory) }); } +namespace StorageKafkaUtils +{ +Names parseTopics(String topic_list) +{ + Names result; + boost::split(result, topic_list, [](char c) { return c == ','; }); + for (String & topic : result) + boost::trim(topic); + return result; +} + +String getDefaultClientId(const StorageID & table_id) +{ + return fmt::format("{}-{}-{}-{}", VERSION_NAME, getFQDNOrHostName(), table_id.database_name, table_id.table_name); +} +} + template struct StorageKafkaInterceptors; template struct StorageKafkaInterceptors; diff --git a/src/Storages/Kafka/StorageKafkaCommon.h b/src/Storages/Kafka/StorageKafkaCommon.h index bed09e9a9cd..f0bae7c9c9c 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.h +++ b/src/Storages/Kafka/StorageKafkaCommon.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -60,8 +61,13 @@ struct KafkaConfigLoader const String & config_prefix, const Names & topics); }; -} +namespace StorageKafkaUtils +{ +Names parseTopics(String topic_list); +String getDefaultClientId(const StorageID & table_id); +} +} template <> struct fmt::formatter : fmt::ostream_formatter From 7c1a181469949f3ddb09380c60bf5f97021e71df Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 15 Jul 2024 12:04:46 +0000 Subject: [PATCH 168/644] Refactor how we handle activation of new Kafka storage --- src/Storages/Kafka/StorageKafka2.cpp | 51 ++++++++++++++++++++++------ src/Storages/Kafka/StorageKafka2.h | 3 +- 2 files changed, 43 insertions(+), 11 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 652ded635e4..428426d9e6a 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -158,7 +158,7 @@ StorageKafka2::StorageKafka2( if (!first_replica) createReplica(); - activating_task = getContext()->getSchedulePool().createTask(log->name() + "(activating task)", [this]() { activate(); }); + activating_task = getContext()->getSchedulePool().createTask(log->name() + "(activating task)", [this]() { activateAndReschedule(); }); activating_task->deactivate(); } @@ -185,6 +185,7 @@ VirtualColumnsDescription StorageKafka2::createVirtuals(StreamingHandleErrorMode } void StorageKafka2::partialShutdown() { + // This is called in a background task within a catch block, thus this function shouldn't throw LOG_TRACE(log, "Cancelling streams"); for (auto & task : tasks) { @@ -208,12 +209,7 @@ bool StorageKafka2::activate() return true; } - if (first_activation_time) - { - LOG_DEBUG(log, "Activating replica"); - assert(!is_active); - } - else if (!is_active) + if (!is_active) { LOG_WARNING(log, "Table was not active. Will try to activate it"); } @@ -305,13 +301,48 @@ bool StorageKafka2::activate() task->holder->activateAndSchedule(); } - if (first_activation_time) - first_activation_time = false; - LOG_DEBUG(log, "Table activated successfully"); return true; } +void StorageKafka2::activateAndReschedule() +{ + if (shutdown_called) + return; + + /// It would be ideal to introduce a setting for this + constexpr static size_t check_period_ms = 60000; + /// In case of any exceptions we want to rerun the this task as fast as possible but we also don't want to keep retrying immediately + /// in a close loop (as fast as tasks can be processed), so we'll retry in between 100 and 10000 ms + const size_t backoff_ms = 100 * ((consecutive_activate_failures + 1) * (consecutive_activate_failures + 2)) / 2; + const size_t next_failure_retry_ms = std::min(size_t{10000}, backoff_ms); + + try + { + bool replica_is_active = activate(); + if (replica_is_active) + { + consecutive_activate_failures = 0; + activating_task->scheduleAfter(check_period_ms); + } + else + { + consecutive_activate_failures++; + activating_task->scheduleAfter(next_failure_retry_ms); + } + } + catch (...) + { + consecutive_activate_failures++; + activating_task->scheduleAfter(next_failure_retry_ms); + + /// We couldn't activate table let's set it into readonly mode if necessary + /// We do this after scheduling the task in case it throws + partialShutdown(); + tryLogCurrentException(log, "Failed to restart the table. Will try again"); + } +} + void StorageKafka2::assertActive() const { // TODO(antaljanosbenjamin): change LOGICAL_ERROR to something sensible diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index e7c192bc3b0..d7909df1b2c 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -162,8 +162,9 @@ private: zkutil::EphemeralNodeHolderPtr replica_is_active_node; BackgroundSchedulePool::TaskHolder activating_task; String active_node_identifier; - bool first_activation_time = true; + UInt64 consecutive_activate_failures = 0; bool activate(); + void activateAndReschedule(); void partialShutdown(); void assertActive() const; From 123fd6b7503256b95991b098a3ddf875e5bcdcda Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 15 Jul 2024 12:40:44 +0000 Subject: [PATCH 169/644] Remove seemingly unnecessary `nullptr` check --- src/Storages/Kafka/KafkaConsumer2.cpp | 11 ++-- tests/integration/test_storage_kafka/test.py | 60 ++++++++++++++++++++ 2 files changed, 67 insertions(+), 4 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 51fc9bbe968..52a4cadd60b 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -401,15 +401,14 @@ void KafkaConsumer2::commit(const TopicPartition & topic_partition) ReadBufferPtr KafkaConsumer2::getNextMessage() { - while (current != messages.end()) + if (current != messages.end()) { const auto * data = current->get_payload().get_data(); size_t size = current->get_payload().get_size(); ++current; - // TODO(antaljanosbenjamin): When this can be nullptr? - if (data) - return std::make_shared(data, size); + chassert(data != nullptr); + return std::make_shared(data, size); } return nullptr; @@ -433,7 +432,11 @@ size_t KafkaConsumer2::filterMessageErrors() }); if (skipped) + { LOG_ERROR(log, "There were {} messages with an error", skipped); + // Technically current is invalidated as soon as we erased a single message + current = messages.begin(); + } return skipped; } diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index ea8f6671d20..710e05b5669 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -5444,6 +5444,66 @@ def test_multiple_read_in_materialized_views(kafka_cluster, create_query_generat ) + + +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_null_message(kafka_cluster, create_query_generator): + topic_name = "null_message" + + instance.query( + f""" + DROP TABLE IF EXISTS test.null_message_view; + DROP TABLE IF EXISTS test.null_message_consumer; + DROP TABLE IF EXISTS test.null_message_kafka; + + {create_query_generator("null_message_kafka", "value UInt64", topic_list=topic_name, consumer_group="mv")}; + CREATE TABLE test.null_message_view (value UInt64) + ENGINE = MergeTree() + ORDER BY value; + CREATE MATERIALIZED VIEW test.null_message_consumer TO test.null_message_view AS + SELECT * FROM test.null_message_kafka; + """ + ) + + message_key_values = [] + for i in range(5): + # Here the key is key for Kafka message + message = json.dumps({"value": i}) if i != 3 else None + message_key_values.append({"key": f"{i}".encode(), "message": message}) + + producer = get_kafka_producer( + kafka_cluster.kafka_port, producer_serializer, 15 + ) + for message_kv in message_key_values: + producer.send(topic=topic_name, key = message_kv["key"], value=message_kv["message"]) + producer.flush() + + expected = TSV( + """ +0 +1 +2 +4 +""" + ) + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): + result = instance.query_with_retry( + "SELECT * FROM test.null_message_view", check_callback=lambda res: TSV(res) == expected + ) + + assert expected == TSV(result) + + instance.query( + """ + DROP TABLE test.null_message_consumer; + DROP TABLE test.null_message_view; + DROP TABLE test.null_message_kafka; + """ + ) + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 87e9b7c5bbd2799e2ed7216b0416d137537fba17 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 15 Jul 2024 13:13:47 +0000 Subject: [PATCH 170/644] Automatic style fix --- tests/integration/test_storage_kafka/test.py | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 710e05b5669..85b420cc10e 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -5444,8 +5444,6 @@ def test_multiple_read_in_materialized_views(kafka_cluster, create_query_generat ) - - @pytest.mark.parametrize( "create_query_generator", [generate_old_create_table_query, generate_new_create_table_query], @@ -5474,24 +5472,25 @@ def test_kafka_null_message(kafka_cluster, create_query_generator): message = json.dumps({"value": i}) if i != 3 else None message_key_values.append({"key": f"{i}".encode(), "message": message}) - producer = get_kafka_producer( - kafka_cluster.kafka_port, producer_serializer, 15 - ) + producer = get_kafka_producer(kafka_cluster.kafka_port, producer_serializer, 15) for message_kv in message_key_values: - producer.send(topic=topic_name, key = message_kv["key"], value=message_kv["message"]) + producer.send( + topic=topic_name, key=message_kv["key"], value=message_kv["message"] + ) producer.flush() expected = TSV( - """ + """ 0 1 2 4 """ - ) + ) with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): result = instance.query_with_retry( - "SELECT * FROM test.null_message_view", check_callback=lambda res: TSV(res) == expected + "SELECT * FROM test.null_message_view", + check_callback=lambda res: TSV(res) == expected, ) assert expected == TSV(result) @@ -5504,6 +5503,7 @@ def test_kafka_null_message(kafka_cluster, create_query_generator): """ ) + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 36fb1cc3e79c9570ba43f81e4e47041100a63d0d Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 15 Jul 2024 13:15:14 +0000 Subject: [PATCH 171/644] temporarily disable the setting in taskcontext --- src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 666dbe7e61e..19aa63d90a2 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -140,7 +140,7 @@ ContextMutablePtr MutatePlainMergeTreeTask::createTaskContext() const auto queryId = getQueryId(); context->setCurrentQueryId(queryId); context->setBackgroundOperationTypeForContext(ClientInfo::BackgroundOperationType::MUTATION); - context->setSetting("lightweight_mutation_projection_mode", merge_mutate_entry->lightweight_delete_projection_mode); + // context->setSetting("lightweight_mutation_projection_mode", merge_mutate_entry->lightweight_delete_projection_mode); return context; } From 4df94a0ef3f8af73328d0a8f45bb217cc70b2e45 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 15 Jul 2024 14:47:52 +0000 Subject: [PATCH 172/644] cleanup for setting in mergetree --- src/Core/Settings.h | 1 - src/Core/SettingsChangesHistory.cpp | 1 - src/Interpreters/InterpreterDeleteQuery.cpp | 66 +++++-------------- src/Storages/IStorage.h | 3 - .../MergeTree/MergeMutateSelectedEntry.h | 7 +- src/Storages/MergeTree/MergeTreeData.cpp | 15 ----- src/Storages/MergeTree/MergeTreeData.h | 2 - .../MergeTree/MergeTreeMutationEntry.cpp | 4 +- .../MergeTree/MergeTreeMutationEntry.h | 7 +- src/Storages/MergeTree/MergeTreeSettings.h | 1 + .../MergeTree/MutatePlainMergeTreeTask.cpp | 1 - src/Storages/MergeTree/MutateTask.cpp | 10 --- src/Storages/StorageMergeTree.cpp | 10 +-- 13 files changed, 25 insertions(+), 103 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f7b44ea775c..bafc3f93846 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -588,7 +588,6 @@ class IColumn; M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) ALIAS(allow_experimental_lightweight_delete) \ M(UInt64, lightweight_deletes_sync, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes", 0) \ - M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop projections of this table's relevant parts.", 0) \ M(Bool, apply_deleted_mask, true, "Enables filtering out rows deleted with lightweight DELETE. If disabled, a query will be able to read those rows. This is useful for debugging and \"undelete\" scenarios", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 194292a467e..d6cc0112e0a 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -70,7 +70,6 @@ static std::initializer_listlockForShare(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = table->getInMemoryMetadataPtr(); - bool hasProjection = table->hasProjection(); - auto lightweightDelete = [&]() + if (table->supportsDelete()) + { + /// Convert to MutationCommand + MutationCommands mutation_commands; + MutationCommand mut_command; + + mut_command.type = MutationCommand::Type::DELETE; + mut_command.predicate = delete_query.predicate; + + mutation_commands.emplace_back(mut_command); + + table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); + MutationsInterpreter::Settings settings(false); + MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), settings).validate(); + table->mutate(mutation_commands, getContext()); + return {}; + } + else if (table->supportsLightweightDelete()) { if (!getContext()->getSettingsRef().enable_lightweight_delete) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, @@ -89,54 +104,9 @@ BlockIO InterpreterDeleteQuery::execute() context->setSetting("mutations_sync", Field(context->getSettingsRef().lightweight_deletes_sync)); InterpreterAlterQuery alter_interpreter(alter_ast, context); return alter_interpreter.execute(); - }; - - if (table->supportsDelete()) - { - /// Convert to MutationCommand - MutationCommands mutation_commands; - MutationCommand mut_command; - - mut_command.type = MutationCommand::Type::DELETE; - mut_command.predicate = delete_query.predicate; - - mutation_commands.emplace_back(mut_command); - - table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); - MutationsInterpreter::Settings settings(false); - MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), settings).validate(); - table->mutate(mutation_commands, getContext()); - return {}; - } - else if (!hasProjection && table->supportsLightweightDelete()) - { - return lightweightDelete(); } else { - if (hasProjection) - { - auto context = Context::createCopy(getContext()); - auto mode = context->getSettingsRef().lightweight_mutation_projection_mode; - - if (mode == LightweightMutationProjectionMode::THROW) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "DELETE query is not supported for table {} as it has projections. " - "User should drop all the projections manually before running the query", - table->getStorageID().getFullTableName()); - } - else if (mode == LightweightMutationProjectionMode::DROP) - { - return lightweightDelete(); - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unrecognized lightweight_mutation_projection_mode, only throw and drop are allowed."); - } - } - throw Exception(ErrorCodes::BAD_ARGUMENTS, "DELETE query is not supported for table {}", table->getStorageID().getFullTableName()); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 6217470780d..991c8ff64af 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -262,9 +262,6 @@ public: /// Return true if storage can execute lightweight delete mutations. virtual bool supportsLightweightDelete() const { return false; } - /// Return true if storage has any projection. - virtual bool hasProjection() const { return false; } - /// Return true if storage can execute 'DELETE FROM' mutations. This is different from lightweight delete /// because those are internally translated into 'ALTER UDPATE' mutations. virtual bool supportsDelete() const { return false; } diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index f75d10d9ecb..e7efe00741c 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -2,7 +2,7 @@ #include #include -#include + namespace DB { @@ -41,15 +41,12 @@ struct MergeMutateSelectedEntry CurrentlyMergingPartsTaggerPtr tagger; MutationCommandsConstPtr commands; MergeTreeTransactionPtr txn; - LightweightMutationProjectionMode lightweight_delete_projection_mode; MergeMutateSelectedEntry(FutureMergedMutatedPartPtr future_part_, CurrentlyMergingPartsTaggerPtr tagger_, - MutationCommandsConstPtr commands_, const MergeTreeTransactionPtr & txn_ = NO_TRANSACTION_PTR, - const LightweightMutationProjectionMode & lightweight_delete_projection_mode_ = LightweightMutationProjectionMode::THROW) + MutationCommandsConstPtr commands_, const MergeTreeTransactionPtr & txn_ = NO_TRANSACTION_PTR) : future_part(future_part_) , tagger(std::move(tagger_)) , commands(commands_) , txn(txn_) - , lightweight_delete_projection_mode(lightweight_delete_projection_mode_) {} }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e31f6db5409..5182147350e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6158,21 +6158,6 @@ bool MergeTreeData::supportsLightweightDelete() const return true; } -bool MergeTreeData::hasProjection() const -{ - auto lock = lockParts(); - for (const auto & part : data_parts_by_info) - { - if (part->getState() == MergeTreeDataPartState::Outdated - || part->getState() == MergeTreeDataPartState::Deleting) - continue; - - if (part->hasProjection()) - return true; - } - return false; -} - MergeTreeData::ProjectionPartsVector MergeTreeData::getAllProjectionPartsVector(MergeTreeData::DataPartStateVector * out_states) const { ProjectionPartsVector res; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index c8b721038c6..7d216f989c1 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -439,8 +439,6 @@ public: bool supportsLightweightDelete() const override; - bool hasProjection() const override; - bool areAsynchronousInsertsEnabled() const override { return getSettings()->async_insert; } bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override; diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index d1bd8efa7a5..4dbccb91620 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -48,8 +48,7 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) } MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number, - const TransactionID & tid_, const WriteSettings & settings, - const LightweightMutationProjectionMode & lightweight_delete_projection_mode_) + const TransactionID & tid_, const WriteSettings & settings) : create_time(time(nullptr)) , commands(std::move(commands_)) , disk(std::move(disk_)) @@ -57,7 +56,6 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP , file_name("tmp_mutation_" + toString(tmp_number) + ".txt") , is_temp(true) , tid(tid_) - , lightweight_delete_projection_mode(lightweight_delete_projection_mode_) { try { diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index dbb17654ddd..04297f2852a 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB @@ -37,13 +36,9 @@ struct MergeTreeMutationEntry /// or UnknownCSN if it's not committed (yet) or RolledBackCSN if it's rolled back or PrehistoricCSN if there is no transaction. CSN csn = Tx::UnknownCSN; - /// From query context. - LightweightMutationProjectionMode lightweight_delete_projection_mode; - /// Create a new entry and write it to a temporary file. MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number, - const TransactionID & tid_, const WriteSettings & settings, - const LightweightMutationProjectionMode & lightweight_delete_projection_mode_); + const TransactionID & tid_, const WriteSettings & settings); MergeTreeMutationEntry(const MergeTreeMutationEntry &) = delete; MergeTreeMutationEntry(MergeTreeMutationEntry &&) = default; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c0afd781c7e..a458a21ca1b 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -215,6 +215,7 @@ struct Settings; M(Float, primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns, 0.9f, "If the value of a column of the primary key in data part changes at least in this ratio of times, skip loading next columns in memory. This allows to save memory usage by not loading useless columns of the primary key.", 0) \ /** Projection settings. */ \ M(UInt64, max_projections, 25, "The maximum number of merge tree projections.", 0) \ + M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop projections of this table's relevant parts.", 0) \ #define MAKE_OBSOLETE_MERGE_TREE_SETTING(M, TYPE, NAME, DEFAULT) \ M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", BaseSettingsHelpers::Flags::OBSOLETE) diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 19aa63d90a2..20f387137e7 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -140,7 +140,6 @@ ContextMutablePtr MutatePlainMergeTreeTask::createTaskContext() const auto queryId = getQueryId(); context->setCurrentQueryId(queryId); context->setBackgroundOperationTypeForContext(ClientInfo::BackgroundOperationType::MUTATION); - // context->setSetting("lightweight_mutation_projection_mode", merge_mutate_entry->lightweight_delete_projection_mode); return context; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index ed603abd9c3..0734174dbef 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2200,7 +2200,6 @@ bool MutateTask::prepare() context_for_reading->setSetting("allow_asynchronous_read_from_io_pool_for_merge_tree", false); context_for_reading->setSetting("max_streams_for_merge_tree_reading", Field(0)); context_for_reading->setSetting("read_from_filesystem_cache_if_exists_otherwise_bypass_cache", 1); - context_for_reading->setSetting("lightweight_mutation_projection_mode", Field(ctx->context->getSettingsRef().lightweight_mutation_projection_mode)); MutationHelpers::splitAndModifyMutationCommands( ctx->source_part, ctx->metadata_snapshot, @@ -2225,15 +2224,6 @@ bool MutateTask::prepare() ctx->mutating_pipeline_builder = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); ctx->progress_callback = MergeProgressCallback((*ctx->mutate_entry)->ptr(), ctx->watch_prev_elapsed, *ctx->stage_progress); - - // ctx->updated_header.has(RowExistsColumn::name); - // for (const auto & projection : ctx->metadata_snapshot->getProjections()) - // { - // if (!ctx->source_part->hasProjection(projection.name)) - // continue; - - // ctx->materialized_projections.insert(projection.name); - // } } auto single_disk_volume = std::make_shared("volume_" + ctx->future_part->name, ctx->space_reservation->getDisk(), 0); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 7f210779916..8404e5c9cd9 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -517,8 +517,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context { std::lock_guard lock(currently_processing_in_background_mutex); - MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings(), - query_context->getSettingsRef().lightweight_mutation_projection_mode); + MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings()); version = increment.get(); entry.commit(version); String mutation_id = entry.file_name; @@ -1284,17 +1283,12 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( size_t current_ast_elements = 0; auto last_mutation_to_apply = mutations_end_it; - /// Trying to grab it from query context. - LightweightMutationProjectionMode lightweight_delete_projection_mode = LightweightMutationProjectionMode::THROW; - for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { /// Do not squash mutations from different transactions to be able to commit/rollback them independently. if (first_mutation_tid != it->second.tid) break; - lightweight_delete_projection_mode = it->second.lightweight_delete_projection_mode; - size_t commands_size = 0; MutationCommands commands_for_size_validation; for (const auto & command : it->second.commands) @@ -1371,7 +1365,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( future_part->part_format = part->getFormat(); tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}, false), *this, metadata_snapshot, true); - return std::make_shared(future_part, std::move(tagger), commands, txn, lightweight_delete_projection_mode); + return std::make_shared(future_part, std::move(tagger), commands, txn); } } From 68ed5767d795e7b5792fed839198f53d43581c47 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 15 Jul 2024 15:31:17 +0000 Subject: [PATCH 173/644] fix merge problem --- src/Storages/MergeTree/MergeTreeData.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 125b2c8c513..38ca0aed9da 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6160,6 +6160,11 @@ bool MergeTreeData::supportsLightweightDelete() const return true; } +bool MergeTreeData::areAsynchronousInsertsEnabled() const +{ + return getSettings()->async_insert; +} + MergeTreeData::ProjectionPartsVector MergeTreeData::getAllProjectionPartsVector(MergeTreeData::DataPartStateVector * out_states) const { ProjectionPartsVector res; From 7841a8b401d0ddb3f9bce9e5dc03049a65a1067a Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 15 Jul 2024 16:27:38 +0000 Subject: [PATCH 174/644] fixes after review --- docs/en/operations/settings/settings.md | 2 +- .../table-functions/azureBlobStorage.md | 13 ++++ docs/en/sql-reference/table-functions/file.md | 8 +-- docs/en/sql-reference/table-functions/hdfs.md | 4 +- docs/en/sql-reference/table-functions/s3.md | 8 +-- docs/en/sql-reference/table-functions/url.md | 13 ++++ .../ObjectStorage/StorageObjectStorage.cpp | 16 +++-- .../StorageObjectStorageCluster.cpp | 4 +- .../StorageObjectStorageSource.cpp | 17 ++--- .../StorageObjectStorageSource.h | 4 +- src/Storages/StorageFile.cpp | 20 ++---- src/Storages/StorageURL.cpp | 11 +-- src/Storages/VirtualColumnUtils.cpp | 64 +++++++++--------- src/Storages/VirtualColumnUtils.h | 7 +- .../03203_hive_style_partitioning.reference | 5 ++ .../03203_hive_style_partitioning.sh | 13 ++++ .../column0=Elizabeth/sample.parquet | Bin 0 -> 1308 bytes .../partitioning/identifier=2070/email.csv | 5 ++ 18 files changed, 114 insertions(+), 100 deletions(-) create mode 100644 tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth/sample.parquet create mode 100644 tests/queries/0_stateless/data_hive/partitioning/identifier=2070/email.csv diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 84912b4574f..fb076e76bdd 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5611,6 +5611,6 @@ Default value: `1GiB`. ## use_hive_partitioning -Allows the usage of Hive-style partitioning in queries. When enabled, ClickHouse interprets and maintains table partitions in a way that is consistent with the Hive partitioning scheme, which is commonly used in Hadoop ecosystems. +When enabled, ClickHouse will detect Hive-style partitioning in path (`/name=value/`) in file-like table engines [File](../../engines/table-engines/special/file.md#hive-style-partitioning)/[S3](../../engines/table-engines/integrations/s3.md#hive-style-partitioning)/[URL](../../engines/table-engines/special/url.md#hive-style-partitioning)/[HDFS](../../engines/table-engines/integrations/hdfs.md#hive-style-partitioning)/[AzureBlobStorage](../../engines/table-engines/integrations/azureBlobStorage.md#hive-style-partitioning) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. Default value: `0`. diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md index f59fedeb3a2..104ac4e26df 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorage.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -77,3 +77,16 @@ SELECT count(*) FROM azureBlobStorage('DefaultEndpointsProtocol=https;AccountNam **See Also** - [AzureBlobStorage Table Engine](/docs/en/engines/table-engines/integrations/azureBlobStorage.md) + +## Hive-style partitioning {#hive-style-partitioning} + +When setting `use_hive_partitioning` is set to 1, ClickHouse will detect Hive-style partitioning in the path (`/name=value/`) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. + +**Example** + +Use virtual column, created with Hive-style partitioning + +``` sql +SET use_hive_partitioning = 1; +SELECT * from azureBlobStorage(config, storage_account_url='...', container='...', blob_path='http://data/path/date=*/country=*/code=*/*.parquet', format='Parquet', structure='Date DateTime64, Country String, Code UInt64') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; +``` diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 838a7ab61de..0669609a22a 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -208,7 +208,7 @@ SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt3 ## Hive-style partitioning {#hive-style-partitioning} -When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. +When setting `use_hive_partitioning` is set to 1, ClickHouse will detect Hive-style partitioning in the path (`/name=value/`) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. **Example** @@ -216,11 +216,7 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT _specified_column from file('/specified_column=specified_data/file.txt'); -``` - -``` reference -specified_data +SELECT * from file('data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; ``` ## Settings {#settings} diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index fc84c431066..6963d4e4b79 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -101,7 +101,7 @@ FROM hdfs('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name Strin ## Hive-style partitioning {#hive-style-partitioning} -When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. +When setting `use_hive_partitioning` is set to 1, ClickHouse will detect Hive-style partitioning in the path (`/name=value/`) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. **Example** @@ -109,7 +109,7 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT _specified_column from HDFS('hdfs://hdfs1:9000/specified_column=specified_data/file.txt'); +SELECT * from HDFS('hdfs://hdfs1:9000/data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; ``` ``` reference diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 15074a77475..f3ee83afef4 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -276,7 +276,7 @@ FROM s3( ## Hive-style partitioning {#hive-style-partitioning} -When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. +When setting `use_hive_partitioning` is set to 1, ClickHouse will detect Hive-style partitioning in the path (`/name=value/`) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. **Example** @@ -284,11 +284,7 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT _specified_column from HDFS('hdfs://hdfs1:9000/specified_column=specified_data/file.txt'); -``` - -``` reference -specified_data +SELECT * from s3('s3://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; ``` ## Storage Settings {#storage-settings} diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 3bb7aff53a7..596355e2577 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -55,6 +55,19 @@ Character `|` inside patterns is used to specify failover addresses. They are it - `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. +## Hive-style partitioning {#hive-style-partitioning} + +When setting `use_hive_partitioning` is set to 1, ClickHouse will detect Hive-style partitioning in the path (`/name=value/`) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. + +**Example** + +Use virtual column, created with Hive-style partitioning + +``` sql +SET use_hive_partitioning = 1; +SELECT * from url('http://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; +``` + ## Storage Settings {#storage-settings} - [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default. diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 48e9118e321..35cd1492642 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -35,16 +35,19 @@ namespace ErrorCodes std::string StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata, ContextPtr context) { + auto query_settings = configuration->getQuerySettings(context); + /// We don't want to throw an exception if there are no files with specified path. + query_settings.throw_on_zero_files_match = false; auto file_iterator = StorageObjectStorageSource::createFileIterator( configuration, + query_settings, object_storage, distributed_processing, context, {}, // predicate metadata.getColumns().getAll(), // virtual_columns nullptr, // read_keys - {}, // file_progress_callback - true // override_settings_for_hive_partitioning + {} // file_progress_callback ); if (auto file = file_iterator->next(0)) @@ -82,12 +85,10 @@ StorageObjectStorage::StorageObjectStorage( metadata.setConstraints(constraints_); metadata.setComment(comment); - if (sample_path.empty() && context->getSettings().use_hive_partitioning) + if (sample_path.empty()) sample_path = getPathSample(metadata, context); - else if (!context->getSettings().use_hive_partitioning) - sample_path = ""; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context, sample_path)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context, sample_path, format_settings)); setInMemoryMetadata(metadata); } @@ -224,7 +225,7 @@ private: return; auto context = getContext(); iterator_wrapper = StorageObjectStorageSource::createFileIterator( - configuration, object_storage, distributed_processing, + configuration, configuration->getQuerySettings(context), object_storage, distributed_processing, context, predicate, virtual_columns, nullptr, context->getFileProgressCallback()); } }; @@ -376,6 +377,7 @@ std::unique_ptr StorageObjectStorage::createReadBufferIterat { auto file_iterator = StorageObjectStorageSource::createFileIterator( configuration, + configuration->getQuerySettings(context), object_storage, false/* distributed_processing */, context, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 92327b4cde0..a88532e1ea9 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -84,8 +84,8 @@ RemoteQueryExecutor::Extension StorageObjectStorageCluster::getTaskIteratorExten const ActionsDAG::Node * predicate, const ContextPtr & local_context) const { auto iterator = StorageObjectStorageSource::createFileIterator( - configuration, object_storage, /* distributed_processing */false, local_context, - predicate, virtual_columns, nullptr, local_context->getFileProgressCallback()); + configuration, configuration->getQuerySettings(local_context), object_storage, /* distributed_processing */false, + local_context, predicate, virtual_columns, nullptr, local_context->getFileProgressCallback()); auto callback = std::make_shared>([iterator]() mutable -> String { diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 2e5416d1ffd..707e7603368 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -99,14 +99,14 @@ std::string StorageObjectStorageSource::getUniqueStoragePathIdentifier( std::shared_ptr StorageObjectStorageSource::createFileIterator( ConfigurationPtr configuration, + const StorageObjectStorage::QuerySettings & query_settings, ObjectStoragePtr object_storage, bool distributed_processing, const ContextPtr & local_context, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, - std::function file_progress_callback, - bool override_settings_for_hive_partitioning) + std::function file_progress_callback) { if (distributed_processing) return std::make_shared( @@ -117,20 +117,16 @@ std::shared_ptr StorageObjectStorageSourc throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression can not have wildcards inside {} name", configuration->getNamespaceType()); - auto settings = configuration->getQuerySettings(local_context); const bool is_archive = configuration->isArchive(); std::unique_ptr iterator; if (configuration->isPathWithGlobs()) { - bool throw_on_zero_files_match = settings.throw_on_zero_files_match; - if (override_settings_for_hive_partitioning) - throw_on_zero_files_match = false; /// Iterate through disclosed globs and make a source for each file iterator = std::make_unique( object_storage, configuration, predicate, virtual_columns, - local_context, is_archive ? nullptr : read_keys, settings.list_object_keys_size, - throw_on_zero_files_match, file_progress_callback); + local_context, is_archive ? nullptr : read_keys, query_settings.list_object_keys_size, + query_settings.throw_on_zero_files_match, file_progress_callback); } else { @@ -149,7 +145,7 @@ std::shared_ptr StorageObjectStorageSourc iterator = std::make_unique( object_storage, copy_configuration, virtual_columns, is_archive ? nullptr : read_keys, - settings.ignore_non_existent_file, file_progress_callback); + query_settings.ignore_non_existent_file, file_progress_callback); } if (is_archive) @@ -208,8 +204,7 @@ Chunk StorageObjectStorageSource::generate() .size = object_info->isArchive() ? object_info->fileSizeInArchive() : object_info->metadata->size_bytes, .filename = &filename, .last_modified = object_info->metadata->last_modified, - .hive_partitioning_path = object_info->getPath(), - }); + }, read_from_format_info.columns_description, getContext()); const auto & partition_columns = configuration->getPartitionColumns(); if (!partition_columns.empty() && chunk_size && chunk.hasColumns()) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index a99bb068372..ff6d588b364 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -52,14 +52,14 @@ public: static std::shared_ptr createFileIterator( ConfigurationPtr configuration, + const StorageObjectStorage::QuerySettings & query_settings, ObjectStoragePtr object_storage, bool distributed_processing, const ContextPtr & local_context, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, - std::function file_progress_callback = {}, - bool override_settings_for_hive_partitioning = false); + std::function file_progress_callback = {}); static std::string getUniqueStoragePathIdentifier( const Configuration & configuration, diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 047631cbc54..42e27a13ca9 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1108,9 +1108,9 @@ void StorageFile::setStorageMetadata(CommonArguments args) setInMemoryMetadata(storage_metadata); std::string path_for_virtuals; - if (args.getContext()->getSettingsRef().use_hive_partitioning && !paths.empty()) + if (!paths.empty()) path_for_virtuals = paths[0]; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), args.getContext(), path_for_virtuals, format_settings.value_or(FormatSettings{}))); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), args.getContext(), path_for_virtuals, format_settings)); } @@ -1452,10 +1452,6 @@ Chunk StorageFileSource::generate() chunk_size = input_format->getApproxBytesReadForChunk(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - std::string hive_partitioning_path; - if (getContext()->getSettingsRef().use_hive_partitioning) - hive_partitioning_path = current_path; - /// Enrich with virtual columns. VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( chunk, requested_virtual_columns, @@ -1463,9 +1459,8 @@ Chunk StorageFileSource::generate() .path = current_path, .size = current_file_size, .filename = (filename_override.has_value() ? &filename_override.value() : nullptr), - .last_modified = current_file_last_modified, - .hive_partitioning_path = hive_partitioning_path, - }); + .last_modified = current_file_last_modified + }, columns_description, getContext()); return chunk; } @@ -1648,17 +1643,10 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui size_t num_streams = max_num_streams; size_t files_to_read = 0; - Strings paths; if (storage->archive_info) - { files_to_read = storage->archive_info->paths_to_archives.size(); - paths = storage->archive_info->paths_to_archives; - } else - { files_to_read = storage->paths.size(); - paths = storage->paths; - } if (max_num_streams > files_to_read) num_streams = files_to_read; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 5da42638b87..f7560fa7910 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -164,10 +164,7 @@ IStorageURLBase::IStorageURLBase( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - std::string uri_for_partitioning; - if (context_->getSettingsRef().use_hive_partitioning) - uri_for_partitioning = getSampleURI(uri, context_); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_, uri_for_partitioning, format_settings.value_or(FormatSettings{}))); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_, getSampleURI(uri, context_), format_settings)); } @@ -426,9 +423,6 @@ Chunk StorageURLSource::generate() size_t chunk_size = 0; if (input_format) chunk_size = input_format->getApproxBytesReadForChunk(); - std::string hive_partitioning_path; - if (getContext()->getSettingsRef().use_hive_partitioning) - hive_partitioning_path = curr_uri.getPath(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( @@ -436,8 +430,7 @@ Chunk StorageURLSource::generate() { .path = curr_uri.getPath(), .size = current_file_size, - .hive_partitioning_path = hive_partitioning_path, - }); + }, columns_description, getContext()); return chunk; } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 0efa9522ac6..fb5a345f424 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -119,15 +119,25 @@ NameSet getVirtualNamesForFileLikeStorage() return {"_path", "_file", "_size", "_time"}; } -std::map parseHivePartitioningKeysAndValues(const std::string& path) +std::unordered_map parseHivePartitioningKeysAndValues(const std::string& path, const ColumnsDescription & storage_columns) { std::string pattern = "/([^/]+)=([^/]+)"; re2::StringPiece input_piece(path); - std::map key_values; + std::unordered_map key_values; std::string key, value; + std::set used_keys; while (RE2::FindAndConsume(&input_piece, pattern, &key, &value)) - key_values["_" + key] = value; + { + if (used_keys.contains(key)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Link to file with enabled hive-style partitioning contains duplicated key {}, only unique keys required", key); + used_keys.insert(key); + + auto col_name = "_" + key; + while (storage_columns.has(col_name)) + col_name = "_" + col_name; + key_values[col_name] = value; + } return key_values; } @@ -148,17 +158,20 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription add_virtual("_size", makeNullable(std::make_shared())); add_virtual("_time", makeNullable(std::make_shared())); - auto map = parseHivePartitioningKeysAndValues(path); - for (auto& item : map) + if (context->getSettingsRef().use_hive_partitioning) { + auto map = parseHivePartitioningKeysAndValues(path, storage_columns); auto format_settings = format_settings_ ? *format_settings_ : getFormatSettings(context); - auto type = tryInferDataTypeByEscapingRule(item.second, format_settings, FormatSettings::EscapingRule::Raw); - if (type == nullptr) - type = std::make_shared(); - if (type->canBeInsideLowCardinality()) - add_virtual(item.first, std::make_shared(type)); - else - add_virtual(item.first, type); + for (auto & item : map) + { + auto type = tryInferDataTypeByEscapingRule(item.second, format_settings, FormatSettings::EscapingRule::Raw); + if (type == nullptr) + type = std::make_shared(); + if (type->canBeInsideLowCardinality()) + add_virtual(item.first, std::make_shared(type)); + else + add_virtual(item.first, type); + } } return desc; @@ -207,8 +220,6 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const { if (column.name == "_file" || column.name == "_path") block.insert({column.type->createColumn(), column.type, column.name}); - if (!getVirtualNamesForFileLikeStorage().contains(column.name)) - block.insert({column.type->createColumn(), column.type, column.name}); } block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); @@ -222,9 +233,12 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values) + VirtualsForFileLikeStorage virtual_values, ColumnsDescription columns, ContextPtr context) { - auto hive_map = parseHivePartitioningKeysAndValues(virtual_values.hive_partitioning_path); + std::unordered_map hive_map; + if (context->getSettingsRef().use_hive_partitioning) + hive_map = parseHivePartitioningKeysAndValues(virtual_values.path, columns); + for (const auto & virtual_column : requested_virtual_columns) { if (virtual_column.name == "_path") @@ -258,23 +272,9 @@ void addRequestedFileLikeStorageVirtualsToChunk( else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } - else if (!hive_map.empty()) + else if (auto it = hive_map.find(virtual_column.getNameInStorage()); it != hive_map.end()) { - bool contains_virtual_column = std::any_of(hive_map.begin(), hive_map.end(), - [&](const auto& pair) - { - return requested_virtual_columns.contains(pair.first); - }); - - if (!contains_virtual_column) - hive_map.clear(); // If we cannot find any virtual column in requested, we don't add any of them to chunk - - auto it = hive_map.find(virtual_column.getNameInStorage()); - if (it != hive_map.end()) - { - chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), convertFieldToType(Field(it->second), *virtual_column.type))->convertToFullColumnIfConst()); - hive_map.erase(it); - } + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), convertFieldToType(Field(it->second), *virtual_column.type))->convertToFullColumnIfConst()); } } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index aa7d4c4605b..29ec32ab375 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -7,8 +7,6 @@ #include #include -#include -#include #include @@ -81,14 +79,11 @@ struct VirtualsForFileLikeStorage std::optional size { std::nullopt }; const String * filename { nullptr }; std::optional last_modified { std::nullopt }; - const String & hive_partitioning_path = ""; }; -std::map parseFromPath(const std::string& path); - void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values); + VirtualsForFileLikeStorage virtual_values, ColumnsDescription columns = {}, ContextPtr context = {}); } } diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index be43048dd01..fc6da3a55c1 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -33,6 +33,11 @@ Elizabeth Gordon Elizabeth [1,2,3] 42.42 Array(Int64) LowCardinality(Float64) 101 +2070 +4081 +2070 +2070 +1 1 TESTING THE URL PARTITIONING first last Elizabeth diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 58a74a3ca8f..a5f3e36763d 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -36,6 +36,19 @@ SELECT toTypeName(_array), toTypeName(_float) FROM file('$CURDIR/data_hive/parti SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') WHERE _number = 42; """ +$CLICKHOUSE_LOCAL -n -q """ +set use_hive_partitioning = 1; + +SELECT _identifier FROM file('$CURDIR/data_hive/partitioning/identitier=*/email.csv') LIMIT 2; +SELECT __identifier FROM file('$CURDIR/data_hive/partitioning/identitier=*/email.csv') LIMIT 2; +""" + +$CLICKHOUSE_LOCAL -n -q """ +set use_hive_partitioning = 1; + +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth/sample.parquet') LIMIT 10; +""" 2>&1 | grep -c "INCORRECT_DATA" + $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; diff --git a/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_hive/partitioning/identifier=2070/email.csv b/tests/queries/0_stateless/data_hive/partitioning/identifier=2070/email.csv new file mode 100644 index 00000000000..936d995cc64 --- /dev/null +++ b/tests/queries/0_stateless/data_hive/partitioning/identifier=2070/email.csv @@ -0,0 +1,5 @@ +_login_email,_identifier,_first_name,_last_name +laura@example.com,2070,Laura,Grey +craig@example.com,4081,Craig,Johnson +mary@example.com,9346,Mary,Jenkins +jamie@example.com,5079,Jamie,Smith From 899c5a64e078820caf2e68cbaf892d5d39e0af06 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 15 Jul 2024 17:14:11 +0000 Subject: [PATCH 175/644] some more fixes (docs + storageObjectStorage) --- docs/en/sql-reference/table-functions/azureBlobStorage.md | 2 +- docs/en/sql-reference/table-functions/file.md | 2 +- docs/en/sql-reference/table-functions/hdfs.md | 2 +- docs/en/sql-reference/table-functions/s3.md | 2 +- docs/en/sql-reference/table-functions/url.md | 2 +- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 7 ++++--- 6 files changed, 9 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md index 104ac4e26df..6936c807f96 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorage.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -88,5 +88,5 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT * from azureBlobStorage(config, storage_account_url='...', container='...', blob_path='http://data/path/date=*/country=*/code=*/*.parquet', format='Parquet', structure='Date DateTime64, Country String, Code UInt64') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; +SELECT * from azureBlobStorage(config, storage_account_url='...', container='...', blob_path='http://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42; ``` diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 0669609a22a..7908a3cb934 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -216,7 +216,7 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT * from file('data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; +SELECT * from file('data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42; ``` ## Settings {#settings} diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 6963d4e4b79..73fdc263d68 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -109,7 +109,7 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT * from HDFS('hdfs://hdfs1:9000/data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; +SELECT * from HDFS('hdfs://hdfs1:9000/data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42; ``` ``` reference diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index f3ee83afef4..1bd9f38517e 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -284,7 +284,7 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT * from s3('s3://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; +SELECT * from s3('s3://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42; ``` ## Storage Settings {#storage-settings} diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 596355e2577..b4027594e7c 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -65,7 +65,7 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT * from url('http://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; +SELECT * from url('http://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42; ``` ## Storage Settings {#storage-settings} diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 35cd1492642..d2cc73f14d7 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -85,7 +85,7 @@ StorageObjectStorage::StorageObjectStorage( metadata.setConstraints(constraints_); metadata.setComment(comment); - if (sample_path.empty()) + if (sample_path.empty() && context->getSettingsRef().use_hive_partitioning) sample_path = getPathSample(metadata, context); setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context, sample_path, format_settings)); @@ -412,8 +412,9 @@ std::string StorageObjectStorage::resolveFormatFromData( { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); + auto format_and_schema = detectFormatAndReadSchema(format_settings, *iterator, context).second; sample_path = iterator->getLastFilePath(); - return detectFormatAndReadSchema(format_settings, *iterator, context).second; + return format_and_schema; } std::pair StorageObjectStorage::resolveSchemaAndFormatFromData( @@ -425,8 +426,8 @@ std::pair StorageObjectStorage::resolveSchemaAn { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); - sample_path = iterator->getLastFilePath(); auto [columns, format] = detectFormatAndReadSchema(format_settings, *iterator, context); + sample_path = iterator->getLastFilePath(); configuration->format = format; return std::pair(columns, format); } From 7d39535c989e16d818370cbbd9cbea891b21d07a Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 15 Jul 2024 17:20:46 +0000 Subject: [PATCH 176/644] storageObjectStorage small fix --- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 8291327992c..ee1169d2c5c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -400,8 +400,9 @@ ColumnsDescription StorageObjectStorage::resolveSchemaFromData( { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); + auto schema = readSchemaFromFormat(configuration->format, format_settings, *iterator, context); sample_path = iterator->getLastFilePath(); - return readSchemaFromFormat(configuration->format, format_settings, *iterator, context); + return schema; } std::string StorageObjectStorage::resolveFormatFromData( From 86570eef595c56400c3e04e44b74d4abdbfdcb52 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 15 Jul 2024 16:24:02 +0000 Subject: [PATCH 177/644] Extract shared logic to common place from consumers --- src/Storages/Kafka/KafkaConsumer.cpp | 74 ++++------------------- src/Storages/Kafka/KafkaConsumer.h | 3 +- src/Storages/Kafka/KafkaConsumer2.cpp | 61 ++----------------- src/Storages/Kafka/KafkaConsumer2.h | 3 +- src/Storages/Kafka/StorageKafkaCommon.cpp | 69 +++++++++++++++++++++ src/Storages/Kafka/StorageKafkaCommon.h | 12 ++++ 6 files changed, 99 insertions(+), 123 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index f4385163323..1affbbaf8fd 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -21,13 +21,12 @@ namespace CurrentMetrics namespace ProfileEvents { - extern const Event KafkaRebalanceRevocations; - extern const Event KafkaRebalanceAssignments; - extern const Event KafkaRebalanceErrors; - extern const Event KafkaMessagesPolled; - extern const Event KafkaCommitFailures; - extern const Event KafkaCommits; - extern const Event KafkaConsumerErrors; +extern const Event KafkaRebalanceRevocations; +extern const Event KafkaRebalanceAssignments; +extern const Event KafkaRebalanceErrors; +extern const Event KafkaMessagesPolled; +extern const Event KafkaCommitFailures; +extern const Event KafkaCommits; } namespace DB @@ -200,44 +199,9 @@ KafkaConsumer::~KafkaConsumer() // https://github.com/confluentinc/confluent-kafka-go/issues/189 etc. void KafkaConsumer::drain() { - auto start_time = std::chrono::steady_clock::now(); - cppkafka::Error last_error(RD_KAFKA_RESP_ERR_NO_ERROR); - - while (true) - { - auto msg = consumer->poll(100ms); - if (!msg) - break; - - auto error = msg.get_error(); - - if (error) - { - if (msg.is_eof() || error == last_error) - { - break; - } - else - { - LOG_ERROR(log, "Error during draining: {}", error); - setExceptionInfo(error); - } - } - - // i don't stop draining on first error, - // only if it repeats once again sequentially - last_error = error; - - auto ts = std::chrono::steady_clock::now(); - if (std::chrono::duration_cast(ts-start_time) > DRAIN_TIMEOUT_MS) - { - LOG_ERROR(log, "Timeout during draining."); - break; - } - } + StorageKafkaUtils::drainConsumer(*consumer, DRAIN_TIMEOUT_MS, log, [this](const cppkafka::Error & err) { setExceptionInfo(err); }); } - void KafkaConsumer::commit() { auto print_offsets = [this] (const char * prefix, const cppkafka::TopicPartitionList & offsets) @@ -410,7 +374,7 @@ void KafkaConsumer::resetToLastCommitted(const char * msg) { if (!assignment.has_value() || assignment->empty()) { - LOG_TRACE(log, "Not assignned. Can't reset to last committed position."); + LOG_TRACE(log, "Not assigned. Can't reset to last committed position."); return; } auto committed_offset = consumer->get_offsets_committed(consumer->get_assignment()); @@ -474,7 +438,7 @@ ReadBufferPtr KafkaConsumer::consume() // If we're doing a manual select then it's better to get something after a wait, then immediate nothing. if (!assignment.has_value()) { - waited_for_assignment += poll_timeout; // slightly innaccurate, but rough calculation is ok. + waited_for_assignment += poll_timeout; // slightly inaccurate, but rough calculation is ok. if (waited_for_assignment < MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS) { continue; @@ -536,26 +500,12 @@ ReadBufferPtr KafkaConsumer::getNextMessage() return getNextMessage(); } -size_t KafkaConsumer::filterMessageErrors() +void KafkaConsumer::filterMessageErrors() { assert(current == messages.begin()); - size_t skipped = std::erase_if(messages, [this](auto & message) - { - if (auto error = message.get_error()) - { - ProfileEvents::increment(ProfileEvents::KafkaConsumerErrors); - LOG_ERROR(log, "Consumer error: {}", error); - setExceptionInfo(error); - return true; - } - return false; - }); - - if (skipped) - LOG_ERROR(log, "There were {} messages with an error", skipped); - - return skipped; + StorageKafkaUtils::eraseMessageErrors(messages, log, [this](const cppkafka::Error & err) { setExceptionInfo(err); }); + current = messages.begin(); } void KafkaConsumer::resetIfStopped() diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index a3bc97779b3..285f3680213 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -191,8 +191,7 @@ private: void drain(); void cleanUnprocessed(); void resetIfStopped(); - /// Return number of messages with an error. - size_t filterMessageErrors(); + void filterMessageErrors(); ReadBufferPtr getNextMessage(); }; diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 52a4cadd60b..aaefd6fd6f5 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -32,7 +32,6 @@ extern const Event KafkaRebalanceErrors; extern const Event KafkaMessagesPolled; extern const Event KafkaCommitFailures; extern const Event KafkaCommits; -extern const Event KafkaConsumerErrors; } namespace DB @@ -156,40 +155,7 @@ KafkaConsumer2::~KafkaConsumer2() // https://github.com/confluentinc/confluent-kafka-go/issues/189 etc. void KafkaConsumer2::drainConsumerQueue() { - auto start_time = std::chrono::steady_clock::now(); - cppkafka::Error last_error(RD_KAFKA_RESP_ERR_NO_ERROR); - - while (true) - { - auto msg = consumer->poll(100ms); - if (!msg) - break; - - auto error = msg.get_error(); - - if (error) - { - if (msg.is_eof() || error == last_error) - { - break; - } - else - { - LOG_ERROR(log, "Error during draining: {}", error); - } - } - - // i don't stop draining on first error, - // only if it repeats once again sequentially - last_error = error; - - auto ts = std::chrono::steady_clock::now(); - if (std::chrono::duration_cast(ts - start_time) > DRAIN_TIMEOUT_MS) - { - LOG_ERROR(log, "Timeout during draining."); - break; - } - } + StorageKafkaUtils::drainConsumer(*consumer, DRAIN_TIMEOUT_MS, log); } void KafkaConsumer2::pollEvents() @@ -414,31 +380,12 @@ ReadBufferPtr KafkaConsumer2::getNextMessage() return nullptr; } -size_t KafkaConsumer2::filterMessageErrors() +void KafkaConsumer2::filterMessageErrors() { assert(current == messages.begin()); - size_t skipped = std::erase_if( - messages, - [this](auto & message) - { - if (auto error = message.get_error()) - { - ProfileEvents::increment(ProfileEvents::KafkaConsumerErrors); - LOG_ERROR(log, "Consumer error: {}", error); - return true; - } - return false; - }); - - if (skipped) - { - LOG_ERROR(log, "There were {} messages with an error", skipped); - // Technically current is invalidated as soon as we erased a single message - current = messages.begin(); - } - - return skipped; + StorageKafkaUtils::eraseMessageErrors(messages, log); + current = messages.begin(); } void KafkaConsumer2::resetIfStopped() diff --git a/src/Storages/Kafka/KafkaConsumer2.h b/src/Storages/Kafka/KafkaConsumer2.h index 0df37434caf..57b157416c6 100644 --- a/src/Storages/Kafka/KafkaConsumer2.h +++ b/src/Storages/Kafka/KafkaConsumer2.h @@ -151,8 +151,7 @@ private: bool polledDataUnusable(const TopicPartition & topic_partition) const; void drainConsumerQueue(); void resetIfStopped(); - /// Return number of messages with an error. - size_t filterMessageErrors(); + void filterMessageErrors(); ReadBufferPtr getNextMessage(); void initializeQueues(const cppkafka::TopicPartitionList & topic_partitions); diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp index 470e3445d03..35118f88e53 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.cpp +++ b/src/Storages/Kafka/StorageKafkaCommon.cpp @@ -32,9 +32,16 @@ namespace CurrentMetrics extern const Metric KafkaLibrdkafkaThreads; } +namespace ProfileEvents +{ +extern const Event KafkaConsumerErrors; +} + namespace DB { +using namespace std::chrono_literals; + namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -556,6 +563,68 @@ String getDefaultClientId(const StorageID & table_id) { return fmt::format("{}-{}-{}-{}", VERSION_NAME, getFQDNOrHostName(), table_id.database_name, table_id.table_name); } + +void drainConsumer( + cppkafka::Consumer & consumer, const std::chrono::milliseconds drain_timeout, const LoggerPtr & log, ErrorHandler error_handler) +{ + auto start_time = std::chrono::steady_clock::now(); + cppkafka::Error last_error(RD_KAFKA_RESP_ERR_NO_ERROR); + + while (true) + { + auto msg = consumer.poll(100ms); + if (!msg) + break; + + auto error = msg.get_error(); + + if (error) + { + if (msg.is_eof() || error == last_error) + { + break; + } + else + { + LOG_ERROR(log, "Error during draining: {}", error); + error_handler(error); + } + } + + // i don't stop draining on first error, + // only if it repeats once again sequentially + last_error = error; + + auto ts = std::chrono::steady_clock::now(); + if (std::chrono::duration_cast(ts - start_time) > drain_timeout) + { + LOG_ERROR(log, "Timeout during draining."); + break; + } + } +} + +void eraseMessageErrors(Messages & messages, const LoggerPtr & log, ErrorHandler error_handler) +{ + assert(current == messages.begin()); + + size_t skipped = std::erase_if( + messages, + [&](auto & message) + { + if (auto error = message.get_error()) + { + ProfileEvents::increment(ProfileEvents::KafkaConsumerErrors); + LOG_ERROR(log, "Consumer error: {}", error); + error_handler(error); + return true; + } + return false; + }); + + if (skipped) + LOG_ERROR(log, "There were {} messages with an error", skipped); +} } template struct StorageKafkaInterceptors; diff --git a/src/Storages/Kafka/StorageKafkaCommon.h b/src/Storages/Kafka/StorageKafkaCommon.h index f0bae7c9c9c..d51d9d03208 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.h +++ b/src/Storages/Kafka/StorageKafkaCommon.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -66,6 +67,17 @@ namespace StorageKafkaUtils { Names parseTopics(String topic_list); String getDefaultClientId(const StorageID & table_id); + +using ErrorHandler = std::function; + +void drainConsumer( + cppkafka::Consumer & consumer, + std::chrono::milliseconds drain_timeout, + const LoggerPtr & log, + ErrorHandler error_handler = [](const cppkafka::Error & /*err*/) {}); + +using Messages = std::vector; +void eraseMessageErrors(Messages & messages, const LoggerPtr & log, ErrorHandler error_handler = [](const cppkafka::Error & /*err*/) {}); } } From 137c8b3f64976df0b20294176c09b3d991ff7202 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 15 Jul 2024 17:43:00 +0000 Subject: [PATCH 178/644] Extract config loading to `StorageKafkaCommon` --- src/Storages/Kafka/StorageKafka.cpp | 164 ++---------------- src/Storages/Kafka/StorageKafka.h | 12 -- src/Storages/Kafka/StorageKafka2.cpp | 165 ++----------------- src/Storages/Kafka/StorageKafka2.h | 12 -- src/Storages/Kafka/StorageKafkaCommon.cpp | 192 +++++++++++++++++++--- src/Storages/Kafka/StorageKafkaCommon.h | 48 ++++-- src/Storages/Kafka/parseSyslogLevel.cpp | 3 +- 7 files changed, 236 insertions(+), 360 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 0e907187e11..8bcf59d6db8 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -55,10 +54,6 @@ #include #include -#if USE_KRB5 -#include -#endif // USE_KRB5 - namespace CurrentMetrics { extern const Metric KafkaBackgroundReads; @@ -463,65 +458,26 @@ KafkaConsumerPtr StorageKafka::createKafkaConsumer(size_t consumer_number) topics); return kafka_consumer_ptr; } - cppkafka::Configuration StorageKafka::getConsumerConfiguration(size_t consumer_number) { - cppkafka::Configuration conf; - - conf.set("metadata.broker.list", brokers); - conf.set("group.id", group); - if (num_consumers > 1) - { - conf.set("client.id", fmt::format("{}-{}", client_id, consumer_number)); - } - else - { - conf.set("client.id", client_id); - } - conf.set("client.software.name", VERSION_NAME); - conf.set("client.software.version", VERSION_DESCRIBE); - conf.set("auto.offset.reset", "earliest"); // If no offset stored for this group, read all messages from the start - - // that allows to prevent fast draining of the librdkafka queue - // during building of single insert block. Improves performance - // significantly, but may lead to bigger memory consumption. - size_t default_queued_min_messages = 100000; // must be greater than or equal to default - size_t max_allowed_queued_min_messages = 10000000; // must be less than or equal to max allowed value - conf.set("queued.min.messages", std::min(std::max(getMaxBlockSize(), default_queued_min_messages), max_allowed_queued_min_messages)); - - updateGlobalConfiguration(conf); - updateConsumerConfiguration(conf); - - // those settings should not be changed by users. - conf.set("enable.auto.commit", "false"); // We manually commit offsets after a stream successfully finished - conf.set("enable.auto.offset.store", "false"); // Update offset automatically - to commit them all at once. - conf.set("enable.partition.eof", "false"); // Ignore EOF messages - - for (auto & property : conf.get_all()) - { - LOG_TRACE(log, "Consumer set property {}:{}", property.first, property.second); - } - - return conf; + KafkaConfigLoader::ConsumerConfigParams params{ + {getContext()->getConfigRef(), collection_name, topics, log}, + brokers, + group, + num_consumers > 1, + consumer_number, + client_id, + getMaxBlockSize()}; + return KafkaConfigLoader::getConsumerConfiguration(*this, params); } cppkafka::Configuration StorageKafka::getProducerConfiguration() { - cppkafka::Configuration conf; - conf.set("metadata.broker.list", brokers); - conf.set("client.id", client_id); - conf.set("client.software.name", VERSION_NAME); - conf.set("client.software.version", VERSION_DESCRIBE); - - updateGlobalConfiguration(conf); - updateProducerConfiguration(conf); - - for (auto & property : conf.get_all()) - { - LOG_TRACE(log, "Producer set property {}:{}", property.first, property.second); - } - - return conf; + KafkaConfigLoader::ProducerConfigParams params{ + {getContext()->getConfigRef(), collection_name, topics, log}, + brokers, + client_id}; + return KafkaConfigLoader::getProducerConfiguration(*this, params); } void StorageKafka::cleanConsumers() @@ -599,98 +555,6 @@ size_t StorageKafka::getPollTimeoutMillisecond() const : getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); } -void StorageKafka::updateGlobalConfiguration(cppkafka::Configuration & kafka_config) -{ - const auto & config = getContext()->getConfigRef(); - KafkaConfigLoader::loadFromConfig(kafka_config, config, collection_name, KafkaConfigLoader::CONFIG_KAFKA_TAG, topics); - -#if USE_KRB5 - if (kafka_config.has_property("sasl.kerberos.kinit.cmd")) - LOG_WARNING(log, "sasl.kerberos.kinit.cmd configuration parameter is ignored."); - - kafka_config.set("sasl.kerberos.kinit.cmd",""); - kafka_config.set("sasl.kerberos.min.time.before.relogin","0"); - - if (kafka_config.has_property("sasl.kerberos.keytab") && kafka_config.has_property("sasl.kerberos.principal")) - { - String keytab = kafka_config.get("sasl.kerberos.keytab"); - String principal = kafka_config.get("sasl.kerberos.principal"); - LOG_DEBUG(log, "Running KerberosInit"); - try - { - kerberosInit(keytab,principal); - } - catch (const Exception & e) - { - LOG_ERROR(log, "KerberosInit failure: {}", getExceptionMessage(e, false)); - } - LOG_DEBUG(log, "Finished KerberosInit"); - } -#else // USE_KRB5 - if (kafka_config.has_property("sasl.kerberos.keytab") || kafka_config.has_property("sasl.kerberos.principal")) - LOG_WARNING(log, "Ignoring Kerberos-related parameters because ClickHouse was built without krb5 library support."); -#endif // USE_KRB5 - // No need to add any prefix, messages can be distinguished - kafka_config.set_log_callback( - [this](cppkafka::KafkaHandleBase & handle, int level, const std::string & facility, const std::string & message) - { - auto [poco_level, client_logs_level] = parseSyslogLevel(level); - const auto & kafka_object_config = handle.get_configuration(); - const std::string client_id_key{"client.id"}; - chassert(kafka_object_config.has_property(client_id_key) && "Kafka configuration doesn't have expected client.id set"); - LOG_IMPL( - log, - client_logs_level, - poco_level, - "[client.id:{}] [rdk:{}] {}", - kafka_object_config.get(client_id_key), - facility, - message); - }); - - /// NOTE: statistics should be consumed, otherwise it creates too much - /// entries in the queue, that leads to memory leak and slow shutdown. - if (!kafka_config.has_property("statistics.interval.ms")) - { - // every 3 seconds by default. set to 0 to disable. - kafka_config.set("statistics.interval.ms", "3000"); - } - - // Configure interceptor to change thread name - // - // TODO: add interceptors support into the cppkafka. - // XXX: rdkafka uses pthread_set_name_np(), but glibc-compatibliity overrides it to noop. - { - // This should be safe, since we wait the rdkafka object anyway. - void * self = static_cast(this); - - int status; - - status = rd_kafka_conf_interceptor_add_on_new(kafka_config.get_handle(), - "init", StorageKafkaInterceptors::rdKafkaOnNew, self); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - LOG_ERROR(log, "Cannot set new interceptor due to {} error", status); - - // cppkafka always copy the configuration - status = rd_kafka_conf_interceptor_add_on_conf_dup(kafka_config.get_handle(), - "init", StorageKafkaInterceptors::rdKafkaOnConfDup, self); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - LOG_ERROR(log, "Cannot set dup conf interceptor due to {} error", status); - } -} - -void StorageKafka::updateConsumerConfiguration(cppkafka::Configuration & kafka_config) -{ - const auto & config = getContext()->getConfigRef(); - KafkaConfigLoader::loadConsumerConfig(kafka_config, config, collection_name, KafkaConfigLoader::CONFIG_KAFKA_TAG, topics); -} - -void StorageKafka::updateProducerConfiguration(cppkafka::Configuration & kafka_config) -{ - const auto & config = getContext()->getConfigRef(); - KafkaConfigLoader::loadProducerConfig(kafka_config, config, collection_name, KafkaConfigLoader::CONFIG_KAFKA_TAG, topics); -} - bool StorageKafka::checkDependencies(const StorageID & table_id) { // Check if all dependencies are attached diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 31a5744ee2a..74793292224 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -149,18 +149,6 @@ private: std::atomic shutdown_called = false; - // Load Kafka global configuration - // https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md#global-configuration-properties - void updateGlobalConfiguration(cppkafka::Configuration & kafka_config); - // Load Kafka properties from consumer configuration - // NOTE: librdkafka allow to set a consumer property to a producer and vice versa, - // but a warning will be generated e.g: - // "Configuration property session.timeout.ms is a consumer property and - // will be ignored by this producer instance" - void updateConsumerConfiguration(cppkafka::Configuration & kafka_config); - // Load Kafka properties from producer configuration - void updateProducerConfiguration(cppkafka::Configuration & kafka_config); - void threadFunc(size_t idx); size_t getPollMaxBatchSize() const; diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 428426d9e6a..d980df9d278 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -29,7 +29,6 @@ #include #include #include -#include #include #include #include @@ -51,10 +50,6 @@ #include #include -#if USE_KRB5 -# include -#endif // USE_KRB5 - #include #include #include @@ -478,54 +473,27 @@ KafkaConsumer2Ptr StorageKafka2::createConsumer(size_t consumer_number) cppkafka::Configuration StorageKafka2::getConsumerConfiguration(size_t consumer_number) { - cppkafka::Configuration conf; - - conf.set("metadata.broker.list", brokers); - conf.set("group.id", group); - if (num_consumers > 1) - conf.set("client.id", fmt::format("{}-{}", client_id, consumer_number)); - else - conf.set("client.id", client_id); - conf.set("client.software.name", VERSION_NAME); - conf.set("client.software.version", VERSION_DESCRIBE); - conf.set("auto.offset.reset", "earliest"); // If no offset stored for this group, read all messages from the start - - // that allows to prevent fast draining of the librdkafka queue - // during building of single insert block. Improves performance - // significantly, but may lead to bigger memory consumption. - size_t default_queued_min_messages = 100000; // must be greater than or equal to default - size_t max_allowed_queued_min_messages = 10000000; // must be less than or equal to max allowed value - conf.set("queued.min.messages", std::min(std::max(getMaxBlockSize(), default_queued_min_messages), max_allowed_queued_min_messages)); - - updateGlobalConfiguration(conf); - updateConsumerConfiguration(conf); - - // those settings should not be changed by users. - conf.set("enable.auto.commit", "false"); // We manually commit offsets after a stream successfully finished - conf.set("enable.auto.offset.store", "false"); // Update offset automatically - to commit them all at once. - conf.set("enable.partition.eof", "false"); // Ignore EOF messages - - for (auto & property : conf.get_all()) - LOG_TEST(log, "Consumer set property {}:{}", property.first, property.second); - - return conf; + KafkaConfigLoader::ConsumerConfigParams params{ + {getContext()->getConfigRef(), collection_name, topics, log}, + brokers, + group, + num_consumers > 1, + consumer_number, + client_id, + getMaxBlockSize()}; + auto kafka_config = KafkaConfigLoader::getConsumerConfiguration(*this, params); + // It is disabled, because in case of no materialized views are attached, it can cause live memory leak. To enable it, a similar cleanup mechanism must be introduced as for StorageKafka. + kafka_config.set("statistics.interval.ms", "0"); + return kafka_config; } cppkafka::Configuration StorageKafka2::getProducerConfiguration() { - cppkafka::Configuration conf; - conf.set("metadata.broker.list", brokers); - conf.set("client.id", client_id); - conf.set("client.software.name", VERSION_NAME); - conf.set("client.software.version", VERSION_DESCRIBE); - - updateGlobalConfiguration(conf); - updateProducerConfiguration(conf); - - for (auto & property : conf.get_all()) - LOG_TEST(log, "Producer set property {}:{}", property.first, property.second); - - return conf; + KafkaConfigLoader::ProducerConfigParams params{ + {getContext()->getConfigRef(), collection_name, topics, log}, + brokers, + client_id}; + return KafkaConfigLoader::getProducerConfiguration(*this, params); } size_t StorageKafka2::getMaxBlockSize() const @@ -548,105 +516,6 @@ size_t StorageKafka2::getPollTimeoutMillisecond() const : getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); } -void StorageKafka2::updateGlobalConfiguration(cppkafka::Configuration & kafka_config) -{ - const auto & config = getContext()->getConfigRef(); - KafkaConfigLoader::loadFromConfig(kafka_config, config, collection_name, KafkaConfigLoader::CONFIG_KAFKA_TAG, topics); - -#if USE_KRB5 - if (kafka_config.has_property("sasl.kerberos.kinit.cmd")) - LOG_WARNING(log, "sasl.kerberos.kinit.cmd configuration parameter is ignored."); - - kafka_config.set("sasl.kerberos.kinit.cmd", ""); - kafka_config.set("sasl.kerberos.min.time.before.relogin", "0"); - - if (kafka_config.has_property("sasl.kerberos.keytab") && kafka_config.has_property("sasl.kerberos.principal")) - { - String keytab = kafka_config.get("sasl.kerberos.keytab"); - String principal = kafka_config.get("sasl.kerberos.principal"); - LOG_DEBUG(log, "Running KerberosInit"); - try - { - kerberosInit(keytab, principal); - } - catch (const Exception & e) - { - LOG_ERROR(log, "KerberosInit failure: {}", getExceptionMessage(e, false)); - } - LOG_DEBUG(log, "Finished KerberosInit"); - } -#else // USE_KRB5 - if (kafka_config.has_property("sasl.kerberos.keytab") || kafka_config.has_property("sasl.kerberos.principal")) - LOG_WARNING(log, "Ignoring Kerberos-related parameters because ClickHouse was built without krb5 library support."); -#endif // USE_KRB5 - // No need to add any prefix, messages can be distinguished - kafka_config.set_log_callback( - [this](cppkafka::KafkaHandleBase & handle, int level, const std::string & facility, const std::string & message) - { - auto [poco_level, client_logs_level] = parseSyslogLevel(level); - const auto & kafka_object_config = handle.get_configuration(); - const std::string client_id_key{"client.id"}; - chassert(kafka_object_config.has_property(client_id_key) && "Kafka configuration doesn't have expected client.id set"); - LOG_IMPL( - log, - client_logs_level, - poco_level, - "[client.id:{}] [rdk:{}] {}", - kafka_object_config.get(client_id_key), - facility, - message); - }); - - /// NOTE: statistics should be consumed, otherwise it creates too much - /// entries in the queue, that leads to memory leak and slow shutdown. - if (!kafka_config.has_property("statistics.interval.ms")) - { - // every 3 seconds by default. set to 0 to disable. - kafka_config.set("statistics.interval.ms", "3000"); - } - - // Configure interceptor to change thread name - // - // TODO: add interceptors support into the cppkafka. - // XXX: rdkafka uses pthread_set_name_np(), but glibc-compatibility overrides it to noop. - { - // This should be safe, since we wait the rdkafka object anyway. - void * self = static_cast(this); - - int status; - - status = rd_kafka_conf_interceptor_add_on_new(kafka_config.get_handle(), "init", StorageKafkaInterceptors::rdKafkaOnNew, self); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - LOG_ERROR(log, "Cannot set new interceptor due to {} error", status); - - // cppkafka always copy the configuration - status = rd_kafka_conf_interceptor_add_on_conf_dup( - kafka_config.get_handle(), "init", StorageKafkaInterceptors::rdKafkaOnConfDup, self); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - LOG_ERROR(log, "Cannot set dup conf interceptor due to {} error", status); - } -} - -void StorageKafka2::updateConsumerConfiguration(cppkafka::Configuration & kafka_config) -{ - const auto & config = getContext()->getConfigRef(); - KafkaConfigLoader::loadConsumerConfig(kafka_config, config, collection_name, KafkaConfigLoader::CONFIG_KAFKA_TAG, topics); -} - -void StorageKafka2::updateProducerConfiguration(cppkafka::Configuration & kafka_config) -{ - const auto & config = getContext()->getConfigRef(); - KafkaConfigLoader::loadProducerConfig(kafka_config, config, collection_name, KafkaConfigLoader::CONFIG_KAFKA_TAG, topics); -} - -String StorageKafka2::getConfigPrefix() const -{ - if (!collection_name.empty()) - return "named_collections." + collection_name + "." - + String{KafkaConfigLoader::CONFIG_KAFKA_TAG}; /// Add one more level to separate librdkafka configuration. - return String{KafkaConfigLoader::CONFIG_KAFKA_TAG}; -} - bool StorageKafka2::checkDependencies(const StorageID & table_id) { // Check if all dependencies are attached diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index d7909df1b2c..2daae8e30d4 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -176,19 +176,7 @@ private: // Returns full producer related configuration, also the configuration // contains global kafka properties. cppkafka::Configuration getProducerConfiguration(); - // Load Kafka global configuration - // https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md#global-configuration-properties - void updateGlobalConfiguration(cppkafka::Configuration & kafka_config); - // Load Kafka properties from consumer configuration - // NOTE: librdkafka allow to set a consumer property to a producer and vice versa, - // but a warning will be generated e.g: - // "Configuration property session.timeout.ms is a consumer property and - // will be ignored by this producer instance" - void updateConsumerConfiguration(cppkafka::Configuration & kafka_config); - // Load Kafka properties from producer configuration - void updateProducerConfiguration(cppkafka::Configuration & kafka_config); - String getConfigPrefix() const; void threadFunc(size_t idx); size_t getPollMaxBatchSize() const; diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp index 35118f88e53..a2cb397f777 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.cpp +++ b/src/Storages/Kafka/StorageKafkaCommon.cpp @@ -1,3 +1,4 @@ +#include #include @@ -9,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -26,6 +28,9 @@ #include #include +#if USE_KRB5 +# include +#endif // USE_KRB5 namespace CurrentMetrics { @@ -228,17 +233,18 @@ void loadTopicConfig(cppkafka::Configuration & kafka_config, const Poco::Util::A } /// Read server configuration into cppkafka configuration, used by global configuration and by legacy per-topic configuration -void KafkaConfigLoader::loadFromConfig(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & collection_name, const String & config_prefix, const Names & topics) +static void +loadFromConfig(cppkafka::Configuration & kafka_config, const KafkaConfigLoader::LoadConfigParams & params, const String & config_prefix) { - if (!collection_name.empty()) + if (!params.collection_name.empty()) { - loadNamedCollectionConfig(kafka_config, collection_name, config_prefix); + loadNamedCollectionConfig(kafka_config, params.collection_name, config_prefix); return; } /// Read all tags one level below Poco::Util::AbstractConfiguration::Keys tags; - config.keys(config_prefix, tags); + params.config.keys(config_prefix, tags); for (const auto & tag : tags) { @@ -263,16 +269,16 @@ void KafkaConfigLoader::loadFromConfig(cppkafka::Configuration & kafka_config, c // // Advantages: The period restriction no longer applies (e.g. sports.football will work), everything // Kafka-related is below . - for (const auto & topic : topics) + for (const auto & topic : params.topics) { /// Read topic name between ... const String kafka_topic_path = config_prefix + "." + tag; const String kafka_topic_name_path = kafka_topic_path + "." + KafkaConfigLoader::CONFIG_NAME_TAG; - const String topic_name = config.getString(kafka_topic_name_path); + const String topic_name = params.config.getString(kafka_topic_name_path); if (topic_name != topic) continue; - loadTopicConfig(kafka_config, config, collection_name, kafka_topic_path, topic); + loadTopicConfig(kafka_config, params.config, params.collection_name, kafka_topic_path, topic); } continue; } @@ -285,36 +291,184 @@ void KafkaConfigLoader::loadFromConfig(cppkafka::Configuration & kafka_config, c // 250 // 100000 // - loadConfigProperty(kafka_config, config, config_prefix, tag); + loadConfigProperty(kafka_config, params.config, config_prefix, tag); } } -void loadLegacyConfigSyntax(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & collection_name, const String & prefix, const Names & topics) +void loadLegacyConfigSyntax( + cppkafka::Configuration & kafka_config, + const Poco::Util::AbstractConfiguration & config, + const String & collection_name, + const Names & topics) { for (const auto & topic : topics) { - const String kafka_topic_path = prefix + "." + KafkaConfigLoader::CONFIG_KAFKA_TAG + "_" + topic; + const String kafka_topic_path = KafkaConfigLoader::CONFIG_KAFKA_TAG + "." + KafkaConfigLoader::CONFIG_KAFKA_TAG + "_" + topic; loadLegacyTopicConfig(kafka_config, config, collection_name, kafka_topic_path); } } -void KafkaConfigLoader::loadConsumerConfig(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & collection_name, const String & prefix, const Names & topics) +static void loadConsumerConfig(cppkafka::Configuration & kafka_config, const KafkaConfigLoader::LoadConfigParams & params) { - const String consumer_path = prefix + "." + CONFIG_KAFKA_CONSUMER_TAG; - loadLegacyConfigSyntax(kafka_config, config, collection_name, prefix, topics); + const String consumer_path = KafkaConfigLoader::CONFIG_KAFKA_TAG + "." + KafkaConfigLoader::CONFIG_KAFKA_CONSUMER_TAG; + loadLegacyConfigSyntax(kafka_config, params.config, params.collection_name, params.topics); // A new syntax has higher priority - loadFromConfig(kafka_config, config, collection_name, consumer_path, topics); + loadFromConfig(kafka_config, params, consumer_path); } -void KafkaConfigLoader::loadProducerConfig(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & collection_name, const String & prefix, const Names & topics) +static void loadProducerConfig(cppkafka::Configuration & kafka_config, const KafkaConfigLoader::LoadConfigParams & params) { - const String producer_path = prefix + "." + CONFIG_KAFKA_PRODUCER_TAG; - loadLegacyConfigSyntax(kafka_config, config, collection_name, prefix, topics); + const String producer_path = KafkaConfigLoader::CONFIG_KAFKA_TAG + "." + KafkaConfigLoader::CONFIG_KAFKA_PRODUCER_TAG; + loadLegacyConfigSyntax(kafka_config, params.config, params.collection_name, params.topics); // A new syntax has higher priority - loadFromConfig(kafka_config, config, collection_name, producer_path, topics); - + loadFromConfig(kafka_config, params, producer_path); } +template +static void updateGlobalConfiguration( + cppkafka::Configuration & kafka_config, TKafkaStorage & storage, const KafkaConfigLoader::LoadConfigParams & params) +{ + loadFromConfig(kafka_config, params, KafkaConfigLoader::CONFIG_KAFKA_TAG); + +#if USE_KRB5 + if (kafka_config.has_property("sasl.kerberos.kinit.cmd")) + LOG_WARNING(params.log, "sasl.kerberos.kinit.cmd configuration parameter is ignored."); + + kafka_config.set("sasl.kerberos.kinit.cmd", ""); + kafka_config.set("sasl.kerberos.min.time.before.relogin", "0"); + + if (kafka_config.has_property("sasl.kerberos.keytab") && kafka_config.has_property("sasl.kerberos.principal")) + { + String keytab = kafka_config.get("sasl.kerberos.keytab"); + String principal = kafka_config.get("sasl.kerberos.principal"); + LOG_DEBUG(params.log, "Running KerberosInit"); + try + { + kerberosInit(keytab, principal); + } + catch (const Exception & e) + { + LOG_ERROR(params.log, "KerberosInit failure: {}", getExceptionMessage(e, false)); + } + LOG_DEBUG(params.log, "Finished KerberosInit"); + } +#else // USE_KRB5 + if (kafka_config.has_property("sasl.kerberos.keytab") || kafka_config.has_property("sasl.kerberos.principal")) + LOG_WARNING(log, "Ignoring Kerberos-related parameters because ClickHouse was built without krb5 library support."); +#endif // USE_KRB5 + // No need to add any prefix, messages can be distinguished + kafka_config.set_log_callback( + [log = params.log](cppkafka::KafkaHandleBase & handle, int level, const std::string & facility, const std::string & message) + { + auto [poco_level, client_logs_level] = parseSyslogLevel(level); + const auto & kafka_object_config = handle.get_configuration(); + const std::string client_id_key{"client.id"}; + chassert(kafka_object_config.has_property(client_id_key) && "Kafka configuration doesn't have expected client.id set"); + LOG_IMPL( + log, + client_logs_level, + poco_level, + "[client.id:{}] [rdk:{}] {}", + kafka_object_config.get(client_id_key), + facility, + message); + }); + + /// NOTE: statistics should be consumed, otherwise it creates too much + /// entries in the queue, that leads to memory leak and slow shutdown. + if (!kafka_config.has_property("statistics.interval.ms")) + { + // every 3 seconds by default. set to 0 to disable. + kafka_config.set("statistics.interval.ms", "3000"); + } + // Configure interceptor to change thread name + // + // TODO: add interceptors support into the cppkafka. + // XXX: rdkafka uses pthread_set_name_np(), but glibc-compatibliity overrides it to noop. + { + // This should be safe, since we wait the rdkafka object anyway. + void * self = static_cast(&storage); + + int status; + + status = rd_kafka_conf_interceptor_add_on_new( + kafka_config.get_handle(), "init", StorageKafkaInterceptors::rdKafkaOnNew, self); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(params.log, "Cannot set new interceptor due to {} error", status); + + // cppkafka always copy the configuration + status = rd_kafka_conf_interceptor_add_on_conf_dup( + kafka_config.get_handle(), "init", StorageKafkaInterceptors::rdKafkaOnConfDup, self); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(params.log, "Cannot set dup conf interceptor due to {} error", status); + } +} + +template +cppkafka::Configuration KafkaConfigLoader::getConsumerConfiguration(TKafkaStorage & storage, const ConsumerConfigParams & params) +{ + cppkafka::Configuration conf; + + conf.set("metadata.broker.list", params.brokers); + conf.set("group.id", params.group); + if (params.multiple_consumers) + conf.set("client.id", fmt::format("{}-{}", params.client_id, params.consumer_number)); + else + conf.set("client.id", params.client_id); + conf.set("client.software.name", VERSION_NAME); + conf.set("client.software.version", VERSION_DESCRIBE); + conf.set("auto.offset.reset", "earliest"); // If no offset stored for this group, read all messages from the start + + // that allows to prevent fast draining of the librdkafka queue + // during building of single insert block. Improves performance + // significantly, but may lead to bigger memory consumption. + size_t default_queued_min_messages = 100000; // must be greater than or equal to default + size_t max_allowed_queued_min_messages = 10000000; // must be less than or equal to max allowed value + conf.set( + "queued.min.messages", std::min(std::max(params.max_block_size, default_queued_min_messages), max_allowed_queued_min_messages)); + + updateGlobalConfiguration(conf, storage, params); + loadConsumerConfig(conf, params); + + // those settings should not be changed by users. + conf.set("enable.auto.commit", "false"); // We manually commit offsets after a stream successfully finished + conf.set("enable.auto.offset.store", "false"); // Update offset automatically - to commit them all at once. + conf.set("enable.partition.eof", "false"); // Ignore EOF messages + + for (auto & property : conf.get_all()) + { + LOG_TRACE(params.log, "Consumer set property {}:{}", property.first, property.second); + } + + return conf; +} + +template cppkafka::Configuration KafkaConfigLoader::getConsumerConfiguration(StorageKafka & storage, const ConsumerConfigParams & params); +template cppkafka::Configuration KafkaConfigLoader::getConsumerConfiguration(StorageKafka2 & storage, const ConsumerConfigParams & params); + +template +cppkafka::Configuration KafkaConfigLoader::getProducerConfiguration(TKafkaStorage & storage, const ProducerConfigParams & params) +{ + cppkafka::Configuration conf; + conf.set("metadata.broker.list", params.brokers); + conf.set("client.id", params.client_id); + conf.set("client.software.name", VERSION_NAME); + conf.set("client.software.version", VERSION_DESCRIBE); + + updateGlobalConfiguration(conf, storage, params); + loadProducerConfig(conf, params); + + for (auto & property : conf.get_all()) + { + LOG_TRACE(params.log, "Producer set property {}:{}", property.first, property.second); + } + + return conf; +} + +template cppkafka::Configuration KafkaConfigLoader::getProducerConfiguration(StorageKafka & storage, const ProducerConfigParams & params); +template cppkafka::Configuration KafkaConfigLoader::getProducerConfiguration(StorageKafka2 & storage, const ProducerConfigParams & params); + void registerStorageKafka(StorageFactory & factory) { diff --git a/src/Storages/Kafka/StorageKafkaCommon.h b/src/Storages/Kafka/StorageKafkaCommon.h index d51d9d03208..457247539e0 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.h +++ b/src/Storages/Kafka/StorageKafkaCommon.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -40,27 +41,38 @@ struct KafkaConfigLoader static inline const String CONFIG_NAME_TAG = "name"; static inline const String CONFIG_KAFKA_CONSUMER_TAG = "consumer"; static inline const String CONFIG_KAFKA_PRODUCER_TAG = "producer"; + using LogCallback = cppkafka::Configuration::LogCallback; - static void loadConsumerConfig( - cppkafka::Configuration & kafka_config, - const Poco::Util::AbstractConfiguration & config, - const String & collection_name, - const String & prefix, - const Names & topics); - static void loadProducerConfig( - cppkafka::Configuration & kafka_config, - const Poco::Util::AbstractConfiguration & config, - const String & collection_name, - const String & prefix, - const Names & topics); + struct LoadConfigParams + { + const Poco::Util::AbstractConfiguration & config; + String & collection_name; + const Names & topics; + LoggerPtr & log; + }; - static void loadFromConfig( - cppkafka::Configuration & kafka_config, - const Poco::Util::AbstractConfiguration & config, - const String & collection_name, - const String & config_prefix, - const Names & topics); + struct ConsumerConfigParams : public LoadConfigParams + { + String brokers; + String group; + bool multiple_consumers; + size_t consumer_number; + String client_id; + size_t max_block_size; + }; + + struct ProducerConfigParams : public LoadConfigParams + { + String brokers; + String client_id; + }; + + template + static cppkafka::Configuration getConsumerConfiguration(TKafkaStorage & storage, const ConsumerConfigParams & params); + + template + static cppkafka::Configuration getProducerConfiguration(TKafkaStorage & storage, const ProducerConfigParams & params); }; namespace StorageKafkaUtils diff --git a/src/Storages/Kafka/parseSyslogLevel.cpp b/src/Storages/Kafka/parseSyslogLevel.cpp index 43630a5001f..828cffc311b 100644 --- a/src/Storages/Kafka/parseSyslogLevel.cpp +++ b/src/Storages/Kafka/parseSyslogLevel.cpp @@ -1,4 +1,5 @@ -#include "parseSyslogLevel.h" +#include + #include /// Must be in a separate compilation unit due to macros overlaps: From 47c24c7dbe4394d0176eb91954223e40086b2c63 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 15 Jul 2024 18:11:28 +0000 Subject: [PATCH 179/644] Fix typos --- src/Storages/Kafka/KafkaSource.cpp | 2 +- src/Storages/Kafka/StorageKafka2.cpp | 4 ++-- src/Storages/Kafka/StorageKafkaCommon.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/Kafka/KafkaSource.cpp b/src/Storages/Kafka/KafkaSource.cpp index 9c68107872e..3ddd0d1be8c 100644 --- a/src/Storages/Kafka/KafkaSource.cpp +++ b/src/Storages/Kafka/KafkaSource.cpp @@ -262,7 +262,7 @@ Chunk KafkaSource::generateImpl() // they are not needed here: // and it's misleading to use them here, // as columns 'materialized' that way stays 'ephemeral' - // i.e. will not be stored anythere + // i.e. will not be stored anywhere // If needed any extra columns can be added using DEFAULT they can be added at MV level if needed. auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns()); diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index d980df9d278..3e75f269ec0 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -818,7 +818,7 @@ StorageKafka2::lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const Topi if (code != Coordination::Error::ZNODEEXISTS) zkutil::KeeperMultiException::check(code, ops, responses); - // Possible optimization: check the content of logfiles, if we locked them, then we can clean them up and retry to lock them. + // Possible optimization: check the content of lock files, if we locked them, then we can clean them up and retry to lock them. return std::nullopt; } @@ -1064,7 +1064,7 @@ StorageKafka2::PolledBatchInfo StorageKafka2::pollConsumer( // they are not needed here: // and it's misleading to use them here, // as columns 'materialized' that way stays 'ephemeral' - // i.e. will not be stored anythere + // i.e. will not be stored anywhere // If needed any extra columns can be added using DEFAULT they can be added at MV level if needed. auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns()); diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp index a2cb397f777..2ad3c014b43 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.cpp +++ b/src/Storages/Kafka/StorageKafkaCommon.cpp @@ -384,7 +384,7 @@ static void updateGlobalConfiguration( // Configure interceptor to change thread name // // TODO: add interceptors support into the cppkafka. - // XXX: rdkafka uses pthread_set_name_np(), but glibc-compatibliity overrides it to noop. + // XXX: rdkafka uses pthread_set_name_np(), but glibc-compatibility overrides it to noop. { // This should be safe, since we wait the rdkafka object anyway. void * self = static_cast(&storage); From b3e6383341848a3a3dca94ede24f07e6103a15d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 15 Jul 2024 18:51:53 +0000 Subject: [PATCH 180/644] Extract more common logic --- src/Storages/Kafka/StorageKafka.cpp | 94 +---------------------- src/Storages/Kafka/StorageKafka.h | 4 - src/Storages/Kafka/StorageKafka2.cpp | 61 +-------------- src/Storages/Kafka/StorageKafka2.h | 4 - src/Storages/Kafka/StorageKafkaCommon.cpp | 93 +++++++++++++++++++++- src/Storages/Kafka/StorageKafkaCommon.h | 13 ++++ 6 files changed, 110 insertions(+), 159 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 8bcf59d6db8..a5b709851d7 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -1,12 +1,5 @@ #include -#include -#include -#include -#include -#include -#include -#include #include #include #include @@ -164,7 +157,7 @@ StorageKafka::StorageKafka( , num_consumers(kafka_settings->kafka_num_consumers.value) , log(getLogger("StorageKafka (" + table_id_.table_name + ")")) , intermediate_commit(kafka_settings->kafka_commit_every_batch.value) - , settings_adjustments(createSettingsAdjustments()) + , settings_adjustments(StorageKafkaUtils::createSettingsAdjustments(*kafka_settings, schema_name)) , thread_per_consumer(kafka_settings->kafka_thread_per_consumer.value) , collection_name(collection_name_) { @@ -179,7 +172,7 @@ StorageKafka::StorageKafka( StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); - setVirtuals(createVirtuals(kafka_settings->kafka_handle_error_mode)); + setVirtuals(StorageKafkaUtils::createVirtuals(kafka_settings->kafka_handle_error_mode)); auto task_count = thread_per_consumer ? num_consumers : 1; for (size_t i = 0; i < task_count; ++i) @@ -204,60 +197,6 @@ StorageKafka::StorageKafka( StorageKafka::~StorageKafka() = default; -VirtualColumnsDescription StorageKafka::createVirtuals(StreamingHandleErrorMode handle_error_mode) -{ - VirtualColumnsDescription desc; - - desc.addEphemeral("_topic", std::make_shared(std::make_shared()), ""); - desc.addEphemeral("_key", std::make_shared(), ""); - desc.addEphemeral("_offset", std::make_shared(), ""); - desc.addEphemeral("_partition", std::make_shared(), ""); - desc.addEphemeral("_timestamp", std::make_shared(std::make_shared()), ""); - desc.addEphemeral("_timestamp_ms", std::make_shared(std::make_shared(3)), ""); - desc.addEphemeral("_headers.name", std::make_shared(std::make_shared()), ""); - desc.addEphemeral("_headers.value", std::make_shared(std::make_shared()), ""); - - if (handle_error_mode == StreamingHandleErrorMode::STREAM) - { - desc.addEphemeral("_raw_message", std::make_shared(), ""); - desc.addEphemeral("_error", std::make_shared(), ""); - } - - return desc; -} - -SettingsChanges StorageKafka::createSettingsAdjustments() -{ - SettingsChanges result; - // Needed for backward compatibility - if (!kafka_settings->input_format_skip_unknown_fields.changed) - { - // Always skip unknown fields regardless of the context (JSON or TSKV) - kafka_settings->input_format_skip_unknown_fields = true; - } - - if (!kafka_settings->input_format_allow_errors_ratio.changed) - { - kafka_settings->input_format_allow_errors_ratio = 0.; - } - - if (!kafka_settings->input_format_allow_errors_num.changed) - { - kafka_settings->input_format_allow_errors_num = kafka_settings->kafka_skip_broken_messages.value; - } - - if (!schema_name.empty()) - result.emplace_back("format_schema", schema_name); - - for (const auto & setting : *kafka_settings) - { - const auto & name = setting.getName(); - if (name.find("kafka_") == std::string::npos) - result.emplace_back(name, setting.getValue()); - } - return result; -} - void StorageKafka::read( QueryPlan & query_plan, const Names & column_names, @@ -555,33 +494,6 @@ size_t StorageKafka::getPollTimeoutMillisecond() const : getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); } -bool StorageKafka::checkDependencies(const StorageID & table_id) -{ - // Check if all dependencies are attached - auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); - if (view_ids.empty()) - return true; - - // Check the dependencies are ready? - for (const auto & view_id : view_ids) - { - auto view = DatabaseCatalog::instance().tryGetTable(view_id, getContext()); - if (!view) - return false; - - // If it materialized view, check it's target table - auto * materialized_view = dynamic_cast(view.get()); - if (materialized_view && !materialized_view->tryGetTargetTable()) - return false; - - // Check all its dependencies - if (!checkDependencies(view_id)) - return false; - } - - return true; -} - void StorageKafka::threadFunc(size_t idx) { assert(idx < tasks.size()); @@ -602,7 +514,7 @@ void StorageKafka::threadFunc(size_t idx) // Keep streaming as long as there are attached views and streaming is not cancelled while (!task->stream_cancelled) { - if (!checkDependencies(table_id)) + if (!StorageKafkaUtils::checkDependencies(table_id, getContext())) break; LOG_DEBUG(log, "Started streaming to {} attached views", num_views); diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 74793292224..54d178331bc 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -134,7 +134,6 @@ private: std::mutex thread_statuses_mutex; std::list> thread_statuses; - SettingsChanges createSettingsAdjustments(); /// Creates KafkaConsumer object without real consumer (cppkafka::Consumer) KafkaConsumerPtr createKafkaConsumer(size_t consumer_number); /// Returns full consumer related configuration, also the configuration @@ -156,11 +155,8 @@ private: size_t getPollTimeoutMillisecond() const; bool streamToViews(); - bool checkDependencies(const StorageID & table_id); void cleanConsumers(); - - static VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode); }; } diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 3e75f269ec0..d9fdfd7410f 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -1,13 +1,6 @@ #include #include -#include -#include -#include -#include -#include -#include -#include #include #include #include @@ -122,7 +115,7 @@ StorageKafka2::StorageKafka2( , num_consumers(kafka_settings->kafka_num_consumers.value) , log(getLogger("StorageKafka2 (" + table_id_.getNameForLogs() + ")")) , semaphore(0, static_cast(num_consumers)) - , settings_adjustments(createSettingsAdjustments()) + , settings_adjustments(StorageKafkaUtils::createSettingsAdjustments(*kafka_settings, schema_name)) , thread_per_consumer(kafka_settings->kafka_thread_per_consumer.value) , collection_name(collection_name_) , active_node_identifier(toString(ServerUUID::get())) @@ -138,7 +131,7 @@ StorageKafka2::StorageKafka2( StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); - setVirtuals(createVirtuals(kafka_settings->kafka_handle_error_mode)); + setVirtuals(StorageKafkaUtils::createVirtuals(kafka_settings->kafka_handle_error_mode)); auto task_count = thread_per_consumer ? num_consumers : 1; for (size_t i = 0; i < task_count; ++i) @@ -157,27 +150,6 @@ StorageKafka2::StorageKafka2( activating_task->deactivate(); } -VirtualColumnsDescription StorageKafka2::createVirtuals(StreamingHandleErrorMode handle_error_mode) -{ - VirtualColumnsDescription desc; - - desc.addEphemeral("_topic", std::make_shared(std::make_shared()), ""); - desc.addEphemeral("_key", std::make_shared(), ""); - desc.addEphemeral("_offset", std::make_shared(), ""); - desc.addEphemeral("_partition", std::make_shared(), ""); - desc.addEphemeral("_timestamp", std::make_shared(std::make_shared()), ""); - desc.addEphemeral("_timestamp_ms", std::make_shared(std::make_shared(3)), ""); - desc.addEphemeral("_headers.name", std::make_shared(std::make_shared()), ""); - desc.addEphemeral("_headers.value", std::make_shared(std::make_shared()), ""); - - if (handle_error_mode == StreamingHandleErrorMode::STREAM) - { - desc.addEphemeral("_raw_message", std::make_shared(), ""); - desc.addEphemeral("_error", std::make_shared(), ""); - } - - return desc; -} void StorageKafka2::partialShutdown() { // This is called in a background task within a catch block, thus this function shouldn't throw @@ -516,33 +488,6 @@ size_t StorageKafka2::getPollTimeoutMillisecond() const : getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); } -bool StorageKafka2::checkDependencies(const StorageID & table_id) -{ - // Check if all dependencies are attached - auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); - if (view_ids.empty()) - return true; - - // Check the dependencies are ready? - for (const auto & view_id : view_ids) - { - auto view = DatabaseCatalog::instance().tryGetTable(view_id, getContext()); - if (!view) - return false; - - // If it materialized view, check it's target table - auto * materialized_view = dynamic_cast(view.get()); - if (materialized_view && !materialized_view->tryGetTargetTable()) - return false; - - // Check all its dependencies - if (!checkDependencies(view_id)) - return false; - } - - return true; -} - namespace { const std::string lock_file_name{"lock"}; @@ -1095,7 +1040,7 @@ void StorageKafka2::threadFunc(size_t idx) while (!task->stream_cancelled && num_created_consumers > 0) { maybe_stall_reason.reset(); - if (!checkDependencies(table_id)) + if (!StorageKafkaUtils::checkDependencies(table_id, getContext())) break; LOG_DEBUG(log, "Started streaming to {} attached views", num_views); diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index 2daae8e30d4..d6e564b76f5 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -196,8 +196,6 @@ private: std::optional streamFromConsumer(ConsumerAndAssignmentInfo & consumer_info); - bool checkDependencies(const StorageID & table_id); - // Returns true if this is the first replica bool createTableIfNotExists(); // Returns true if all of the nodes were cleaned up @@ -226,8 +224,6 @@ private: std::filesystem::path getTopicPartitionPath(const TopicPartition & topic_partition); - - static VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode); }; } diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp index 2ad3c014b43..0c893407014 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.cpp +++ b/src/Storages/Kafka/StorageKafkaCommon.cpp @@ -1,4 +1,3 @@ -#include #include @@ -11,8 +10,16 @@ #include #include #include +#include +#include +#include +#include +#include +#include +#include #include #include +#include #include #include #include @@ -469,7 +476,6 @@ cppkafka::Configuration KafkaConfigLoader::getProducerConfiguration(TKafkaStorag template cppkafka::Configuration KafkaConfigLoader::getProducerConfiguration(StorageKafka & storage, const ProducerConfigParams & params); template cppkafka::Configuration KafkaConfigLoader::getProducerConfiguration(StorageKafka2 & storage, const ProducerConfigParams & params); - void registerStorageKafka(StorageFactory & factory) { auto creator_fn = [](const StorageFactory::Arguments & args) -> std::shared_ptr @@ -779,6 +785,89 @@ void eraseMessageErrors(Messages & messages, const LoggerPtr & log, ErrorHandler if (skipped) LOG_ERROR(log, "There were {} messages with an error", skipped); } + +SettingsChanges createSettingsAdjustments(KafkaSettings & kafka_settings, const String & schema_name) +{ + SettingsChanges result; + // Needed for backward compatibility + if (!kafka_settings.input_format_skip_unknown_fields.changed) + { + // Always skip unknown fields regardless of the context (JSON or TSKV) + kafka_settings.input_format_skip_unknown_fields = true; + } + + if (!kafka_settings.input_format_allow_errors_ratio.changed) + { + kafka_settings.input_format_allow_errors_ratio = 0.; + } + + if (!kafka_settings.input_format_allow_errors_num.changed) + { + kafka_settings.input_format_allow_errors_num = kafka_settings.kafka_skip_broken_messages.value; + } + + if (!schema_name.empty()) + result.emplace_back("format_schema", schema_name); + + for (const auto & setting : kafka_settings) + { + const auto & name = setting.getName(); + if (name.find("kafka_") == std::string::npos) + result.emplace_back(name, setting.getValue()); + } + return result; +} + + +bool checkDependencies(const StorageID & table_id, const ContextPtr& context) +{ + // Check if all dependencies are attached + auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); + if (view_ids.empty()) + return true; + + // Check the dependencies are ready? + for (const auto & view_id : view_ids) + { + auto view = DatabaseCatalog::instance().tryGetTable(view_id, context); + if (!view) + return false; + + // If it materialized view, check it's target table + auto * materialized_view = dynamic_cast(view.get()); + if (materialized_view && !materialized_view->tryGetTargetTable()) + return false; + + // Check all its dependencies + if (!checkDependencies(view_id, context)) + return false; + } + + return true; +} + + +VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode) +{ + VirtualColumnsDescription desc; + + desc.addEphemeral("_topic", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_key", std::make_shared(), ""); + desc.addEphemeral("_offset", std::make_shared(), ""); + desc.addEphemeral("_partition", std::make_shared(), ""); + desc.addEphemeral("_timestamp", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_timestamp_ms", std::make_shared(std::make_shared(3)), ""); + desc.addEphemeral("_headers.name", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_headers.value", std::make_shared(std::make_shared()), ""); + + if (handle_error_mode == StreamingHandleErrorMode::STREAM) + { + desc.addEphemeral("_raw_message", std::make_shared(), ""); + desc.addEphemeral("_error", std::make_shared(), ""); + } + + return desc; +} } template struct StorageKafkaInterceptors; diff --git a/src/Storages/Kafka/StorageKafkaCommon.h b/src/Storages/Kafka/StorageKafkaCommon.h index 457247539e0..dd38ee69675 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.h +++ b/src/Storages/Kafka/StorageKafkaCommon.h @@ -2,13 +2,16 @@ #include #include +#include #include #include #include #include #include #include +#include #include +#include namespace Poco { @@ -20,6 +23,10 @@ namespace Util namespace DB { + +struct KafkaSettings; +class VirtualColumnsDescription; + template struct StorageKafkaInterceptors { @@ -90,6 +97,12 @@ void drainConsumer( using Messages = std::vector; void eraseMessageErrors(Messages & messages, const LoggerPtr & log, ErrorHandler error_handler = [](const cppkafka::Error & /*err*/) {}); + +SettingsChanges createSettingsAdjustments(KafkaSettings & kafka_settings, const String & schema_name); + +bool checkDependencies(const StorageID & table_id, const ContextPtr& context); + +VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode); } } From 1bd9a1623f246dbf2a3098a4a022b6764aa3094d Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 15 Jul 2024 19:23:11 +0000 Subject: [PATCH 181/644] add throw option in low level --- src/Interpreters/MutationsInterpreter.cpp | 15 +++++++++++++++ src/Interpreters/MutationsInterpreter.h | 1 + src/Storages/IStorage.h | 3 +++ src/Storages/MergeTree/MergeTreeData.cpp | 15 +++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 2 ++ .../03161_lightweight_delete_projection.sql | 16 ++++++++++------ 6 files changed, 46 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 480c6736bc5..b61f7f78885 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -344,6 +344,11 @@ bool MutationsInterpreter::Source::hasProjection(const String & name) const return part && part->hasProjection(name); } +bool MutationsInterpreter::Source::hasProjection() const +{ + return part && part->hasProjection(); +} + bool MutationsInterpreter::Source::hasBrokenProjection(const String & name) const { return part && part->hasBrokenProjection(name); @@ -491,6 +496,16 @@ static void validateUpdateColumns( { if (!source.supportsLightweightDelete()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); + + if (const MergeTreeData * merge_tree_data = source.getMergeTreeData(); merge_tree_data != nullptr) + { + if (merge_tree_data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW + && merge_tree_data->hasProjection()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "DELETE query is not supported for table {} as it has projections. " + "User should drop all the projections manually before running the query", + source.getStorage()->getStorageID().getFullTableName()); + } } else if (virtual_columns.tryGet(column_name)) { diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 6aaa233cda3..b792a33f904 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -126,6 +126,7 @@ public: bool materializeTTLRecalculateOnly() const; bool hasSecondaryIndex(const String & name) const; bool hasProjection(const String & name) const; + bool hasProjection() const; bool hasBrokenProjection(const String & name) const; bool isCompactPart() const; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 991c8ff64af..d302fcb26a7 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -259,6 +259,9 @@ public: /// Return true if there is at least one part containing lightweight deleted mask. virtual bool hasLightweightDeletedMask() const { return false; } + /// Return true if storage has any projection. + virtual bool hasProjection() const { return false; } + /// Return true if storage can execute lightweight delete mutations. virtual bool supportsLightweightDelete() const { return false; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 38ca0aed9da..78a551591a6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6160,6 +6160,21 @@ bool MergeTreeData::supportsLightweightDelete() const return true; } +bool MergeTreeData::hasProjection() const +{ + auto lock = lockParts(); + for (const auto & part : data_parts_by_info) + { + if (part->getState() == MergeTreeDataPartState::Outdated + || part->getState() == MergeTreeDataPartState::Deleting) + continue; + + if (part->hasProjection()) + return true; + } + return false; +} + bool MergeTreeData::areAsynchronousInsertsEnabled() const { return getSettings()->async_insert; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index d880928098b..7076b680521 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -439,6 +439,8 @@ public: bool supportsLightweightDelete() const override; + bool hasProjection() const override; + bool areAsynchronousInsertsEnabled() const override; bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override; diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index bfeb0127fa4..16a7468234b 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -13,11 +13,13 @@ SETTINGS min_bytes_for_wide_part = 10485760; INSERT INTO users VALUES (1231, 'John', 33); -DELETE FROM users WHERE 1; -- { serverError NOT_IMPLEMENTED } +ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; -DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'throw'; -- { serverError NOT_IMPLEMENTED } +DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } -DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'drop'; +ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; + +DELETE FROM users WHERE uid = 1231; SYSTEM FLUSH LOGS; @@ -45,11 +47,13 @@ SETTINGS min_bytes_for_wide_part = 0; INSERT INTO users VALUES (1231, 'John', 33); -DELETE FROM users WHERE 1; -- { serverError NOT_IMPLEMENTED } +ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; -DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'throw'; -- { serverError NOT_IMPLEMENTED } +DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } -DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'drop'; +ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; + +DELETE FROM users WHERE uid = 1231; SYSTEM FLUSH LOGS; From 12794601921e2d465b27e665b072267b658b8e4c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 15 Jul 2024 22:09:30 +0000 Subject: [PATCH 182/644] fix after review --- .../ObjectStorage/StorageObjectStorage.cpp | 2 +- .../ObjectStorage/StorageObjectStorage.h | 2 +- .../StorageObjectStorageCluster.cpp | 29 ++++++++++++++++++- .../StorageObjectStorageCluster.h | 2 ++ src/Storages/StorageFile.cpp | 5 +--- src/Storages/VirtualColumnUtils.cpp | 6 ++-- src/Storages/VirtualColumnUtils.h | 2 +- .../03203_hive_style_partitioning.reference | 17 +++++++++++ .../03203_hive_style_partitioning.sh | 16 ++++++++++ 9 files changed, 70 insertions(+), 11 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index ee1169d2c5c..ca0ced8dcd3 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -34,7 +34,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -std::string StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata, ContextPtr context) +String StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata, ContextPtr context) { auto query_settings = configuration->getQuerySettings(context); /// We don't want to throw an exception if there are no files with specified path. diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 6ee4ce0c16f..cae0db48f31 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -122,7 +122,7 @@ public: protected: virtual void updateConfiguration(ContextPtr local_context); - std::string getPathSample(StorageInMemoryMetadata metadata, ContextPtr context); + String getPathSample(StorageInMemoryMetadata metadata, ContextPtr context); virtual ReadFromFormatInfo prepareReadingFromFormat( const Strings & requested_columns, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index a88532e1ea9..7f6b3338f9b 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -1,6 +1,8 @@ #include "Storages/ObjectStorage/StorageObjectStorageCluster.h" #include +#include +#include #include #include #include @@ -19,6 +21,28 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +String StorageObjectStorageCluster::getPathSample(StorageInMemoryMetadata metadata, ContextPtr context) +{ + auto query_settings = configuration->getQuerySettings(context); + /// We don't want to throw an exception if there are no files with specified path. + query_settings.throw_on_zero_files_match = false; + auto file_iterator = StorageObjectStorageSource::createFileIterator( + configuration, + query_settings, + object_storage, + false, // distributed_processing + context, + {}, // predicate + metadata.getColumns().getAll(), // virtual_columns + nullptr, // read_keys + {} // file_progress_callback + ); + + if (auto file = file_iterator->next(0)) + return file->getPath(); + return ""; +} + StorageObjectStorageCluster::StorageObjectStorageCluster( const String & cluster_name_, ConfigurationPtr configuration_, @@ -41,7 +65,10 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( metadata.setColumns(columns); metadata.setConstraints(constraints_); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context_)); + if (sample_path.empty() && context_->getSettingsRef().use_hive_partitioning) + sample_path = getPathSample(metadata, context_); + + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context_, sample_path, getFormatSettings(context_))); setInMemoryMetadata(metadata); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index 108aa109616..0088ff28fc2 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -27,6 +27,8 @@ public: RemoteQueryExecutor::Extension getTaskIteratorExtension( const ActionsDAG::Node * predicate, const ContextPtr & context) const override; + String getPathSample(StorageInMemoryMetadata metadata, ContextPtr context); + private: void updateQueryToSendIfNeeded( ASTPtr & query, diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index e2b010bf48f..b43fce370a1 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1109,10 +1109,7 @@ void StorageFile::setStorageMetadata(CommonArguments args) storage_metadata.setComment(args.comment); setInMemoryMetadata(storage_metadata); - std::string path_for_virtuals; - if (!paths.empty()) - path_for_virtuals = paths[0]; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), args.getContext(), path_for_virtuals, format_settings)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), args.getContext(), paths.empty() ? "" : paths[0], format_settings)); } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index fb5a345f424..352fd0d7a76 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -119,18 +119,18 @@ NameSet getVirtualNamesForFileLikeStorage() return {"_path", "_file", "_size", "_time"}; } -std::unordered_map parseHivePartitioningKeysAndValues(const std::string& path, const ColumnsDescription & storage_columns) +std::unordered_map parseHivePartitioningKeysAndValues(const String & path, const ColumnsDescription & storage_columns) { std::string pattern = "/([^/]+)=([^/]+)"; re2::StringPiece input_piece(path); std::unordered_map key_values; std::string key, value; - std::set used_keys; + std::unordered_set used_keys; while (RE2::FindAndConsume(&input_piece, pattern, &key, &value)) { if (used_keys.contains(key)) - throw Exception(ErrorCodes::INCORRECT_DATA, "Link to file with enabled hive-style partitioning contains duplicated key {}, only unique keys required", key); + throw Exception(ErrorCodes::INCORRECT_DATA, "Path '{}' to file with enabled hive-style partitioning contains duplicated partition key {}, only unique keys are allowed", path, key); used_keys.insert(key); auto col_name = "_" + key; diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 29ec32ab375..fef32b149ec 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -51,7 +51,7 @@ NameSet getVirtualNamesForFileLikeStorage(); VirtualColumnsDescription getVirtualsForFileLikeStorage( const ColumnsDescription & storage_columns, const ContextPtr & context, - std::string sample_path = "", + const std::string & sample_path = "", std::optional format_settings_ = std::nullopt); ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index fc6da3a55c1..430a3582f65 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -102,3 +102,20 @@ Jeffery Delgado Elizabeth Clara Cross Elizabeth Elizabeth Gordon Elizabeth OK +TESTING THE S3CLUSTER PARTITIONING +first last Elizabeth +Jorge Frank Elizabeth +Hunter Moreno Elizabeth +Esther Guzman Elizabeth +Dennis Stephens Elizabeth +Nettie Franklin Elizabeth +Stanley Gibson Elizabeth +Eugenia Greer Elizabeth +Jeffery Delgado Elizabeth +Clara Cross Elizabeth +Elizabeth Gordon Elizabeth +Eva Schmidt Elizabeth Schmidt +Samuel Schmidt Elizabeth Schmidt +Eva Schmidt Elizabeth +Samuel Schmidt Elizabeth +Elizabeth Gordon Elizabeth Gordon diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index a5f3e36763d..d2b1f31c85f 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -118,3 +118,19 @@ set use_hive_partitioning = 0; SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -F -q "UNKNOWN_IDENTIFIER" && echo "OK" || echo "FAIL"; + +$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3CLUSTER PARTITIONING'" + +$CLICKHOUSE_CLIENT -n -q """ +set use_hive_partitioning = 1; + +SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; + +SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; + +SELECT *, _column0, _column1 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; + +SELECT *, _column0, _column1 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +""" From 4305a38e7394d7bbf3c3455c3b52b1dc9b86f3c9 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 15 Jul 2024 22:11:08 +0000 Subject: [PATCH 183/644] add include --- src/Storages/VirtualColumnUtils.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 352fd0d7a76..938972cffca 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -41,6 +41,7 @@ #include #include #include +#include #include "Functions/FunctionsLogical.h" #include "Functions/IFunction.h" #include "Functions/IFunctionAdaptors.h" From dcf14e68afbc741fdbf830fc7d01ba84817c0760 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 15 Jul 2024 22:45:21 +0000 Subject: [PATCH 184/644] small fix --- src/Storages/VirtualColumnUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 938972cffca..e84979833ab 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -142,7 +142,7 @@ std::unordered_map parseHivePartitioningKeysAndValues( return key_values; } -VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, const ContextPtr & context, std::string path, std::optional format_settings_) +VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, const ContextPtr & context, const std::string & path, std::optional format_settings_) { VirtualColumnsDescription desc; From 177d006307515e62bbe60082c293c03de4d4cc7c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 15 Jul 2024 23:02:24 +0000 Subject: [PATCH 185/644] add errorcodes --- src/Storages/VirtualColumnUtils.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index e84979833ab..24d0b7160b2 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -56,6 +56,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; +} + namespace VirtualColumnUtils { From 82d283357755e3b667074596ec254ca54598ee5d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 15 Jul 2024 23:29:25 +0000 Subject: [PATCH 186/644] clang tidy fix --- src/Interpreters/Context.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fc1e87e7b7e..2602afd8b78 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -51,7 +51,6 @@ #include #include #include -#include #include #include #include From 8aaf9c1d9824c136bbfc0532b040b2dca7564253 Mon Sep 17 00:00:00 2001 From: morning-color Date: Tue, 16 Jul 2024 11:00:55 +0800 Subject: [PATCH 187/644] Fix tests --- src/Client/Connection.cpp | 2 +- src/Core/ProtocolDefines.h | 4 +- src/Core/Settings.h | 2 +- src/Formats/JSONUtils.cpp | 2 +- src/Processors/Formats/IOutputFormat.h | 16 +- .../Formats/Impl/XMLRowOutputFormat.cpp | 4 +- src/Processors/IProcessor.h | 8 +- src/Processors/LimitTransform.h | 4 +- src/Processors/OffsetTransform.h | 4 +- src/Processors/RowsBeforeStepCounter.h | 4 +- src/Processors/Sources/DelayedSource.h | 8 +- src/Processors/Sources/RemoteSource.cpp | 4 - src/Processors/Sources/RemoteSource.h | 9 +- .../AggregatingInOrderTransform.cpp | 2 + .../Transforms/AggregatingInOrderTransform.h | 4 + .../Transforms/AggregatingTransform.cpp | 4 +- .../Transforms/AggregatingTransform.h | 7 +- .../Transforms/PartialSortingTransform.h | 4 +- src/QueryPipeline/ProfileInfo.cpp | 22 +- src/QueryPipeline/ProfileInfo.h | 4 +- src/QueryPipeline/QueryPipeline.cpp | 15 +- src/Server/TCPHandler.cpp | 2 +- ...74_exact_rows_before_aggregation.reference | 196 +++++++++++++++++- .../03174_exact_rows_before_aggregation.sql | 26 ++- 24 files changed, 276 insertions(+), 81 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 19cd8cc4ee5..d545278220a 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -1279,7 +1279,7 @@ Progress Connection::receiveProgress() const ProfileInfo Connection::receiveProfileInfo() const { ProfileInfo profile_info; - profile_info.read(*in); + profile_info.read(*in, server_revision); return profile_info; } diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 7e6893c6d85..02d54221ed3 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -81,6 +81,8 @@ static constexpr auto DBMS_MIN_REVISION_WITH_TABLE_READ_ONLY_CHECK = 54467; static constexpr auto DBMS_MIN_REVISION_WITH_SYSTEM_KEYWORDS_TABLE = 54468; +static constexpr auto DBMS_MIN_REVISION_WITH_ROWS_BEFORE_AGGREGATION = 54469; + /// Version of ClickHouse TCP protocol. /// /// Should be incremented manually on protocol changes. @@ -88,6 +90,6 @@ static constexpr auto DBMS_MIN_REVISION_WITH_SYSTEM_KEYWORDS_TABLE = 54468; /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54468; +static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54469; } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2296a880bd6..842aa54f620 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1200,7 +1200,7 @@ class IColumn; M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \ \ M(Bool, exact_rows_before_limit, false, "When enabled, ClickHouse will provide exact value for rows_before_limit_at_least statistic, but with the cost that the data before limit will have to be read completely", 0) \ - M(Bool, rows_before_aggregation, false, "When enabled, ClickHouse will provide exact value for rows_before_aggregation_at_least statistic, represents the number of rows read before aggregation", 0) \ + M(Bool, rows_before_aggregation, false, "When enabled, ClickHouse will provide exact value for rows_before_aggregation statistic, represents the number of rows read before aggregation", 0) \ M(UInt64, cross_to_inner_join_rewrite, 1, "Use inner join instead of comma/cross join if there are joining expressions in the WHERE section. Values: 0 - no rewrite, 1 - apply if possible for comma/cross, 2 - force rewrite all comma joins, cross - if possible", 0) \ \ M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \ diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index 363e9344770..017befe5b0e 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -507,7 +507,7 @@ namespace JSONUtils if (applied_aggregation) { writeFieldDelimiter(out, 2); - writeTitle("rows_before_aggregation_at_least", out, 1, " "); + writeTitle("rows_before_aggregation", out, 1, " "); writeIntText(rows_before_aggregation, out); } if (write_statistics) diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index 40ac1317618..e9af4ca7cf5 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -39,17 +39,17 @@ public: virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) { } /// Counter to calculate rows_before_limit_at_least in processors pipeline. - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_counter.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { rows_before_limit_counter.swap(counter); } - /// Value for rows_before_aggregation_at_least field. - virtual void setRowsBeforeAggregation(size_t /*rows_before_limit*/) { } + /// Value for rows_before_aggregation field. + virtual void setRowsBeforeAggregation(size_t /*rows_before_aggregation*/) { } - /// Counter to calculate rows_before_aggregation_at_least in processors pipeline. - void setRowsBeforeAggregationCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_aggregation_counter.swap(counter); } + /// Counter to calculate rows_before_aggregation in processors pipeline. + void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation_counter.swap(counter); } /// Notify about progress. Method could be called from different threads. /// Passed value are delta, that must be summarized. - virtual void onProgress(const Progress & /*progress*/) {} + virtual void onProgress(const Progress & /*progress*/) { } /// Content-Type to set when sending HTTP response. virtual std::string getContentType() const { return "text/plain; charset=UTF-8"; } @@ -192,8 +192,8 @@ protected: bool need_write_prefix = true; bool need_write_suffix = true; - RowsBeforeLimitCounterPtr rows_before_limit_counter; - RowsBeforeAggregationCounterPtr rows_before_aggregation_counter; + RowsBeforeStepCounterPtr rows_before_limit_counter; + RowsBeforeStepCounterPtr rows_before_aggregation_counter; Statistics statistics; private: diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp index 2fd0536ed02..b19fcfd4a4a 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp @@ -224,9 +224,9 @@ void XMLRowOutputFormat::writeRowsBeforeAggregationAtLeast() { if (statistics.applied_aggregation) { - writeCString("\t", *ostr); + writeCString("\t", *ostr); writeIntText(statistics.rows_before_aggregation, *ostr); - writeCString("\n", *ostr); + writeCString("\n", *ostr); } } diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index ccdd6308de5..28050691d9e 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -22,9 +22,7 @@ struct StorageLimits; using StorageLimitsList = std::list; class RowsBeforeStepCounter; -using RowsBeforeLimitCounterPtr = std::shared_ptr; - -using RowsBeforeAggregationCounterPtr = std::shared_ptr; +using RowsBeforeStepCounterPtr = std::shared_ptr; class IProcessor; using ProcessorPtr = std::shared_ptr; @@ -366,11 +364,11 @@ public: /// Set rows_before_limit counter for current processor. /// This counter is used to calculate the number of rows right before any filtration of LimitTransform. - virtual void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr /* counter */) {} + virtual void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr /* counter */) { } /// Set rows_before_aggregation counter for current processor. /// This counter is used to calculate the number of rows right before AggregatingTransform. - virtual void setRowsBeforeAggregationCounter(RowsBeforeAggregationCounterPtr /* counter */) { } + virtual void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr /* counter */) { } protected: virtual void onCancel() {} diff --git a/src/Processors/LimitTransform.h b/src/Processors/LimitTransform.h index 515203f6829..45ae5b0ce81 100644 --- a/src/Processors/LimitTransform.h +++ b/src/Processors/LimitTransform.h @@ -30,7 +30,7 @@ private: std::vector sort_column_positions; UInt64 rows_read = 0; /// including the last read block - RowsBeforeLimitCounterPtr rows_before_limit_at_least; + RowsBeforeStepCounterPtr rows_before_limit_at_least; /// State of port's pair. /// Chunks from different port pairs are not mixed for better cache locality. @@ -71,7 +71,7 @@ public: InputPort & getInputPort() { return inputs.front(); } OutputPort & getOutputPort() { return outputs.front(); } - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_at_least.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { rows_before_limit_at_least.swap(counter); } void setInputPortHasCounter(size_t pos) { ports_data[pos].input_port_has_counter = true; } }; diff --git a/src/Processors/OffsetTransform.h b/src/Processors/OffsetTransform.h index 7ef16518540..04486a6c940 100644 --- a/src/Processors/OffsetTransform.h +++ b/src/Processors/OffsetTransform.h @@ -16,7 +16,7 @@ private: UInt64 offset; UInt64 rows_read = 0; /// including the last read block - RowsBeforeLimitCounterPtr rows_before_limit_at_least; + RowsBeforeStepCounterPtr rows_before_limit_at_least; /// State of port's pair. /// Chunks from different port pairs are not mixed for better cache locality. @@ -45,7 +45,7 @@ public: InputPort & getInputPort() { return inputs.front(); } OutputPort & getOutputPort() { return outputs.front(); } - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_at_least.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { rows_before_limit_at_least.swap(counter); } }; } diff --git a/src/Processors/RowsBeforeStepCounter.h b/src/Processors/RowsBeforeStepCounter.h index d9912bfa076..789731f82bd 100644 --- a/src/Processors/RowsBeforeStepCounter.h +++ b/src/Processors/RowsBeforeStepCounter.h @@ -5,7 +5,7 @@ namespace DB { -/// This class helps to calculate rows_before_limit_at_least and rows_before_aggregation_at_least. +/// This class helps to calculate rows_before_limit_at_least and rows_before_aggregation. class RowsBeforeStepCounter { public: @@ -31,6 +31,6 @@ private: std::atomic_bool has_applied_step = false; }; -using RowsBeforeLimitCounterPtr = std::shared_ptr; +using RowsBeforeStepCounterPtr = std::shared_ptr; } diff --git a/src/Processors/Sources/DelayedSource.h b/src/Processors/Sources/DelayedSource.h index 3138a1ab42a..4ee90e34599 100644 --- a/src/Processors/Sources/DelayedSource.h +++ b/src/Processors/Sources/DelayedSource.h @@ -30,15 +30,15 @@ public: OutputPort * getTotalsPort() { return totals; } OutputPort * getExtremesPort() { return extremes; } - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); } - void setRowsBeforeAggregationCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_aggregation.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { rows_before_limit.swap(counter); } + void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation.swap(counter); } private: QueryPlanResourceHolder resources; Creator creator; Processors processors; - RowsBeforeLimitCounterPtr rows_before_limit; - RowsBeforeLimitCounterPtr rows_before_aggregation; + RowsBeforeStepCounterPtr rows_before_limit; + RowsBeforeStepCounterPtr rows_before_aggregation; /// Outputs for DelayedSource. OutputPort * main = nullptr; diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 9a6fe239ee6..683db2c1c2c 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -50,8 +50,6 @@ RemoteSource::RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation { if (info.hasAppliedAggregation()) rows_before_aggregation->add(info.getRowsBeforeAggregation()); - else - manually_add_rows_before_aggregation_counter = true; /// Remote subquery doesn't contain a group by } }); } @@ -171,8 +169,6 @@ std::optional RemoteSource::tryGenerate() { if (manually_add_rows_before_limit_counter) rows_before_limit->add(rows); - if (manually_add_rows_before_aggregation_counter) - rows_before_aggregation->add(rows); query_executor->finish(); return {}; } diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index bbc563ec5fe..94dfb46e5f6 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -25,8 +25,8 @@ public: void work() override; String getName() const override { return "Remote"; } - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); } - void setRowsBeforeAggregationCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_aggregation.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { rows_before_limit.swap(counter); } + void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation.swap(counter); } /// Stop reading from stream if output port is finished. void onUpdatePorts() override; @@ -45,8 +45,8 @@ private: bool executor_finished = false; bool add_aggregation_info = false; RemoteQueryExecutorPtr query_executor; - RowsBeforeLimitCounterPtr rows_before_limit; - RowsBeforeLimitCounterPtr rows_before_aggregation; + RowsBeforeStepCounterPtr rows_before_limit; + RowsBeforeStepCounterPtr rows_before_aggregation; const bool async_read; const bool async_query_sending; @@ -54,7 +54,6 @@ private: int fd = -1; size_t rows = 0; bool manually_add_rows_before_limit_counter = false; - bool manually_add_rows_before_aggregation_counter = false; }; /// Totals source from RemoteQueryExecutor. diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 9ffe15d0f85..7e7bf815832 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -81,6 +81,8 @@ void AggregatingInOrderTransform::consume(Chunk chunk) is_consume_started = true; } + if (rows_before_aggregation) + rows_before_aggregation->add(rows); src_rows += rows; src_bytes += chunk.bytes(); diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index 5d50e97f552..1fea9bad131 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -42,6 +42,8 @@ public: void work() override; void consume(Chunk chunk); + + void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation.swap(counter); } private: void generate(); @@ -83,6 +85,8 @@ private: Chunk current_chunk; Chunk to_push_chunk; + RowsBeforeStepCounterPtr rows_before_aggregation; + LoggerPtr log = getLogger("AggregatingInOrderTransform"); }; diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 684de0a3e8c..1a0395f0c9a 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -683,8 +683,8 @@ void AggregatingTransform::consume(Chunk chunk) LOG_TRACE(log, "Aggregating"); is_consume_started = true; } - if (rows_before_aggregation_at_least) - rows_before_aggregation_at_least->add(num_rows); + if (rows_before_aggregation) + rows_before_aggregation->add(num_rows); src_rows += num_rows; src_bytes += chunk.bytes(); diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 9f9638175f0..3f7a698d0e0 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -169,10 +169,7 @@ public: Status prepare() override; void work() override; Processors expandPipeline() override; - void setRowsBeforeAggregationCounter(RowsBeforeAggregationCounterPtr counter) override - { - rows_before_aggregation_at_least.swap(counter); - } + void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation.swap(counter); } protected: void consume(Chunk chunk); @@ -216,7 +213,7 @@ private: bool is_consume_started = false; - RowsBeforeAggregationCounterPtr rows_before_aggregation_at_least; + RowsBeforeStepCounterPtr rows_before_aggregation; void initGenerate(); }; diff --git a/src/Processors/Transforms/PartialSortingTransform.h b/src/Processors/Transforms/PartialSortingTransform.h index abb4b290322..73c490d5b92 100644 --- a/src/Processors/Transforms/PartialSortingTransform.h +++ b/src/Processors/Transforms/PartialSortingTransform.h @@ -20,7 +20,7 @@ public: String getName() const override { return "PartialSortingTransform"; } - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { read_rows.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { read_rows.swap(counter); } protected: void transform(Chunk & chunk) override; @@ -29,7 +29,7 @@ private: const SortDescription description; SortDescriptionWithPositions description_with_positions; const UInt64 limit; - RowsBeforeLimitCounterPtr read_rows; + RowsBeforeStepCounterPtr read_rows; Columns sort_description_threshold_columns; diff --git a/src/QueryPipeline/ProfileInfo.cpp b/src/QueryPipeline/ProfileInfo.cpp index 87729b7c90e..69575939edc 100644 --- a/src/QueryPipeline/ProfileInfo.cpp +++ b/src/QueryPipeline/ProfileInfo.cpp @@ -1,14 +1,14 @@ #include +#include +#include #include #include -#include - namespace DB { -void ProfileInfo::read(ReadBuffer & in) +void ProfileInfo::read(ReadBuffer & in, UInt64 server_revision) { readVarUInt(rows, in); readVarUInt(blocks, in); @@ -16,12 +16,15 @@ void ProfileInfo::read(ReadBuffer & in) readBinary(applied_limit, in); readVarUInt(rows_before_limit, in); readBinary(calculated_rows_before_limit, in); - readBinary(applied_aggregation, in); - readVarUInt(rows_before_aggregation, in); + if (server_revision >= DBMS_MIN_REVISION_WITH_ROWS_BEFORE_AGGREGATION) + { + readBinary(applied_aggregation, in); + readVarUInt(rows_before_aggregation, in); + } } -void ProfileInfo::write(WriteBuffer & out) const +void ProfileInfo::write(WriteBuffer & out, UInt64 client_revision) const { writeVarUInt(rows, out); writeVarUInt(blocks, out); @@ -29,8 +32,11 @@ void ProfileInfo::write(WriteBuffer & out) const writeBinary(hasAppliedLimit(), out); writeVarUInt(getRowsBeforeLimit(), out); writeBinary(calculated_rows_before_limit, out); - writeBinary(hasAppliedAggregation(), out); - writeVarUInt(getRowsBeforeAggregation(), out); + if (client_revision >= DBMS_MIN_REVISION_WITH_ROWS_BEFORE_AGGREGATION) + { + writeBinary(hasAppliedAggregation(), out); + writeVarUInt(getRowsBeforeAggregation(), out); + } } diff --git a/src/QueryPipeline/ProfileInfo.h b/src/QueryPipeline/ProfileInfo.h index e2467afd6f4..92c83c8c3be 100644 --- a/src/QueryPipeline/ProfileInfo.h +++ b/src/QueryPipeline/ProfileInfo.h @@ -40,8 +40,8 @@ struct ProfileInfo /// Binary serialization and deserialization of main fields. /// Writes only main fields i.e. fields that required by internal transmission protocol. - void read(ReadBuffer & in); - void write(WriteBuffer & out) const; + void read(ReadBuffer & in, UInt64 server_revision); + void write(WriteBuffer & out, UInt64 client_revision) const; /// Sets main fields from other object (see methods above). /// If skip_block_size_info if true, then rows, bytes and block fields are ignored. diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 311c8a60531..4ec5cca4dc5 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -140,7 +141,7 @@ static void checkCompleted(Processors & processors) static void initRowsBeforeLimit(IOutputFormat * output_format) { - RowsBeforeLimitCounterPtr rows_before_limit_at_least; + RowsBeforeStepCounterPtr rows_before_limit_at_least; std::vector processors; std::map> limit_candidates; std::unordered_set visited; @@ -280,20 +281,20 @@ static void initRowsBeforeAggregation(std::shared_ptr processors, IO if (!processors->empty()) { - RowsBeforeAggregationCounterPtr rows_before_aggregation_at_least = std::make_shared(); + RowsBeforeStepCounterPtr rows_before_aggregation = std::make_shared(); for (auto processor : *processors) { - if (auto transform = std::dynamic_pointer_cast(processor)) + if (typeid_cast(processor.get()) || typeid_cast(processor.get())) { - transform->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); + processor->setRowsBeforeAggregationCounter(rows_before_aggregation); has_aggregation = true; } if (typeid_cast(processor.get()) || typeid_cast(processor.get())) - processor->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); + processor->setRowsBeforeAggregationCounter(rows_before_aggregation); } if (has_aggregation) - rows_before_aggregation_at_least->add(0); - output_format->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); + rows_before_aggregation->add(0); + output_format->setRowsBeforeAggregationCounter(rows_before_aggregation); } } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 14a2bceebf1..833f84bb866 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1212,7 +1212,7 @@ void TCPHandler::sendMergeTreeReadTaskRequestAssumeLocked(ParallelReadRequest re void TCPHandler::sendProfileInfo(const ProfileInfo & info) { writeVarUInt(Protocol::Server::ProfileInfo, *out); - info.write(*out); + info.write(*out, client_tcp_protocol_version); out->next(); } diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference index 36db9721599..2fbdf325760 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference @@ -23,7 +23,7 @@ "rows": 10, - "rows_before_aggregation_at_least": 10 + "rows_before_aggregation": 10 } @@ -68,7 +68,7 @@ 10 - 10 + 10 { "meta": @@ -88,7 +88,7 @@ "rows": 3, - "rows_before_aggregation_at_least": 3 + "rows_before_aggregation": 3 } { "meta": @@ -125,7 +125,7 @@ "rows": 20, - "rows_before_aggregation_at_least": 20 + "rows_before_aggregation": 20 } { "meta": @@ -145,7 +145,7 @@ "rows_before_limit_at_least": 1, - "rows_before_aggregation_at_least": 20 + "rows_before_aggregation": 20 } { "meta": @@ -194,7 +194,7 @@ "rows_before_limit_at_least": 60, - "rows_before_aggregation_at_least": 60 + "rows_before_aggregation": 60 } { "meta": @@ -233,7 +233,7 @@ "rows_before_limit_at_least": 40, - "rows_before_aggregation_at_least": 40 + "rows_before_aggregation": 40 } { "meta": @@ -282,7 +282,7 @@ "rows_before_limit_at_least": 30, - "rows_before_aggregation_at_least": 60 + "rows_before_aggregation": 60 } { "meta": @@ -321,7 +321,7 @@ "rows_before_limit_at_least": 20, - "rows_before_aggregation_at_least": 40 + "rows_before_aggregation": 40 } { "meta": @@ -341,7 +341,7 @@ "rows_before_limit_at_least": 1, - "rows_before_aggregation_at_least": 40 + "rows_before_aggregation": 40 } { "meta": @@ -370,5 +370,179 @@ "rows_before_limit_at_least": 10, - "rows_before_aggregation_at_least": 20 + "rows_before_aggregation": 20 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [4], + [3], + [2], + [5], + [1], + [6], + [7], + [9], + [8] + ], + + "rows": 10, + + "rows_before_limit_at_least": 20, + + "rows_before_aggregation": 20 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9] + ], + + "rows": 10, + + "rows_before_aggregation": 10 +} +{ + "meta": + [ + { + "name": "max(i)", + "type": "Int32" + } + ], + + "data": + [ + [19] + ], + + "rows": 1, + + "rows_before_limit_at_least": 1, + + "rows_before_aggregation": 20 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9], + [10], + [11], + [12], + [13], + [14], + [15], + [16], + [17], + [18], + [19], + [20], + [21], + [22], + [23], + [24], + [25], + [26], + [27], + [28], + [29] + ], + + "rows": 30, + + "rows_before_limit_at_least": 60, + + "rows_before_aggregation": 60 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9], + [10], + [11], + [12], + [13], + [14], + [15], + [16], + [17], + [18], + [19], + [20], + [21], + [22], + [23], + [24], + [25], + [26], + [27], + [28], + [29] + ], + + "rows": 30, + + "rows_before_limit_at_least": 30, + + "rows_before_aggregation": 60 } diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql index 31b817e8a65..17e3f3c2cef 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql @@ -1,18 +1,15 @@ -- Tags: no-parallel, no-random-merge-tree-settings +set rows_before_aggregation = 1, exact_rows_before_limit = 1, output_format_write_statistics = 0, max_block_size = 100; + drop table if exists test; create table test (i int) engine MergeTree order by tuple(); - insert into test select arrayJoin(range(10000)); -set rows_before_aggregation = 1, exact_rows_before_limit = 1, output_format_write_statistics = 0, max_block_size = 100; - select * from test where i < 10 group by i order by i FORMAT JSONCompact; select * from test where i < 10 group by i order by i FORMAT XML; - select * from test group by i having i in (10, 11, 12) order by i FORMAT JSONCompact; - select * from test where i < 20 group by i order by i FORMAT JSONCompact; select max(i) from test where i < 20 limit 1 FORMAT JSONCompact; @@ -23,8 +20,27 @@ select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i set prefer_localhost_replica = 1; select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 30 group by i order by i FORMAT JSONCompact; select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 group by i order by i FORMAT JSONCompact; + select max(i) from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 FORMAT JSONCompact; +select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i limit 10 FORMAT JSONCompact; +set prefer_localhost_replica = 0; select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i limit 10 FORMAT JSONCompact; drop table if exists test; + +create table test (i int) engine MergeTree order by i; + +insert into test select arrayJoin(range(10000)); + +set optimize_aggregation_in_order=1; +select * from test where i < 10 group by i order by i FORMAT JSONCompact; +select max(i) from test where i < 20 limit 1 FORMAT JSONCompact; + +set prefer_localhost_replica = 0; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 30 group by i order by i FORMAT JSONCompact; + +set prefer_localhost_replica = 1; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 30 group by i order by i FORMAT JSONCompact; + +drop table if exists test; From b8b076fd5bedc80cf53781a0190aabb60212a7ef Mon Sep 17 00:00:00 2001 From: morning-color Date: Tue, 16 Jul 2024 11:37:17 +0800 Subject: [PATCH 188/644] Fix style --- src/Processors/Transforms/AggregatingInOrderTransform.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index 1fea9bad131..9a1ba513f43 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -42,7 +42,6 @@ public: void work() override; void consume(Chunk chunk); - void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation.swap(counter); } private: From da5c92e83ef5b485a119ad5a79098bfc9c216d3e Mon Sep 17 00:00:00 2001 From: morning-color Date: Tue, 16 Jul 2024 12:00:27 +0800 Subject: [PATCH 189/644] FIx bug --- src/QueryPipeline/QueryPipeline.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 4ec5cca4dc5..ff2b5aba13d 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -544,7 +544,7 @@ void QueryPipeline::complete(std::shared_ptr format) extremes = nullptr; initRowsBeforeLimit(format.get()); - for (const auto context : resources.interpreter_context) + for (const auto & context : resources.interpreter_context) { if (context->getSettingsRef().rows_before_aggregation) { From b863ef83c351bd87007b4cacc66cff3c5276666f Mon Sep 17 00:00:00 2001 From: morning-color Date: Tue, 16 Jul 2024 13:00:01 +0800 Subject: [PATCH 190/644] Fix compile problem --- src/QueryPipeline/QueryPipeline.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index ff2b5aba13d..844b9e3b039 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -544,7 +545,7 @@ void QueryPipeline::complete(std::shared_ptr format) extremes = nullptr; initRowsBeforeLimit(format.get()); - for (const auto & context : resources.interpreter_context) + for (const auto context : resources.interpreter_context) { if (context->getSettingsRef().rows_before_aggregation) { From 403afbc77057125560d88239f6db182056303657 Mon Sep 17 00:00:00 2001 From: morning-color Date: Tue, 16 Jul 2024 16:05:08 +0800 Subject: [PATCH 191/644] Add settings rows_before_aggregation to src/Core/SettingsChangesHistory.h --- src/Core/SettingsChangesHistory.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index cdc955b38bc..37c91561b0a 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,7 +85,10 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static const std::map settings_changes_history = { - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, + {"24.6", + { + {"rows_before_aggregation", true, true, "Provide exact value for rows_before_aggregation statistic, represents the number of rows read before aggregation"}, + {"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, From 808acea1b6677c1b6bdbc28ddeb01c28e95e43d8 Mon Sep 17 00:00:00 2001 From: morning-color Date: Tue, 16 Jul 2024 19:45:39 +0800 Subject: [PATCH 192/644] Fix compile problem --- src/Core/SettingsChangesHistory.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b9b72209103..922946b192e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,7 +57,8 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, + {"24.7", {{"rows_before_aggregation", true, true, "Provide exact value for rows_before_aggregation statistic, represents the number of rows read before aggregation"}, + {"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, From 9690a5a334b4991eaa9dfa58ce804c91bbff4385 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 16 Jul 2024 13:37:59 +0000 Subject: [PATCH 193/644] fix --- tests/ci/libfuzzer_test_check.py | 2 +- {utils/libfuzzer => tests/fuzz}/runner.py | 0 2 files changed, 1 insertion(+), 1 deletion(-) rename {utils/libfuzzer => tests/fuzz}/runner.py (100%) diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index d9e33229932..8f19dd7d023 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -75,7 +75,7 @@ def get_run_command( f"--volume={result_path}:/test_output " "--security-opt seccomp=unconfined " # required to issue io_uring sys-calls f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image} " - "python3 ./utils/runner.py" + "python3 /usr/share/clickhouse-test/fuzz/runner.py" ) diff --git a/utils/libfuzzer/runner.py b/tests/fuzz/runner.py similarity index 100% rename from utils/libfuzzer/runner.py rename to tests/fuzz/runner.py From c9e02eee7a1d7b2e7aa85bfc87d6a54a3bcaedfa Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 16 Jul 2024 13:48:52 +0000 Subject: [PATCH 194/644] fix after review --- docs/en/sql-reference/table-functions/hdfs.md | 4 ---- .../StorageObjectStorageSource.cpp | 2 +- .../ObjectStorageQueueSource.cpp | 2 +- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageFileCluster.cpp | 2 +- src/Storages/StorageURL.cpp | 19 +------------------ src/Storages/StorageURL.h | 18 ++++++++++++++++++ src/Storages/StorageURLCluster.cpp | 3 +-- src/Storages/VirtualColumnUtils.cpp | 2 +- src/Storages/VirtualColumnUtils.h | 2 +- 10 files changed, 26 insertions(+), 30 deletions(-) diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 73fdc263d68..60c2fd40e6a 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -112,10 +112,6 @@ SET use_hive_partitioning = 1; SELECT * from HDFS('hdfs://hdfs1:9000/data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42; ``` -``` reference -specified_data -``` - ## Storage Settings {#storage-settings} - [hdfs_truncate_on_insert](/docs/en/operations/settings/settings.md#hdfs_truncate_on_insert) - allows to truncate file before insert into it. Disabled by default. diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 93f8eaacbc0..d29e33444b0 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -204,7 +204,7 @@ Chunk StorageObjectStorageSource::generate() .size = object_info->isArchive() ? object_info->fileSizeInArchive() : object_info->metadata->size_bytes, .filename = &filename, .last_modified = object_info->metadata->last_modified, - }, read_from_format_info.columns_description, getContext()); + }, getContext(), read_from_format_info.columns_description); const auto & partition_columns = configuration->getPartitionColumns(); if (!partition_columns.empty() && chunk_size && chunk.hasColumns()) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp index 4d921003e04..2634a7b2f1e 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp @@ -524,7 +524,7 @@ Chunk ObjectStorageQueueSource::generateImpl() { .path = path, .size = reader.getObjectInfo()->metadata->size_bytes - }); + }, getContext(), read_from_format_info.columns_description); return chunk; } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index b43fce370a1..5cbc2b38887 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1459,7 +1459,7 @@ Chunk StorageFileSource::generate() .size = current_file_size, .filename = (filename_override.has_value() ? &filename_override.value() : nullptr), .last_modified = current_file_last_modified - }, columns_description, getContext()); + }, getContext(), columns_description); return chunk; } diff --git a/src/Storages/StorageFileCluster.cpp b/src/Storages/StorageFileCluster.cpp index f7684182e79..82ae0b761ae 100644 --- a/src/Storages/StorageFileCluster.cpp +++ b/src/Storages/StorageFileCluster.cpp @@ -61,7 +61,7 @@ StorageFileCluster::StorageFileCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context, paths.empty() ? "" : paths[0])); } void StorageFileCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 87911230819..6e7788cfc1d 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -39,7 +39,6 @@ #include #include -#include #include #include @@ -92,27 +91,11 @@ static const std::vector> optional_regex_keys = { std::make_shared(R"(headers.header\[[0-9]*\].value)"), }; -static bool urlWithGlobs(const String & uri) -{ - return (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) || uri.find('|') != std::string::npos; -} - static ConnectionTimeouts getHTTPTimeouts(ContextPtr context) { return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings().keep_alive_timeout); } -String getSampleURI(String uri, ContextPtr context) -{ - if (urlWithGlobs(uri)) - { - auto uris = parseRemoteDescription(uri, 0, uri.size(), ',', context->getSettingsRef().glob_expansion_max_elements); - if (!uris.empty()) - return uris[0]; - } - return uri; -} - IStorageURLBase::IStorageURLBase( const String & uri_, const ContextPtr & context_, @@ -433,7 +416,7 @@ Chunk StorageURLSource::generate() { .path = curr_uri.getPath(), .size = current_file_size, - }, columns_description, getContext()); + }, getContext(), columns_description); return chunk; } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index fa7cc6eeeef..a874ca9147c 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -12,6 +12,8 @@ #include #include #include +#include +#include namespace DB @@ -267,6 +269,22 @@ private: bool cancelled = false; }; +static bool urlWithGlobs(const String & uri) +{ + return (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) || uri.find('|') != std::string::npos; +} + +inline String getSampleURI(String uri, ContextPtr context) +{ + if (urlWithGlobs(uri)) + { + auto uris = parseRemoteDescription(uri, 0, uri.size(), ',', context->getSettingsRef().glob_expansion_max_elements); + if (!uris.empty()) + return uris[0]; + } + return uri; +} + class StorageURL : public IStorageURLBase { public: diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 664f170c17e..e80f4ebcd06 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -3,7 +3,6 @@ #include #include -#include #include #include #include @@ -76,7 +75,7 @@ StorageURLCluster::StorageURLCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context, getSampleURI(uri, context), getFormatSettings(context))); } void StorageURLCluster::updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 24d0b7160b2..31cee485dde 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -239,7 +239,7 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values, ColumnsDescription columns, ContextPtr context) + VirtualsForFileLikeStorage virtual_values, ContextPtr context, const ColumnsDescription & columns) { std::unordered_map hive_map; if (context->getSettingsRef().use_hive_partitioning) diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index fef32b149ec..1bd74189559 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -83,7 +83,7 @@ struct VirtualsForFileLikeStorage void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values, ColumnsDescription columns = {}, ContextPtr context = {}); + VirtualsForFileLikeStorage virtual_values, ContextPtr context, const ColumnsDescription & columns); } } From e7e62b358360083eda6d2ec983fb5a1b733d1eba Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 16 Jul 2024 14:17:51 +0000 Subject: [PATCH 195/644] fix style --- tests/fuzz/runner.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index bbe648dbbc2..0862ea29e42 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -11,7 +11,7 @@ FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") def run_fuzzer(fuzzer: str): - logging.info(f"Running fuzzer {fuzzer}...") + logging.info("Running fuzzer %s...", fuzzer) corpus_dir = f"{fuzzer}.in" with Path(corpus_dir) as path: @@ -29,28 +29,28 @@ def run_fuzzer(fuzzer: str): if parser.has_section("asan"): os.environ["ASAN_OPTIONS"] = ( - f"{os.environ['ASAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['asan'].items())}" + f"{os.environ['ASAN_OPTIONS']}:{':'.join(f"{key}={value}" for key, value in parser['asan'].items())}" ) if parser.has_section("msan"): os.environ["MSAN_OPTIONS"] = ( - f"{os.environ['MSAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['msan'].items())}" + f"{os.environ['MSAN_OPTIONS']}:{':'.join(f"{key}={value}" for key, value in parser['msan'].items())}" ) if parser.has_section("ubsan"): os.environ["UBSAN_OPTIONS"] = ( - f"{os.environ['UBSAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['ubsan'].items())}" + f"{os.environ['UBSAN_OPTIONS']}:{':'.join(f"{key}={value}" for key, value in parser['ubsan'].items())}" ) if parser.has_section("libfuzzer"): custom_libfuzzer_options = " ".join( - "-%s=%s" % (key, value) + f"-{key}={value}" for key, value in parser["libfuzzer"].items() ) if parser.has_section("fuzzer_arguments"): fuzzer_arguments = " ".join( - ("%s" % key) if value == "" else ("%s=%s" % (key, value)) + (f"{key}") if value == "" else (f"{key}={value}") for key, value in parser["fuzzer_arguments"].items() ) @@ -65,7 +65,7 @@ def run_fuzzer(fuzzer: str): cmd_line += " < /dev/null" - logging.info(f"...will execute: {cmd_line}") + logging.info("...will execute: %s", cmd_line) subprocess.check_call(cmd_line, shell=True) From c974430e68bff97986379410e9a94c1ea641d1bd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 16 Jul 2024 15:01:43 +0000 Subject: [PATCH 196/644] fix --- tests/fuzz/runner.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 0862ea29e42..047a2245bfa 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -29,17 +29,17 @@ def run_fuzzer(fuzzer: str): if parser.has_section("asan"): os.environ["ASAN_OPTIONS"] = ( - f"{os.environ['ASAN_OPTIONS']}:{':'.join(f"{key}={value}" for key, value in parser['asan'].items())}" + f"{os.environ['ASAN_OPTIONS']}:{':'.join(f'{key}={value}' for key, value in parser['asan'].items())}" ) if parser.has_section("msan"): os.environ["MSAN_OPTIONS"] = ( - f"{os.environ['MSAN_OPTIONS']}:{':'.join(f"{key}={value}" for key, value in parser['msan'].items())}" + f"{os.environ['MSAN_OPTIONS']}:{':'.join(f'{key}={value}' for key, value in parser['msan'].items())}" ) if parser.has_section("ubsan"): os.environ["UBSAN_OPTIONS"] = ( - f"{os.environ['UBSAN_OPTIONS']}:{':'.join(f"{key}={value}" for key, value in parser['ubsan'].items())}" + f"{os.environ['UBSAN_OPTIONS']}:{':'.join(f'{key}={value}' for key, value in parser['ubsan'].items())}" ) if parser.has_section("libfuzzer"): From 8660aec5d79f7a16ab3bcac2aaab291e4bcf0c2d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 16 Jul 2024 15:16:11 +0000 Subject: [PATCH 197/644] Automatic style fix --- tests/fuzz/runner.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 047a2245bfa..44259228f60 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -44,8 +44,7 @@ def run_fuzzer(fuzzer: str): if parser.has_section("libfuzzer"): custom_libfuzzer_options = " ".join( - f"-{key}={value}" - for key, value in parser["libfuzzer"].items() + f"-{key}={value}" for key, value in parser["libfuzzer"].items() ) if parser.has_section("fuzzer_arguments"): From b6672b9952caeff523b2836a710dd3be3d6ed4e8 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Tue, 16 Jul 2024 15:20:01 +0000 Subject: [PATCH 198/644] add rebuild for compact part --- src/Core/SettingsEnums.cpp | 3 +- src/Core/SettingsEnums.h | 1 + src/Interpreters/MutationsInterpreter.cpp | 5 -- src/Interpreters/MutationsInterpreter.h | 1 - .../MergeTree/MergeMutateSelectedEntry.h | 1 - src/Storages/MergeTree/MergeTreeSettings.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 24 ++++++++-- src/Storages/StorageMergeTree.cpp | 1 - ...61_lightweight_delete_projection.reference | 5 ++ .../03161_lightweight_delete_projection.sql | 46 +++++++++++++++++-- 10 files changed, 69 insertions(+), 20 deletions(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 82e7d6db410..6c000d83254 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -175,7 +175,8 @@ IMPLEMENT_SETTING_ENUM(ParallelReplicasCustomKeyFilterType, ErrorCodes::BAD_ARGU IMPLEMENT_SETTING_ENUM(LightweightMutationProjectionMode, ErrorCodes::BAD_ARGUMENTS, {{"throw", LightweightMutationProjectionMode::THROW}, - {"drop", LightweightMutationProjectionMode::DROP}}) + {"drop", LightweightMutationProjectionMode::DROP}, + {"rebuild", LightweightMutationProjectionMode::REBUILD}}) IMPLEMENT_SETTING_AUTO_ENUM(LocalFSReadMethod, ErrorCodes::BAD_ARGUMENTS) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index f6d9593ca56..0281176417a 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -311,6 +311,7 @@ enum class LightweightMutationProjectionMode : uint8_t { THROW, DROP, + REBUILD, }; DECLARE_SETTING_ENUM(LightweightMutationProjectionMode) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index b61f7f78885..db4ea9c0754 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -344,11 +344,6 @@ bool MutationsInterpreter::Source::hasProjection(const String & name) const return part && part->hasProjection(name); } -bool MutationsInterpreter::Source::hasProjection() const -{ - return part && part->hasProjection(); -} - bool MutationsInterpreter::Source::hasBrokenProjection(const String & name) const { return part && part->hasBrokenProjection(name); diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index b792a33f904..6aaa233cda3 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -126,7 +126,6 @@ public: bool materializeTTLRecalculateOnly() const; bool hasSecondaryIndex(const String & name) const; bool hasProjection(const String & name) const; - bool hasProjection() const; bool hasBrokenProjection(const String & name) const; bool isCompactPart() const; diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index e7efe00741c..c420cbca12b 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -3,7 +3,6 @@ #include #include - namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c84ca9956fc..74e7a7f43bc 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -215,7 +215,7 @@ struct Settings; M(Float, primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns, 0.9f, "If the value of a column of the primary key in data part changes at least in this ratio of times, skip loading next columns in memory. This allows to save memory usage by not loading useless columns of the primary key.", 0) \ /** Projection settings. */ \ M(UInt64, max_projections, 25, "The maximum number of merge tree projections.", 0) \ - M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop projections of this table's relevant parts.", 0) \ + M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop projections of this table's relevant parts, or rebuild the projections.", 0) \ #define MAKE_OBSOLETE_MERGE_TREE_SETTING(M, TYPE, NAME, DEFAULT) \ M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", BaseSettingsHelpers::Flags::OBSOLETE) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 8790ce6628e..092a6d0d6ed 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -662,7 +662,7 @@ static NameSet collectFilesToSkip( const std::set & projections_to_recalc, const std::set & stats_to_recalc, const StorageMetadataPtr & metadata_snapshot, - bool lightweight_delete_mode) + bool skip_all_projections) { NameSet files_to_skip = source_part->getFileNamesWithoutChecksums(); @@ -686,7 +686,7 @@ static NameSet collectFilesToSkip( } } - if (lightweight_delete_mode) + if (skip_all_projections) { for (const auto & projection : metadata_snapshot->getProjections()) files_to_skip.insert(projection.getDirectoryName()); @@ -2211,6 +2211,8 @@ bool MutateTask::prepare() ctx->stage_progress = std::make_unique(1.0); + bool lightweight_delete_mode = false; + if (!ctx->for_interpreter.empty()) { /// Always disable filtering in mutations: we want to read and write all rows because for updates we rewrite only some of the @@ -2228,6 +2230,16 @@ bool MutateTask::prepare() ctx->mutating_pipeline_builder = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); ctx->progress_callback = MergeProgressCallback((*ctx->mutate_entry)->ptr(), ctx->watch_prev_elapsed, *ctx->stage_progress); + + lightweight_delete_mode = ctx->updated_header.has(RowExistsColumn::name); + /// If under the condition of lightweight delete mode with rebuild option, add projections again here as we can only know + /// the condition as early as from here. + if (lightweight_delete_mode + && ctx->data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::REBUILD) + { + for (const auto & projection : ctx->metadata_snapshot->getProjections()) + ctx->materialized_projections.insert(projection.name); + } } auto single_disk_volume = std::make_shared("volume_" + ctx->future_part->name, ctx->space_reservation->getDisk(), 0); @@ -2269,7 +2281,6 @@ bool MutateTask::prepare() if (ctx->mutating_pipeline_builder.initialized()) ctx->execute_ttl_type = MutationHelpers::shouldExecuteTTL(ctx->metadata_snapshot, ctx->interpreter->getColumnDependencies()); - bool lightweight_delete_mode = ctx->updated_header.has(RowExistsColumn::name); if (ctx->data->getSettings()->exclude_deleted_rows_for_part_size_in_merge && lightweight_delete_mode) { /// This mutation contains lightweight delete and we need to count the deleted rows, @@ -2307,7 +2318,10 @@ bool MutateTask::prepare() ctx->context, ctx->materialized_indices); - if (!lightweight_delete_mode) + bool lightweight_delete_projection_drop = lightweight_delete_mode + && ctx->data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::DROP; + /// Under lightweight delete mode, if option is drop, projections_to_recalc should be empty. + if (!lightweight_delete_projection_drop) { ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate( ctx->source_part, @@ -2326,7 +2340,7 @@ bool MutateTask::prepare() ctx->projections_to_recalc, ctx->stats_to_recalc, ctx->metadata_snapshot, - lightweight_delete_mode); + lightweight_delete_projection_drop); ctx->files_to_rename = MutationHelpers::collectFilesForRenames( ctx->source_part, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index d8c61da2a98..40b3a12297b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1285,7 +1285,6 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( auto commands = std::make_shared(); size_t current_ast_elements = 0; auto last_mutation_to_apply = mutations_end_it; - for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { /// Do not squash mutations from different transactions to be able to commit/rollback them independently. diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index e69de29bb2d..bc7e1faecff 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -0,0 +1,5 @@ +8888 Alice 50 +p1 +p2 +p1 +p2 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 16a7468234b..b63341f5371 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -13,25 +13,43 @@ SETTINGS min_bytes_for_wide_part = 10485760; INSERT INTO users VALUES (1231, 'John', 33); +-- testing throw default mode ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } +-- testing drop mode ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; DELETE FROM users WHERE uid = 1231; +SELECT * FROM users ORDER BY uid; + SYSTEM FLUSH LOGS; -- expecting no projection SELECT - name, - `table` + name FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +-- testing rebuild mode +INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); + +ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; + +DELETE FROM users WHERE uid = 6666; + SELECT * FROM users ORDER BY uid; +SYSTEM FLUSH LOGS; + +-- expecting projection p1, p2 in 2 parts +SELECT + name +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); + DROP TABLE users; @@ -47,23 +65,41 @@ SETTINGS min_bytes_for_wide_part = 0; INSERT INTO users VALUES (1231, 'John', 33); +-- testing throw default mode ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } +-- testing drop mode ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; DELETE FROM users WHERE uid = 1231; +SELECT * FROM users ORDER BY uid; + SYSTEM FLUSH LOGS; -- expecting no projection SELECT - name, - `table` + name FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); -SELECT * FROM users ORDER BY uid; +-- -- testing rebuild mode +-- INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); + +-- ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; + +-- DELETE FROM users WHERE uid = 6666; + +-- SELECT * FROM users ORDER BY uid; + +-- SYSTEM FLUSH LOGS; + +-- -- expecting projection p1, p2 in 2 parts +-- SELECT +-- name +-- FROM system.projection_parts +-- WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); DROP TABLE users; \ No newline at end of file From c3507979cfc0359ab38762525ab0306904a387b8 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Tue, 16 Jul 2024 15:41:54 +0000 Subject: [PATCH 199/644] fix --- src/Storages/MergeTree/MutateTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 092a6d0d6ed..489c8863a8a 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2234,7 +2234,7 @@ bool MutateTask::prepare() lightweight_delete_mode = ctx->updated_header.has(RowExistsColumn::name); /// If under the condition of lightweight delete mode with rebuild option, add projections again here as we can only know /// the condition as early as from here. - if (lightweight_delete_mode + if (lightweight_delete_mode && ctx->data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::REBUILD) { for (const auto & projection : ctx->metadata_snapshot->getProjections()) From 3ae4211b3af575cf8d7186a4cc915f9ecb6b4182 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 16 Jul 2024 17:59:57 +0200 Subject: [PATCH 200/644] fix tests --- tests/queries/0_stateless/03203_hive_style_partitioning.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index d2b1f31c85f..0f687d532b0 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -39,8 +39,8 @@ SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01 $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; -SELECT _identifier FROM file('$CURDIR/data_hive/partitioning/identitier=*/email.csv') LIMIT 2; -SELECT __identifier FROM file('$CURDIR/data_hive/partitioning/identitier=*/email.csv') LIMIT 2; +SELECT _identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') LIMIT 2; +SELECT __identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') LIMIT 2; """ $CLICKHOUSE_LOCAL -n -q """ From d91cb40bbdbd18a2bef811002033d0c99fe693d3 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 16 Jul 2024 16:15:24 +0000 Subject: [PATCH 201/644] fix include and remove unused getFormatSettings --- src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp | 2 +- src/Storages/StorageFile.cpp | 1 - src/Storages/StorageURLCluster.cpp | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 7f6b3338f9b..c214665f7e0 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -68,7 +68,7 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( if (sample_path.empty() && context_->getSettingsRef().use_hive_partitioning) sample_path = getPathSample(metadata, context_); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context_, sample_path, getFormatSettings(context_))); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context_, sample_path)); setInMemoryMetadata(metadata); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 5cbc2b38887..ed05f57b418 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -52,7 +52,6 @@ #include #include #include -#include "Formats/FormatSettings.h" #include #include diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index e80f4ebcd06..1522a18a083 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -75,7 +75,7 @@ StorageURLCluster::StorageURLCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context, getSampleURI(uri, context), getFormatSettings(context))); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context, getSampleURI(uri, context))); } void StorageURLCluster::updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) From d11d1a42bc7101993e5c85f1c1c3298e6334dbf9 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 16 Jul 2024 16:49:33 +0000 Subject: [PATCH 202/644] fix for storageURL functions --- src/Storages/StorageURL.cpp | 16 ++++++++++++++++ src/Storages/StorageURL.h | 19 +++---------------- 2 files changed, 19 insertions(+), 16 deletions(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 6e7788cfc1d..1d1deebf9f5 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -91,6 +91,22 @@ static const std::vector> optional_regex_keys = { std::make_shared(R"(headers.header\[[0-9]*\].value)"), }; +bool urlWithGlobs(const String & uri) +{ + return (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) || uri.find('|') != std::string::npos; +} + +String getSampleURI(String uri, ContextPtr context) +{ + if (urlWithGlobs(uri)) + { + auto uris = parseRemoteDescription(uri, 0, uri.size(), ',', context->getSettingsRef().glob_expansion_max_elements); + if (!uris.empty()) + return uris[0]; + } + return uri; +} + static ConnectionTimeouts getHTTPTimeouts(ContextPtr context) { return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings().keep_alive_timeout); diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index a874ca9147c..cd48ecb767b 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -143,6 +143,9 @@ private: virtual Block getHeaderBlock(const Names & column_names, const StorageSnapshotPtr & storage_snapshot) const = 0; }; +bool urlWithGlobs(const String & uri); + +String getSampleURI(String uri, ContextPtr context); class StorageURLSource : public SourceWithKeyCondition, WithContext { @@ -269,22 +272,6 @@ private: bool cancelled = false; }; -static bool urlWithGlobs(const String & uri) -{ - return (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) || uri.find('|') != std::string::npos; -} - -inline String getSampleURI(String uri, ContextPtr context) -{ - if (urlWithGlobs(uri)) - { - auto uris = parseRemoteDescription(uri, 0, uri.size(), ',', context->getSettingsRef().glob_expansion_max_elements); - if (!uris.empty()) - return uris[0]; - } - return uri; -} - class StorageURL : public IStorageURLBase { public: From 14dcb97e353fb4739c7f7d37b9c3c11c9ad40923 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 16 Jul 2024 19:09:18 +0200 Subject: [PATCH 203/644] Update src/Storages/StorageURL.h Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Storages/StorageURL.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index cd48ecb767b..1f3d63b4c85 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -12,8 +12,6 @@ #include #include #include -#include -#include namespace DB From 771b39fa2179a9a548580c41859bbecf0165000d Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 16 Jul 2024 19:44:10 +0200 Subject: [PATCH 204/644] Update StorageURLCluster.cpp --- src/Storages/StorageURLCluster.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 1522a18a083..7c7a299c64e 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include From 9d332911fb0f4b25bbdb67a9c10c5f3b42db4ea6 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 16 Jul 2024 20:10:19 +0200 Subject: [PATCH 205/644] Update StorageURL.cpp --- src/Storages/StorageURL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 1d1deebf9f5..4cf191f7e8a 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -39,6 +39,7 @@ #include #include +#include #include #include From 949e69c0573354417e21bd83d446e1ea085db04d Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 16 Jul 2024 21:58:48 +0200 Subject: [PATCH 206/644] add documentation for getSubcolumn and getTypeSerializationStreams --- .../functions/other-functions.md | 91 +++++++++++++++++++ 1 file changed, 91 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 260457b3be1..40f1b82562d 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -4055,3 +4055,94 @@ Result: │ 32 │ └─────────────────────────────┘ ``` + +## getSubcolumn + +Takes a table expression or identifier and constant string with the name of the sub-column, and returns the requested sub-column extracted from the expression. + +**Syntax** + +```sql +getSubcolumn(col_name, subcol_name) +``` + +**Arguments** + +- `col_name` — Table expression or identifier. [Expression](../syntax.md/#expressions), [Identifier](../syntax.md/#identifiers). +- `subcol_name` — The name of the sub-column. [String](../data-types/string.md). + +**Returned value** + +- Returns the extracted sub-colum. + +**Example** + +Query: + +```sql +CREATE TABLE t_arr (arr Array(Tuple(subcolumn1 UInt32, subcolumn2 String))) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO t_arr VALUES ([(1, 'Hello'), (2, 'World')]), ([(3, 'This'), (4, 'is'), (5, 'subcolumn')]); +SELECT getSubcolumn(arr, 'subcolumn1'), getSubcolumn(arr, 'subcolumn2') FROM t_arr; +``` + +Result: + +```response + ┌─getSubcolumn(arr, 'subcolumn1')─┬─getSubcolumn(arr, 'subcolumn2')─┐ +1. │ [1,2] │ ['Hello','World'] │ +2. │ [3,4,5] │ ['This','is','subcolumn'] │ + └─────────────────────────────────┴─────────────────────────────────┘ +``` + +## getTypeSerializationStreams + +Enumerates stream paths of a data type. + +:::note +This function is intended for use by developers. +::: + +**Syntax** + +```sql +getTypeSerializationStreams(col) +``` + +**Arguments** + +- `col` — Column or string representation of a data-type from which the data type will be detected. + +**Returned value** + +- Returns an array with all the serialization sub-stream paths.[Array](../data-types/array.md)([String](../data-types/string.md)). + +**Examples** + +Query: + +```sql +SELECT getTypeSerializationStreams(tuple('a', 1, 'b', 2)); +``` + +Result: + +```response + ┌─getTypeSerializationStreams(('a', 1, 'b', 2))─────────────────────────────────────────────────────────────────────────┐ +1. │ ['{TupleElement(1), Regular}','{TupleElement(2), Regular}','{TupleElement(3), Regular}','{TupleElement(4), Regular}'] │ + └───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +Query: + +```sql +SELECT getTypeSerializationStreams('Map(String, Int64)'); +``` + +Result: + +```response + ┌─getTypeSerializationStreams('Map(String, Int64)')────────────────────────────────────────────────────────────────┐ +1. │ ['{ArraySizes}','{ArrayElements, TupleElement(keys), Regular}','{ArrayElements, TupleElement(values), Regular}'] │ + └──────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + From 122673592b21a2a0e60d1cedc9f9337b471ebcb8 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 17 Jul 2024 02:37:48 +0000 Subject: [PATCH 207/644] add rebuild for wide part --- src/Storages/MergeTree/MutateTask.cpp | 5 ++++ src/Storages/StorageInMemoryMetadata.cpp | 12 +++++++-- ...61_lightweight_delete_projection.reference | 1 + .../03161_lightweight_delete_projection.sql | 25 ++++++++++--------- 4 files changed, 29 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 489c8863a8a..fe14c5a4f05 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2238,7 +2238,12 @@ bool MutateTask::prepare() && ctx->data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::REBUILD) { for (const auto & projection : ctx->metadata_snapshot->getProjections()) + { + if (!ctx->source_part->hasProjection(projection.name)) + continue; + ctx->materialized_projections.insert(projection.name); + } } } diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 2226de3e64f..4a655cac566 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -334,10 +335,17 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( NameSet required_ttl_columns; NameSet updated_ttl_columns; - auto add_dependent_columns = [&updated_columns](const Names & required_columns, auto & to_set) + auto add_dependent_columns = [&updated_columns](const Names & required_columns, auto & to_set, bool is_projection = false) { for (const auto & dependency : required_columns) { + /// useful in the case of lightweight delete with wide part and option of rebuild projection + if (is_projection && updated_columns.contains(RowExistsColumn::name)) + { + to_set.insert(required_columns.begin(), required_columns.end()); + return true; + } + if (updated_columns.contains(dependency)) { to_set.insert(required_columns.begin(), required_columns.end()); @@ -357,7 +365,7 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( for (const auto & projection : getProjections()) { if (has_dependency(projection.name, ColumnDependency::PROJECTION)) - add_dependent_columns(projection.getRequiredColumns(), projections_columns); + add_dependent_columns(projection.getRequiredColumns(), projections_columns, true); } auto add_for_rows_ttl = [&](const auto & expression, auto & to_set) diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index bc7e1faecff..3401eaf6162 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -1,5 +1,6 @@ 8888 Alice 50 p1 p2 +8888 Alice 50 p1 p2 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index b63341f5371..2c60d83d74d 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -44,7 +44,7 @@ SELECT * FROM users ORDER BY uid; SYSTEM FLUSH LOGS; --- expecting projection p1, p2 in 2 parts +-- expecting projection p1, p2 SELECT name FROM system.projection_parts @@ -85,21 +85,22 @@ SELECT FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); --- -- testing rebuild mode --- INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); +-- testing rebuild mode +INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); --- ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; +ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; --- DELETE FROM users WHERE uid = 6666; +DELETE FROM users WHERE uid = 6666; --- SELECT * FROM users ORDER BY uid; +SELECT * FROM users ORDER BY uid; --- SYSTEM FLUSH LOGS; +SYSTEM FLUSH LOGS; + +-- expecting projection p1, p2 +SELECT + name +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); --- -- expecting projection p1, p2 in 2 parts --- SELECT --- name --- FROM system.projection_parts --- WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); DROP TABLE users; \ No newline at end of file From f29700e04d3ca4d455908ed354472945597da4f5 Mon Sep 17 00:00:00 2001 From: morning-color Date: Wed, 17 Jul 2024 20:01:01 +0800 Subject: [PATCH 208/644] Fix flaky tests --- ...74_exact_rows_before_aggregation.reference | 26 +++++++++---------- .../03174_exact_rows_before_aggregation.sql | 4 +-- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference index 2fbdf325760..5f7fedbbcb3 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference @@ -355,15 +355,15 @@ "data": [ [0], - [4], - [3], - [2], - [5], [1], + [2], + [3], + [4], + [5], [6], [7], - [9], - [8] + [8], + [9] ], "rows": 10, @@ -384,20 +384,20 @@ "data": [ [0], - [4], - [3], - [2], - [5], [1], + [2], + [3], + [4], + [5], [6], [7], - [9], - [8] + [8], + [9] ], "rows": 10, - "rows_before_limit_at_least": 20, + "rows_before_limit_at_least": 10, "rows_before_aggregation": 20 } diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql index 17e3f3c2cef..0afc0be4370 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql @@ -23,9 +23,9 @@ select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i select max(i) from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 FORMAT JSONCompact; -select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i limit 10 FORMAT JSONCompact; +select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i order by i limit 10 FORMAT JSONCompact; set prefer_localhost_replica = 0; -select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i limit 10 FORMAT JSONCompact; +select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i order by i limit 10 FORMAT JSONCompact; drop table if exists test; From 63936364b1abf345349403e656c4cf58c44715bc Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 17 Jul 2024 16:53:35 +0200 Subject: [PATCH 209/644] fixes of tests --- src/Storages/VirtualColumnUtils.cpp | 2 +- .../0_stateless/03203_hive_style_partitioning.sh | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 31cee485dde..87c1aecc3a7 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -127,7 +127,7 @@ NameSet getVirtualNamesForFileLikeStorage() std::unordered_map parseHivePartitioningKeysAndValues(const String & path, const ColumnsDescription & storage_columns) { - std::string pattern = "/([^/]+)=([^/]+)"; + std::string pattern = "([^/]+)=([^/]+)/"; re2::StringPiece input_piece(path); std::unordered_map key_values; diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 0f687d532b0..db1f073d736 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -124,13 +124,13 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3CLUSTER PARTITIONING'" $CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; +SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; -SELECT *, _column0, _column1 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0, _column1 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0, _column1 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0, _column1 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; """ From 938071cd55913c3bb2b8781750ef37bf6307acab Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 17 Jul 2024 14:56:00 +0000 Subject: [PATCH 210/644] add ci_include_fuzzer to PR body template --- .github/PULL_REQUEST_TEMPLATE.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index e045170561d..146542e980c 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -59,6 +59,8 @@ At a minimum, the following information should be added (but add more as needed) - [ ] Exclude: All with TSAN, MSAN, UBSAN, Coverage - [ ] Exclude: All with aarch64, release, debug --- +- [ ] Run only libFuzzer related jobs +--- - [ ] Do not test - [ ] Woolen Wolfdog - [ ] Upload binaries for special builds From 542542b44d4688bc125887f811843249a4024379 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 17 Jul 2024 14:58:58 +0000 Subject: [PATCH 211/644] fix test --- .../queries/0_stateless/03161_lightweight_delete_projection.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 2c60d83d74d..3bf459cc32d 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -1,4 +1,6 @@ +SET lightweight_deletes_sync = 2; + DROP TABLE IF EXISTS users; -- compact part From bb01920370e1dd5faa7b17694f74175190537445 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 17 Jul 2024 16:19:02 +0000 Subject: [PATCH 212/644] add ci_exclude_ast to PR body template --- .github/PULL_REQUEST_TEMPLATE.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 146542e980c..8b6e957e1d8 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -59,7 +59,8 @@ At a minimum, the following information should be added (but add more as needed) - [ ] Exclude: All with TSAN, MSAN, UBSAN, Coverage - [ ] Exclude: All with aarch64, release, debug --- -- [ ] Run only libFuzzer related jobs +- [ ] Run only fuzzers related jobs (libFuzzer fuzzers, AST fuzzers, etc.) +- [ ] Exclude AST fuzzers --- - [ ] Do not test - [ ] Woolen Wolfdog From 275b3666dadece731e368dd672e8d6e83ec22d8f Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 18 Jul 2024 01:18:34 +0000 Subject: [PATCH 213/644] try to fix the test --- .../03161_lightweight_delete_projection.sql | 114 +++++++++++++++++- 1 file changed, 111 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 3bf459cc32d..9d577f8a701 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -1,5 +1,73 @@ -SET lightweight_deletes_sync = 2; +SET lightweight_deletes_sync = 2, alter_sync = 2; + +Set max_insert_threads = 2, +group_by_two_level_threshold = 704642, +group_by_two_level_threshold_bytes = 49659607, +distributed_aggregation_memory_efficient = 0, +fsync_metadata = 0, +output_format_parallel_formatting = 0, +input_format_parallel_parsing = 1, +min_chunk_bytes_for_parallel_parsing = 14437539, +max_read_buffer_size = 887507, +prefer_localhost_replica = 0, +max_block_size = 73908, +max_joined_block_size_rows = 21162, +max_threads = 2, +optimize_append_index = 0, +optimize_if_chain_to_multiif = 1, +optimize_if_transform_strings_to_enum = 0, +optimize_read_in_order = 0, +optimize_or_like_chain = 1, +optimize_substitute_columns = 1, +enable_multiple_prewhere_read_steps = 1, +read_in_order_two_level_merge_threshold = 13, +optimize_aggregation_in_order = 1, +aggregation_in_order_max_block_bytes = 37110261, +use_uncompressed_cache = 0, +min_bytes_to_use_direct_io = 10737418240, +min_bytes_to_use_mmap_io = 1, +local_filesystem_read_method ='pread', +remote_filesystem_read_method ='threadpool', +local_filesystem_read_prefetch = 0, +filesystem_cache_segments_batch_size = 3, +read_from_filesystem_cache_if_exists_otherwise_bypass_cache = 1, +throw_on_error_from_cache_on_write_operations = 0, +remote_filesystem_read_prefetch = 1, +allow_prefetched_read_pool_for_remote_filesystem = 0, +filesystem_prefetch_max_memory_usage = '32Mi', +filesystem_prefetches_limit = 0, +filesystem_prefetch_min_bytes_for_single_read_task ='16Mi', +filesystem_prefetch_step_marks = 50, +filesystem_prefetch_step_bytes = 0, +compile_aggregate_expressions = 0, +compile_sort_description = 1, +merge_tree_coarse_index_granularity = 16, +optimize_distinct_in_order = 0, +max_bytes_before_external_sort = 0, +max_bytes_before_external_group_by = 0, +max_bytes_before_remerge_sort = 820113150, +min_compress_block_size = 1262249, +max_compress_block_size = 1472188, +merge_tree_compact_parts_min_granules_to_multibuffer_read = 56, +optimize_sorting_by_input_stream_properties = 1, +http_response_buffer_size = 1883022, +http_wait_end_of_query = False, +enable_memory_bound_merging_of_aggregation_results = 1, +min_count_to_compile_expression = 0, +min_count_to_compile_aggregate_expression = 0, +min_count_to_compile_sort_description = 0, +session_timezone ='Africa/Khartoum', +prefer_warmed_unmerged_parts_seconds = 10, +use_page_cache_for_disks_without_file_cache = True, +page_cache_inject_eviction = False, +merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.02, +prefer_external_sort_block_bytes = 100000000, +cross_join_min_rows_to_compress = 100000000, +cross_join_min_bytes_to_compress = 1, +min_external_table_block_size_bytes = 100000000, +max_parsing_threads = 0; + DROP TABLE IF EXISTS users; @@ -11,7 +79,27 @@ CREATE TABLE users ( projection p1 (select count(), age group by age), projection p2 (select age, name group by age, name) ) ENGINE = MergeTree order by uid -SETTINGS min_bytes_for_wide_part = 10485760; +SETTINGS min_bytes_for_wide_part = 10485760, +ratio_of_defaults_for_sparse_serialization = 1.0, +prefer_fetch_merged_part_size_threshold = 1, +vertical_merge_algorithm_min_rows_to_activate = 1, +vertical_merge_algorithm_min_columns_to_activate = 100, +allow_vertical_merges_from_compact_to_wide_parts = 0, +min_merge_bytes_to_use_direct_io = 114145183, +index_granularity_bytes = 2660363, +merge_max_block_size = 13460, +index_granularity = 51768, +marks_compress_block_size = 59418, +primary_key_compress_block_size = 88795, +replace_long_file_name_to_hash = 0, +max_file_name_length = 0, +min_bytes_for_full_part_storage = 536870912, +compact_parts_max_bytes_to_buffer = 378557913, +compact_parts_max_granules_to_buffer = 254, +compact_parts_merge_max_bytes_to_prefetch_part = 26969686, +cache_populated_by_fetch = 0, +concurrent_part_removal_threshold = 38, +old_parts_lifetime = 480; INSERT INTO users VALUES (1231, 'John', 33); @@ -63,7 +151,27 @@ CREATE TABLE users ( projection p1 (select count(), age group by age), projection p2 (select age, name group by age, name) ) ENGINE = MergeTree order by uid -SETTINGS min_bytes_for_wide_part = 0; +SETTINGS min_bytes_for_wide_part = 0, +ratio_of_defaults_for_sparse_serialization = 1.0, +prefer_fetch_merged_part_size_threshold = 1, +vertical_merge_algorithm_min_rows_to_activate = 1, +vertical_merge_algorithm_min_columns_to_activate = 100, +allow_vertical_merges_from_compact_to_wide_parts = 0, +min_merge_bytes_to_use_direct_io = 114145183, +index_granularity_bytes = 2660363, +merge_max_block_size = 13460, +index_granularity = 51768, +marks_compress_block_size = 59418, +primary_key_compress_block_size = 88795, +replace_long_file_name_to_hash = 0, +max_file_name_length = 0, +min_bytes_for_full_part_storage = 536870912, +compact_parts_max_bytes_to_buffer = 378557913, +compact_parts_max_granules_to_buffer = 254, +compact_parts_merge_max_bytes_to_prefetch_part = 26969686, +cache_populated_by_fetch = 0, +concurrent_part_removal_threshold = 38, +old_parts_lifetime = 480; INSERT INTO users VALUES (1231, 'John', 33); From 2504a6c36016b41e33ee5323fca79f5d511fb3ce Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 19 Jul 2024 14:59:38 +0000 Subject: [PATCH 214/644] make test output a bit clear --- ...61_lightweight_delete_projection.reference | 8 ++ .../03161_lightweight_delete_projection.sql | 131 ++---------------- 2 files changed, 21 insertions(+), 118 deletions(-) diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index 3401eaf6162..cb623ea2b50 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -1,6 +1,14 @@ +compact part +testing throw default mode +testing drop mode +testing rebuild mode 8888 Alice 50 p1 p2 +wide part +testing throw default mode +testing drop mode +testing rebuild mode 8888 Alice 50 p1 p2 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 9d577f8a701..f2d6dcb164f 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -1,77 +1,11 @@ SET lightweight_deletes_sync = 2, alter_sync = 2; -Set max_insert_threads = 2, -group_by_two_level_threshold = 704642, -group_by_two_level_threshold_bytes = 49659607, -distributed_aggregation_memory_efficient = 0, -fsync_metadata = 0, -output_format_parallel_formatting = 0, -input_format_parallel_parsing = 1, -min_chunk_bytes_for_parallel_parsing = 14437539, -max_read_buffer_size = 887507, -prefer_localhost_replica = 0, -max_block_size = 73908, -max_joined_block_size_rows = 21162, -max_threads = 2, -optimize_append_index = 0, -optimize_if_chain_to_multiif = 1, -optimize_if_transform_strings_to_enum = 0, -optimize_read_in_order = 0, -optimize_or_like_chain = 1, -optimize_substitute_columns = 1, -enable_multiple_prewhere_read_steps = 1, -read_in_order_two_level_merge_threshold = 13, -optimize_aggregation_in_order = 1, -aggregation_in_order_max_block_bytes = 37110261, -use_uncompressed_cache = 0, -min_bytes_to_use_direct_io = 10737418240, -min_bytes_to_use_mmap_io = 1, -local_filesystem_read_method ='pread', -remote_filesystem_read_method ='threadpool', -local_filesystem_read_prefetch = 0, -filesystem_cache_segments_batch_size = 3, -read_from_filesystem_cache_if_exists_otherwise_bypass_cache = 1, -throw_on_error_from_cache_on_write_operations = 0, -remote_filesystem_read_prefetch = 1, -allow_prefetched_read_pool_for_remote_filesystem = 0, -filesystem_prefetch_max_memory_usage = '32Mi', -filesystem_prefetches_limit = 0, -filesystem_prefetch_min_bytes_for_single_read_task ='16Mi', -filesystem_prefetch_step_marks = 50, -filesystem_prefetch_step_bytes = 0, -compile_aggregate_expressions = 0, -compile_sort_description = 1, -merge_tree_coarse_index_granularity = 16, -optimize_distinct_in_order = 0, -max_bytes_before_external_sort = 0, -max_bytes_before_external_group_by = 0, -max_bytes_before_remerge_sort = 820113150, -min_compress_block_size = 1262249, -max_compress_block_size = 1472188, -merge_tree_compact_parts_min_granules_to_multibuffer_read = 56, -optimize_sorting_by_input_stream_properties = 1, -http_response_buffer_size = 1883022, -http_wait_end_of_query = False, -enable_memory_bound_merging_of_aggregation_results = 1, -min_count_to_compile_expression = 0, -min_count_to_compile_aggregate_expression = 0, -min_count_to_compile_sort_description = 0, -session_timezone ='Africa/Khartoum', -prefer_warmed_unmerged_parts_seconds = 10, -use_page_cache_for_disks_without_file_cache = True, -page_cache_inject_eviction = False, -merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.02, -prefer_external_sort_block_bytes = 100000000, -cross_join_min_rows_to_compress = 100000000, -cross_join_min_bytes_to_compress = 1, -min_external_table_block_size_bytes = 100000000, -max_parsing_threads = 0; - - DROP TABLE IF EXISTS users; --- compact part + +SELECT 'compact part'; + CREATE TABLE users ( uid Int16, name String, @@ -79,36 +13,17 @@ CREATE TABLE users ( projection p1 (select count(), age group by age), projection p2 (select age, name group by age, name) ) ENGINE = MergeTree order by uid -SETTINGS min_bytes_for_wide_part = 10485760, -ratio_of_defaults_for_sparse_serialization = 1.0, -prefer_fetch_merged_part_size_threshold = 1, -vertical_merge_algorithm_min_rows_to_activate = 1, -vertical_merge_algorithm_min_columns_to_activate = 100, -allow_vertical_merges_from_compact_to_wide_parts = 0, -min_merge_bytes_to_use_direct_io = 114145183, -index_granularity_bytes = 2660363, -merge_max_block_size = 13460, -index_granularity = 51768, -marks_compress_block_size = 59418, -primary_key_compress_block_size = 88795, -replace_long_file_name_to_hash = 0, -max_file_name_length = 0, -min_bytes_for_full_part_storage = 536870912, -compact_parts_max_bytes_to_buffer = 378557913, -compact_parts_max_granules_to_buffer = 254, -compact_parts_merge_max_bytes_to_prefetch_part = 26969686, -cache_populated_by_fetch = 0, -concurrent_part_removal_threshold = 38, -old_parts_lifetime = 480; +SETTINGS min_bytes_for_wide_part = 10485760; INSERT INTO users VALUES (1231, 'John', 33); --- testing throw default mode +SELECT 'testing throw default mode'; + ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } --- testing drop mode +SELECT 'testing drop mode'; ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; DELETE FROM users WHERE uid = 1231; @@ -123,7 +38,7 @@ SELECT FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); --- testing rebuild mode +SELECT 'testing rebuild mode'; INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; @@ -143,7 +58,7 @@ WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); DROP TABLE users; --- wide part +SELECT 'wide part'; CREATE TABLE users ( uid Int16, name String, @@ -151,36 +66,16 @@ CREATE TABLE users ( projection p1 (select count(), age group by age), projection p2 (select age, name group by age, name) ) ENGINE = MergeTree order by uid -SETTINGS min_bytes_for_wide_part = 0, -ratio_of_defaults_for_sparse_serialization = 1.0, -prefer_fetch_merged_part_size_threshold = 1, -vertical_merge_algorithm_min_rows_to_activate = 1, -vertical_merge_algorithm_min_columns_to_activate = 100, -allow_vertical_merges_from_compact_to_wide_parts = 0, -min_merge_bytes_to_use_direct_io = 114145183, -index_granularity_bytes = 2660363, -merge_max_block_size = 13460, -index_granularity = 51768, -marks_compress_block_size = 59418, -primary_key_compress_block_size = 88795, -replace_long_file_name_to_hash = 0, -max_file_name_length = 0, -min_bytes_for_full_part_storage = 536870912, -compact_parts_max_bytes_to_buffer = 378557913, -compact_parts_max_granules_to_buffer = 254, -compact_parts_merge_max_bytes_to_prefetch_part = 26969686, -cache_populated_by_fetch = 0, -concurrent_part_removal_threshold = 38, -old_parts_lifetime = 480; +SETTINGS min_bytes_for_wide_part = 0; INSERT INTO users VALUES (1231, 'John', 33); --- testing throw default mode +SELECT 'testing throw default mode'; ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } --- testing drop mode +SELECT 'testing drop mode'; ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; DELETE FROM users WHERE uid = 1231; @@ -195,7 +90,7 @@ SELECT FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); --- testing rebuild mode +SELECT 'testing rebuild mode'; INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; From 4ae0daf5d3149a2e9e4e8494e52164c91c27af0e Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 19 Jul 2024 18:46:37 +0000 Subject: [PATCH 215/644] output more info --- ...61_lightweight_delete_projection.reference | 14 +++++--- .../03161_lightweight_delete_projection.sql | 32 ++++++++++++++++--- 2 files changed, 38 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index cb623ea2b50..960fa1dcc33 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -1,14 +1,20 @@ compact part testing throw default mode testing drop mode +all_1_1_0_2 testing rebuild mode 8888 Alice 50 -p1 -p2 +all_1_1_0_4 +all_3_3_0_4 +p1 all_3_3_0_4 +p2 all_3_3_0_4 wide part testing throw default mode testing drop mode +all_1_1_0_2 testing rebuild mode 8888 Alice 50 -p1 -p2 +all_1_1_0_4 +all_3_3_0_4 +p1 all_3_3_0_4 +p2 all_3_3_0_4 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index f2d6dcb164f..f33653fc652 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -32,9 +32,15 @@ SELECT * FROM users ORDER BY uid; SYSTEM FLUSH LOGS; --- expecting no projection +-- all_1_1_0_2 SELECT name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); + +-- expecting no projection +SELECT + name, parent_name FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); @@ -49,9 +55,15 @@ SELECT * FROM users ORDER BY uid; SYSTEM FLUSH LOGS; --- expecting projection p1, p2 +-- all_1_1_0_4, all_3_3_0_4 SELECT name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); + +-- expecting projection p1, p2 +SELECT + name, parent_name FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); @@ -84,9 +96,15 @@ SELECT * FROM users ORDER BY uid; SYSTEM FLUSH LOGS; --- expecting no projection +-- all_1_1_0_2 SELECT name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); + +-- expecting no projection +SELECT + name, parent_name FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); @@ -101,9 +119,15 @@ SELECT * FROM users ORDER BY uid; SYSTEM FLUSH LOGS; --- expecting projection p1, p2 +-- all_1_1_0_4, all_3_3_0_4 SELECT name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); + +-- expecting projection p1, p2 +SELECT + name, parent_name FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); From e3a0b6ab5ff21518a494ebede1aea47edda22b6c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 11:45:52 +0200 Subject: [PATCH 216/644] Randomize `trace_profile_events` --- tests/clickhouse-test | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 0c04d8fb2c3..e3aba5994d9 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -835,6 +835,7 @@ class SettingsRandomizer: "cross_join_min_bytes_to_compress": lambda: random.choice([0, 1, 100000000]), "min_external_table_block_size_bytes": lambda: random.choice([0, 1, 100000000]), "max_parsing_threads": lambda: random.choice([0, 1, 10]), + "trace_profile_events": lambda: random.randint(0, 1), } @staticmethod From 841012698dda51d20b3162c8a6f44fc3e870d608 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 23 Jul 2024 21:47:20 +0000 Subject: [PATCH 217/644] Add brief docs to `StorageKafka2` --- src/Storages/Kafka/KafkaConsumer2.cpp | 23 ----------------------- src/Storages/Kafka/KafkaConsumer2.h | 9 ++++----- src/Storages/Kafka/StorageKafka2.h | 22 +++++++++++++++++----- 3 files changed, 21 insertions(+), 33 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index aaefd6fd6f5..8659465a805 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -173,29 +173,6 @@ void KafkaConsumer2::pollEvents() } }; -KafkaConsumer2::TopicPartitionCounts KafkaConsumer2::getPartitionCounts() const -{ - TopicPartitionCounts result; - try - { - auto metadata = consumer->get_metadata(); - auto topic_metadatas = metadata.get_topics(); - - for (auto & topic_metadata : topic_metadatas) - { - if (const auto it = std::find(topics.begin(), topics.end(), topic_metadata.get_name()); it != topics.end()) - { - result.push_back({topic_metadata.get_name(), topic_metadata.get_partitions().size()}); - } - } - } - catch (cppkafka::HandleException & e) - { - chassert(e.what() != nullptr); - } - return result; -} - bool KafkaConsumer2::polledDataUnusable(const TopicPartition & topic_partition) const { const auto different_topic_partition = current == messages.end() diff --git a/src/Storages/Kafka/KafkaConsumer2.h b/src/Storages/Kafka/KafkaConsumer2.h index 57b157416c6..3c91df8a02f 100644 --- a/src/Storages/Kafka/KafkaConsumer2.h +++ b/src/Storages/Kafka/KafkaConsumer2.h @@ -81,10 +81,10 @@ public: ~KafkaConsumer2(); + // Poll only the main consumer queue without any topic-partition queues. This is useful to get notified about events, such as rebalance, + // new assignment, etc.. void pollEvents(); - TopicPartitionCounts getPartitionCounts() const; - auto pollTimeout() const { return poll_timeout; } inline bool hasMorePolledMessages() const { return (stalled_status == StalledStatus::NOT_STALLED) && (current != messages.end()); } @@ -96,12 +96,11 @@ public: TopicPartitions const * getKafkaAssignment() const; // As the main source of offsets is not Kafka, the offsets needs to be pushed to the consumer from outside + // Returns true if it received new assignment and internal state should be updated by updateOffsets bool needsOffsetUpdate() const { return needs_offset_update; } - - // Returns true if it received new assignment and could update the internal state accordingly, false otherwise void updateOffsets(const TopicPartitions & topic_partitions); - /// Polls batch of messages from Kafka and returns read buffer containing the next message or + /// Polls batch of messages from the given topic-partition and returns read buffer containing the next message or /// nullptr when there are no messages to process. ReadBufferPtr consume(const TopicPartition & topic_partition, const std::optional & message_count); diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index d6e564b76f5..834125d52b7 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -34,9 +34,21 @@ struct StorageKafkaInterceptors; using KafkaConsumer2Ptr = std::shared_ptr; -/** Implements a Kafka queue table engine that can be used as a persistent queue / buffer, - * or as a basic building block for creating pipelines with a continuous insertion / ETL. - */ +/// Implements a Kafka queue table engine that can be used as a persistent queue / buffer, +/// or as a basic building block for creating pipelines with a continuous insertion / ETL. +/// +/// It is similar to the already existing StorageKafka, it instead of storing the offsets +/// in Kafka, its main source of information about offsets is Keeper. On top of the +/// offsets, it also stores the number of messages (intent size) it tried to insert from +/// each topic. By storing the intent sizes it possible to retry the same batch of +/// messages in case of any errors and giving deduplication a chance to deduplicate +/// blocks. +/// +/// To not complicate things too much, the current implementation makes sure to fetch +/// messages only from a single topic-partition on a single thread at a time by +/// manipulating the queues of librdkafka. By pulling from multiple topic-partitions +/// the order of messages are not guaranteed, therefore they would have different +/// hashes for deduplication. class StorageKafka2 final : public IStorage, WithContext { using StorageKafkaInterceptors = StorageKafkaInterceptors; @@ -97,7 +109,7 @@ private: struct ConsumerAndAssignmentInfo { - KafkaConsumer2Ptr consumer; /// available consumers + KafkaConsumer2Ptr consumer; size_t consume_from_topic_partition_index{0}; TopicPartitions topic_partitions{}; zkutil::ZooKeeperPtr keeper; @@ -204,7 +216,7 @@ private: void createReplica(); void dropReplica(); - // Takes lock over topic partitions and set's the committed offset in topic_partitions + // Takes lock over topic partitions and sets the committed offset in topic_partitions. std::optional lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const TopicPartitions & topic_partitions); void saveCommittedOffset(zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition); void saveIntent(zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition, int64_t intent); From 284c0204b06f9ecd21b6eb361097b7d986342bd7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 10:11:50 +0200 Subject: [PATCH 218/644] Fix test --- .../02845_threads_count_in_distributed_queries.sql.j2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 index ffdd4e3400e..2c62279169f 100644 --- a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 +++ b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 @@ -1,5 +1,5 @@ -- enforce some defaults to be sure that the env settings will not affect the test -SET max_threads=5, async_socket_for_remote=1, prefer_localhost_replica=1, optimize_read_in_order=1, load_marks_asynchronously=0, local_filesystem_read_method='pread', remote_filesystem_read_method='read'; +SET max_threads=5, async_socket_for_remote=1, prefer_localhost_replica=1, optimize_read_in_order=1, load_marks_asynchronously=0, local_filesystem_read_method='pread', remote_filesystem_read_method='read', trace_profile_events=0; -- we use query_thread_log to check peak thread usage -- after https://github.com/ClickHouse/ClickHouse/issues/53417 there is a simpler way to check it From a3f7642d05a730dac2d5030e22947598da70cb72 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 14:01:02 +0200 Subject: [PATCH 219/644] Fix a test --- tests/queries/0_stateless/00974_query_profiler.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/00974_query_profiler.sql b/tests/queries/0_stateless/00974_query_profiler.sql index 24e4241b813..cd2f65eb94a 100644 --- a/tests/queries/0_stateless/00974_query_profiler.sql +++ b/tests/queries/0_stateless/00974_query_profiler.sql @@ -2,7 +2,9 @@ -- Tag no-fasttest: Not sure why fail even in sequential mode. Disabled for now to make some progress. SET allow_introspection_functions = 1; +SET trace_profile_events = 0; -- This can inhibit profiler from working, because it prevents sending samples from different profilers concurrently. +SET query_profiler_cpu_time_period_ns = 0; SET query_profiler_real_time_period_ns = 100000000; SET log_queries = 1; SELECT sleep(0.5), ignore('test real time query profiler'); From f37f228af9d70a04166178bac24127693d8b8c35 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 24 Jul 2024 13:06:29 +0000 Subject: [PATCH 220/644] Improve columns squashing for String/Array/Map/Variant/Dynamic types --- src/Columns/ColumnArray.cpp | 16 +++ src/Columns/ColumnArray.h | 1 + src/Columns/ColumnDynamic.cpp | 110 ++++++++++++++++++ src/Columns/ColumnDynamic.h | 2 + src/Columns/ColumnMap.cpp | 9 ++ src/Columns/ColumnMap.h | 1 + src/Columns/ColumnNullable.cpp | 16 +++ src/Columns/ColumnNullable.h | 1 + src/Columns/ColumnString.cpp | 15 +++ src/Columns/ColumnString.h | 1 + src/Columns/ColumnTuple.cpp | 13 +++ src/Columns/ColumnTuple.h | 1 + src/Columns/ColumnVariant.cpp | 21 +++- src/Columns/ColumnVariant.h | 1 + src/Columns/IColumn.h | 9 ++ src/Interpreters/Squashing.cpp | 27 +++-- tests/performance/insert_select_squashing.xml | 23 ++++ .../03210_dynamic_squashing.reference | 8 ++ .../0_stateless/03210_dynamic_squashing.sql | 20 ++++ 19 files changed, 285 insertions(+), 10 deletions(-) create mode 100644 tests/performance/insert_select_squashing.xml create mode 100644 tests/queries/0_stateless/03210_dynamic_squashing.reference create mode 100644 tests/queries/0_stateless/03210_dynamic_squashing.sql diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 19cce678cc7..9244d75a04d 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -452,6 +452,22 @@ void ColumnArray::reserve(size_t n) getData().reserve(n); /// The average size of arrays is not taken into account here. Or it is considered to be no more than 1. } +void ColumnArray::prepareForSquashing(const Columns & source_columns) +{ + size_t new_size = size(); + Columns source_data_columns; + source_data_columns.reserve(source_columns.size()); + for (const auto & source_column : source_columns) + { + const auto & source_array_column = assert_cast(*source_column); + new_size += source_array_column.size(); + source_data_columns.push_back(source_array_column.getDataPtr()); + } + + getOffsets().reserve_exact(new_size); + data->prepareForSquashing(source_data_columns); +} + void ColumnArray::shrinkToFit() { getOffsets().shrink_to_fit(); diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 63affb86d9d..d6f71b72940 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -118,6 +118,7 @@ public: void updatePermutationWithCollation(const Collator & collator, PermutationSortDirection direction, PermutationSortStability stability, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override; void reserve(size_t n) override; + void prepareForSquashing(const Columns & source_columns) override; void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index a92d54dd675..74b7ef69d8d 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -643,6 +643,116 @@ ColumnPtr ColumnDynamic::compress() const }); } +void ColumnDynamic::prepareForSquashing(const Columns & source_columns) +{ + if (source_columns.empty()) + return; + + /// Internal variants of source dynamic columns may differ. + /// We want to preallocate memory for all variants we will have after squashing. + /// It may happen that the total number of variants in source columns will + /// exceed the limit, in this case we will choose the most frequent variants. + + /// First, preallocate memory for variant discriminators and offsets. + size_t new_size = size(); + for (const auto & source_column : source_columns) + new_size += source_column->size(); + auto & variant_col = getVariantColumn(); + variant_col.getLocalDiscriminators().reserve_exact(new_size); + variant_col.getOffsets().reserve_exact(new_size); + + /// Second, collect all variants and their total sizes. + std::unordered_map total_variant_sizes; + DataTypes all_variants; + + auto add_variants = [&](const ColumnDynamic & source_dynamic) + { + const auto & source_variant_column = source_dynamic.getVariantColumn(); + const auto & source_variant_info = source_dynamic.getVariantInfo(); + const auto & source_variants = assert_cast(*source_variant_info.variant_type).getVariants(); + + for (size_t i = 0; i != source_variants.size(); ++i) + { + const auto & variant_name = source_variant_info.variant_names[i]; + auto it = total_variant_sizes.find(variant_name); + /// Add this variant to the list of all variants if we didn't see it yet. + if (it == total_variant_sizes.end()) + { + all_variants.push_back(source_variants[i]); + it = total_variant_sizes.emplace(variant_name, 0).first; + } + + it->second += source_variant_column.getVariantByGlobalDiscriminator(i).size(); + } + }; + + for (const auto & source_column : source_columns) + add_variants(assert_cast(*source_column)); + + /// Add variants from this dynamic column. + add_variants(*this); + + DataTypePtr result_variant_type; + /// Check if the number of all variants exceeds the limit. + if (all_variants.size() > max_dynamic_types || (all_variants.size() == max_dynamic_types && !total_variant_sizes.contains("String"))) + { + /// We want to keep the most frequent variants in the resulting dynamic column. + DataTypes result_variants; + result_variants.reserve(max_dynamic_types); + /// Add variants from current variant column as we will not rewrite it. + for (const auto & variant : assert_cast(*variant_info.variant_type).getVariants()) + result_variants.push_back(variant); + /// Add String variant in advance (if we didn't add it yet) as we must have it across variants when we reach the limit. + if (!variant_info.variant_name_to_discriminator.contains("String")) + result_variants.push_back(std::make_shared()); + + /// Create list of remaining variants with their sizes and sort it. + std::vector> variants_with_sizes; + variants_with_sizes.reserve(all_variants.size() - variant_info.variant_names.size()); + for (const auto & variant : all_variants) + { + /// Add variant to the list only of we didn't add it yet. + auto variant_name = variant->getName(); + if (variant_name != "String" && !variant_info.variant_name_to_discriminator.contains(variant_name)) + variants_with_sizes.emplace_back(total_variant_sizes[variant->getName()], variant); + } + + std::sort(variants_with_sizes.begin(), variants_with_sizes.end(), std::greater()); + /// Add the most frequent variants until we reach max_dynamic_types. + size_t num_new_variants = max_dynamic_types - result_variants.size(); + for (size_t i = 0; i != num_new_variants; ++i) + result_variants.push_back(variants_with_sizes[i].second); + + result_variant_type = std::make_shared(result_variants); + } + else + { + result_variant_type = std::make_shared(all_variants); + } + + if (!result_variant_type->equals(*variant_info.variant_type)) + updateVariantInfoAndExpandVariantColumn(result_variant_type); + + /// Now current dynamic column has all resulting variants and we can call + /// prepareForSquashing on them to preallocate the memory. + for (size_t i = 0; i != variant_info.variant_names.size(); ++i) + { + Columns source_variant_columns; + source_variant_columns.reserve(source_columns.size()); + for (const auto & source_column : source_columns) + { + const auto & source_dynamic_column = assert_cast(*source_column); + const auto & source_variant_info = source_dynamic_column.getVariantInfo(); + /// Try to find this variant in the current source column. + auto it = source_variant_info.variant_name_to_discriminator.find(variant_info.variant_names[i]); + if (it != source_variant_info.variant_name_to_discriminator.end()) + source_variant_columns.push_back(source_dynamic_column.getVariantColumn().getVariantPtrByGlobalDiscriminator(it->second)); + } + + variant_col.getVariantByGlobalDiscriminator(i).prepareForSquashing(source_variant_columns); + } +} + void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source_columns) { if (!empty()) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index e92cabd3db9..cb3a896d2cb 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -254,6 +254,8 @@ public: variant_column->reserve(n); } + void prepareForSquashing(const Columns & source_columns) override; + void ensureOwnership() override { variant_column->ensureOwnership(); diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 1025b4e77b9..3bab20dfbf2 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -249,6 +249,15 @@ void ColumnMap::reserve(size_t n) nested->reserve(n); } +void ColumnMap::prepareForSquashing(const Columns & source_columns) +{ + Columns nested_source_columns; + nested_source_columns.reserve(source_columns.size()); + for (const auto & source_column : source_columns) + nested_source_columns.push_back(assert_cast(*source_column).getNestedColumnPtr()); + nested->prepareForSquashing(nested_source_columns); +} + void ColumnMap::shrinkToFit() { nested->shrinkToFit(); diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 3eaaa0ad562..191476839f1 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -94,6 +94,7 @@ public: void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override; void reserve(size_t n) override; + void prepareForSquashing(const Columns & source_columns) override; void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 6529f0b78db..2a25cac6461 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -706,6 +706,22 @@ void ColumnNullable::reserve(size_t n) getNullMapData().reserve(n); } +void ColumnNullable::prepareForSquashing(const Columns & source_columns) +{ + size_t new_size = size(); + Columns nested_source_columns; + nested_source_columns.reserve(source_columns.size()); + for (const auto & source_column : source_columns) + { + const auto & source_nullable_column = assert_cast(*source_column); + new_size += source_nullable_column.size(); + nested_source_columns.push_back(source_nullable_column.getNestedColumnPtr()); + } + + nested_column->prepareForSquashing(nested_source_columns); + getNullMapData().reserve(new_size); +} + void ColumnNullable::shrinkToFit() { getNestedColumn().shrinkToFit(); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index fe9f5b6dcc2..2c32e0fe5a0 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -125,6 +125,7 @@ public: size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override; size_t estimateCardinalityInPermutedRange(const Permutation & permutation, const EqualRange & equal_range) const override; void reserve(size_t n) override; + void prepareForSquashing(const Columns & source_columns) override; void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 7cfa2571f5a..9ed2c7e3d4d 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -557,6 +557,21 @@ void ColumnString::reserve(size_t n) offsets.reserve_exact(n); } +void ColumnString::prepareForSquashing(const Columns & source_columns) +{ + size_t new_size = size(); + size_t new_chars_size = chars.size(); + for (const auto & source_column : source_columns) + { + const auto & source_string_column = assert_cast(*source_column); + new_size += source_string_column.size(); + new_chars_size += source_string_column.chars.size(); + } + + offsets.reserve_exact(new_size); + chars.reserve_exact(new_chars_size); +} + void ColumnString::shrinkToFit() { chars.shrink_to_fit(); diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index c1012e1e55e..20cd950fe9b 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -283,6 +283,7 @@ public: ColumnPtr compress() const override; void reserve(size_t n) override; + void prepareForSquashing(const Columns & source_columns) override; void shrinkToFit() override; void getExtremes(Field & min, Field & max) const override; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 4fc3f88a87c..c6ee7d775ae 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -595,6 +595,19 @@ void ColumnTuple::reserve(size_t n) getColumn(i).reserve(n); } +void ColumnTuple::prepareForSquashing(const Columns & source_columns) +{ + const size_t tuple_size = columns.size(); + for (size_t i = 0; i < tuple_size; ++i) + { + Columns nested_columns; + nested_columns.reserve(source_columns.size()); + for (const auto & source_column : source_columns) + nested_columns.push_back(assert_cast(*source_column).getColumnPtr(i)); + getColumn(i).prepareForSquashing(nested_columns); + } +} + void ColumnTuple::shrinkToFit() { const size_t tuple_size = columns.size(); diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 16b47a993f6..ef396d6a130 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -110,6 +110,7 @@ public: void updatePermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges& equal_ranges) const override; void reserve(size_t n) override; + void prepareForSquashing(const Columns & source_columns) override; void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index de7efb41d19..68e19861c38 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -1247,8 +1247,25 @@ void ColumnVariant::updatePermutation(IColumn::PermutationSortDirection directio void ColumnVariant::reserve(size_t n) { - local_discriminators->reserve(n); - offsets->reserve(n); + getLocalDiscriminators().reserve_exact(n); + getOffsets().reserve_exact(n); +} + +void ColumnVariant::prepareForSquashing(const Columns & source_columns) +{ + size_t new_size = size(); + for (const auto & source_column : source_columns) + new_size += source_column->size(); + reserve(new_size); + + for (size_t i = 0; i != variants.size(); ++i) + { + Columns source_variant_columns; + source_variant_columns.reserve(source_columns.size()); + for (const auto & source_column : source_columns) + source_variant_columns.push_back(assert_cast(*source_column).getVariantPtrByGlobalDiscriminator(i)); + getVariantByGlobalDiscriminator(i).prepareForSquashing(source_variant_columns); + } } void ColumnVariant::ensureOwnership() diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 34c24b5428d..737eb27abfe 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -237,6 +237,7 @@ public: size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override; void reserve(size_t n) override; + void prepareForSquashing(const Columns & source_columns) override; void ensureOwnership() override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index f9c1a3e7034..edcb9f0bc30 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -475,6 +475,15 @@ public: /// It affects performance only (not correctness). virtual void reserve(size_t /*n*/) {} + /// Reserve memory before squashing all specified source columns into this column. + virtual void prepareForSquashing(const std::vector & source_columns) + { + size_t new_size = size(); + for (const auto & source_column : source_columns) + new_size += source_column->size(); + reserve(new_size); + } + /// Requests the removal of unused capacity. /// It is a non-binding request to reduce the capacity of the underlying container to its size. virtual void shrinkToFit() {} diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 488177c3b4f..5cd40974c45 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -5,7 +5,6 @@ #include #include - namespace DB { @@ -114,20 +113,32 @@ Chunk Squashing::squash(std::vector && input_chunks, Chunk::ChunkInfoColl { auto & first_chunk = input_chunks[0]; Columns columns = first_chunk.detachColumns(); + mutable_columns.reserve(columns.size()); for (auto & column : columns) - { mutable_columns.push_back(IColumn::mutate(std::move(column))); - mutable_columns.back()->reserve(rows); - } } + size_t num_columns = mutable_columns.size(); + /// Collect the list of source columns for each column. + std::vector source_columns_list(num_columns, Columns{}); + for (size_t i = 0; i != num_columns; ++i) + source_columns_list[i].reserve(input_chunks.size() - 1); + for (size_t i = 1; i < input_chunks.size(); ++i) // We've already processed the first chunk above { - Columns columns = input_chunks[i].detachColumns(); - for (size_t j = 0, size = mutable_columns.size(); j < size; ++j) + auto columns = input_chunks[i].detachColumns(); + for (size_t j = 0; j != num_columns; ++j) + source_columns_list[j].emplace_back(std::move(columns[j])); + } + + for (size_t i = 0; i != num_columns; ++i) + { + /// We know all the data we will insert in advance and can make all necessary pre-allocations. + mutable_columns[i]->prepareForSquashing(source_columns_list[i]); + for (auto & source_column : source_columns_list[i]) { - const auto source_column = columns[j]; - mutable_columns[j]->insertRangeFrom(*source_column, 0, source_column->size()); + auto column = std::move(source_column); + mutable_columns[i]->insertRangeFrom(*column, 0, column->size()); } } diff --git a/tests/performance/insert_select_squashing.xml b/tests/performance/insert_select_squashing.xml new file mode 100644 index 00000000000..4c2c88f3d22 --- /dev/null +++ b/tests/performance/insert_select_squashing.xml @@ -0,0 +1,23 @@ + + + 1000 + + + +CREATE TABLE squash_performance +( + s1 String, + s2 Nullable(String), + a1 Array(Array(String)), + a2 Array(Array(UInt32)), + m1 Map(String, Array(String)), + m2 Map(String, Array(UInt64)), + t Tuple(String, Array(String), Map(String, String)) +) +ENGINE = Null; + + + INSERT INTO squash_performance SELECT * FROM generateRandom(42) LIMIT 500000 + + DROP TABLE IF EXISTS squash_performance + diff --git a/tests/queries/0_stateless/03210_dynamic_squashing.reference b/tests/queries/0_stateless/03210_dynamic_squashing.reference new file mode 100644 index 00000000000..4f5b5ba098c --- /dev/null +++ b/tests/queries/0_stateless/03210_dynamic_squashing.reference @@ -0,0 +1,8 @@ +Array(UInt8) +None +UInt64 +None +String +UInt64 +String +UInt64 diff --git a/tests/queries/0_stateless/03210_dynamic_squashing.sql b/tests/queries/0_stateless/03210_dynamic_squashing.sql new file mode 100644 index 00000000000..23b47184e33 --- /dev/null +++ b/tests/queries/0_stateless/03210_dynamic_squashing.sql @@ -0,0 +1,20 @@ +set allow_experimental_dynamic_type = 1; +set max_block_size = 1000; + +drop table if exists test; + +create table test (d Dynamic) engine=MergeTree order by tuple(); +insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=2), number < 3000, range(number % 5)::Dynamic(max_types=2), number::Dynamic(max_types=2)) from numbers(1000000); +select distinct dynamicType(d) as type from test order by type; + +drop table test; +create table test (d Dynamic(max_types=2)) engine=MergeTree order by tuple(); +insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=2), number < 3000, range(number % 5)::Dynamic(max_types=2), number::Dynamic(max_types=2)) from numbers(1000000); +select distinct dynamicType(d) as type from test order by type; + +truncate table test; +insert into test select multiIf(number < 1000, 'Str'::Dynamic(max_types=2), number < 3000, range(number % 5)::Dynamic(max_types=2), number::Dynamic(max_types=2)) from numbers(1000000); +select distinct dynamicType(d) as type from test order by type; + +drop table test; + From 3cbb3dc55f6582bc8abc7d5683080702080adcd8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 26 Jul 2024 08:41:44 +0000 Subject: [PATCH 221/644] Do not spam logs with messages related to connection reset by peer --- src/Server/HTTP/HTTPServerConnection.cpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/src/Server/HTTP/HTTPServerConnection.cpp b/src/Server/HTTP/HTTPServerConnection.cpp index 047db014560..8eb2ecb1224 100644 --- a/src/Server/HTTP/HTTPServerConnection.cpp +++ b/src/Server/HTTP/HTTPServerConnection.cpp @@ -97,6 +97,18 @@ void HTTPServerConnection::run() { sendErrorResponse(session, Poco::Net::HTTPResponse::HTTP_BAD_REQUEST); } + catch (const Poco::Net::NetException & e) + { + /// Do not spam logs with messages related to connection reset by peer. + if (e.code() == POCO_ENOTCONN) + break; + + if (session.networkException()) + session.networkException()->rethrow(); + else + throw; + } + catch (const Poco::Exception &) { if (session.networkException()) From 73e71e7b7a476c7073154a112dd12815bbe49bfe Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 26 Jul 2024 08:45:43 +0000 Subject: [PATCH 222/644] log --- src/Server/HTTP/HTTPServerConnection.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Server/HTTP/HTTPServerConnection.cpp b/src/Server/HTTP/HTTPServerConnection.cpp index 8eb2ecb1224..39e066005b9 100644 --- a/src/Server/HTTP/HTTPServerConnection.cpp +++ b/src/Server/HTTP/HTTPServerConnection.cpp @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -101,7 +102,10 @@ void HTTPServerConnection::run() { /// Do not spam logs with messages related to connection reset by peer. if (e.code() == POCO_ENOTCONN) + { + LOG_DEBUG(LogFrequencyLimiter(getLogger("HTTPServerConnection"), 10), "Connection reset by peer while processing HTTP request: {}", e.message()); break; + } if (session.networkException()) session.networkException()->rethrow(); From 0d8a3f13e39c9e0be87c8459db9582c7c336617f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 26 Jul 2024 21:49:41 +0100 Subject: [PATCH 223/644] impl --- src/Processors/Sources/ShellCommandSource.cpp | 16 ++++++++++++---- .../test_executable_dictionary/test.py | 1 + 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 55eaf67eb3b..1659287c227 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -8,13 +8,15 @@ #include #include -#include -#include -#include -#include #include +#include +#include +#include +#include + #include +#include namespace DB { @@ -137,9 +139,15 @@ public: while (!bytes_read) { + LOG_TRACE( + getLogger("TimeoutReadBufferFromFileDescriptor"), + "Starting polling on descriptors ({}) with timeout {} ms", + fmt::join(std::span(pfds, pfds + num_pfds) | std::views::transform([](const auto & pollfd) { return pollfd.fd; }), ", "), + timeout_milliseconds); pfds[0].revents = 0; pfds[1].revents = 0; size_t num_events = pollWithTimeout(pfds, num_pfds, timeout_milliseconds); + LOG_TRACE(getLogger("TimeoutReadBufferFromFileDescriptor"), "Poll returned with num_events={}", num_events); if (0 == num_events) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Pipe read timeout exceeded {} milliseconds", timeout_milliseconds); diff --git a/tests/integration/test_executable_dictionary/test.py b/tests/integration/test_executable_dictionary/test.py index 22f3442bb95..a1de429a235 100644 --- a/tests/integration/test_executable_dictionary/test.py +++ b/tests/integration/test_executable_dictionary/test.py @@ -245,6 +245,7 @@ def test_executable_input_slow_python(started_cluster): ) +@pytest.mark.repeat(50) def test_executable_implicit_input_slow_python(started_cluster): skip_test_msan(node) assert node.query_and_get_error( From 1df26ab20e37d62c7d0c33bd2a5e112a4425b146 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jul 2024 21:29:22 +0200 Subject: [PATCH 224/644] Remove interference between profilers --- tests/queries/0_stateless/01526_max_untracked_memory.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01526_max_untracked_memory.sh b/tests/queries/0_stateless/01526_max_untracked_memory.sh index b2bad637422..173f6fbe2e6 100755 --- a/tests/queries/0_stateless/01526_max_untracked_memory.sh +++ b/tests/queries/0_stateless/01526_max_untracked_memory.sh @@ -14,7 +14,7 @@ min_trace_entries=2 # do not use _, they should be escaped for LIKE query_id_tcp_prefix="01526-tcp-memory-tracking-$RANDOM-$$" -${CLICKHOUSE_CLIENT} --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 -q "with '$query_id_tcp_prefix' as __id $query FORMAT Null" +${CLICKHOUSE_CLIENT} --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 --trace_profile_events 0 -q "with '$query_id_tcp_prefix' as __id $query FORMAT Null" ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" query_id_tcp="$(${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT query_id FROM system.query_log WHERE current_database = currentDatabase() AND query LIKE '%$query_id_tcp_prefix%'")" ${CLICKHOUSE_CLIENT} -q "SELECT count()>=$min_trace_entries FROM system.trace_log WHERE query_id = '$query_id_tcp' AND abs(size) < 4e6 AND event_time >= now() - interval 1 hour" @@ -23,7 +23,7 @@ ${CLICKHOUSE_CLIENT} -q "SELECT count()>=$min_trace_entries FROM system.trace_lo # query_id cannot be longer then 28 bytes query_id_http="01526_http_${RANDOM}_$$" -echo "$query" | ${CLICKHOUSE_CURL} -sSg -o /dev/null "${CLICKHOUSE_URL}&query_id=$query_id_http&max_untracked_memory=0&memory_profiler_sample_probability=1&max_threads=1" -d @- +echo "$query" | ${CLICKHOUSE_CURL} -sSg -o /dev/null "${CLICKHOUSE_URL}&query_id=$query_id_http&max_untracked_memory=0&memory_profiler_sample_probability=1&max_threads=1&trace_profile_events=0" -d @- ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" # at least 2, one allocation, one deallocation # (but actually even more) From 08229390dfa3263853662e0f39fa393f61b95a44 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 29 Jul 2024 06:17:35 +0000 Subject: [PATCH 225/644] protect socket timeouts --- src/Server/TCPHandler.cpp | 7 ++++++- src/Server/TCPHandler.h | 3 +++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 29568baba58..d184074729b 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -366,7 +366,10 @@ void TCPHandler::runImpl() try { /// If a user passed query-local timeouts, reset socket to initial state at the end of the query - SCOPE_EXIT({state.timeout_setter.reset();}); + SCOPE_EXIT({ + std::scoped_lock lock(in_mutex, out_mutex); + state.timeout_setter.reset(); + }); /** If Query - process it. If Ping or Cancel - go back to the beginning. * There may come settings for a separate query that modify `query_context`. @@ -779,6 +782,8 @@ void TCPHandler::extractConnectionSettingsFromContext(const ContextPtr & context bool TCPHandler::readDataNext() { + std::scoped_lock lock(in_mutex); + Stopwatch watch(CLOCK_MONOTONIC_COARSE); /// Poll interval should not be greater than receive_timeout diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 74afb5a14a5..50ef6bcf20d 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -229,6 +229,9 @@ private: /// `out_mutex` protects `out` (WriteBuffer). /// So it is used for method sendData(), sendProgress(), sendLogs(), etc. std::mutex out_mutex; + /// `in_mutex` protects `in` (ReadBuffer) + /// Used in readDataNext() and to protect socket timeout settings + std::mutex in_mutex; /// `task_callback_mutex` protects tasks callbacks. /// Inside these callbacks we might also change cancellation status, /// so it also protects cancellation status checks. From d5065a43ae4ae5ba0f068e3fdf5952dd5319f561 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 29 Jul 2024 17:55:01 +0200 Subject: [PATCH 226/644] Update StorageObjectStorage.cpp --- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index ca0ced8dcd3..d9c82d68791 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -39,11 +39,16 @@ String StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata, Con auto query_settings = configuration->getQuerySettings(context); /// We don't want to throw an exception if there are no files with specified path. query_settings.throw_on_zero_files_match = false; + + bool local_distributed_processing = distributed_processing; + if (context->getSettingsRef().use_hive_partitioning) + local_distributed_processing = false; + auto file_iterator = StorageObjectStorageSource::createFileIterator( configuration, query_settings, object_storage, - distributed_processing, + local_distributed_processing, context, {}, // predicate metadata.getColumns().getAll(), // virtual_columns From f9a5210bacc418e354ddcf8893fa8c5a291b46d4 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 29 Jul 2024 19:36:31 +0200 Subject: [PATCH 227/644] solve Alexey's review --- src/Storages/VirtualColumnUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 87c1aecc3a7..257a77547c0 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -127,7 +127,7 @@ NameSet getVirtualNamesForFileLikeStorage() std::unordered_map parseHivePartitioningKeysAndValues(const String & path, const ColumnsDescription & storage_columns) { - std::string pattern = "([^/]+)=([^/]+)/"; + std::string pattern = "([^/])=([^/]+)/"; re2::StringPiece input_piece(path); std::unordered_map key_values; From 6317979825794882905bc02b3a18dd82cfd8ec1c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 Jul 2024 20:53:11 +0100 Subject: [PATCH 228/644] add one more --- tests/integration/test_executable_table_function/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_executable_table_function/test.py b/tests/integration/test_executable_table_function/test.py index 801a3c7c14a..a79616fc008 100644 --- a/tests/integration/test_executable_table_function/test.py +++ b/tests/integration/test_executable_table_function/test.py @@ -139,6 +139,7 @@ def test_executable_function_input_signalled_python(started_cluster): assert node.query(query.format(source="(SELECT id FROM test_data_table)")) == "" +@pytest.mark.repeat(50) def test_executable_function_input_slow_python(started_cluster): skip_test_msan(node) From 18327bdf9c7a08d0f88683d51f930ef83b23127f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 29 Jul 2024 22:17:02 +0000 Subject: [PATCH 229/644] move timeout reset to after pipeline reset --- src/Server/TCPHandler.cpp | 10 ++-------- src/Server/TCPHandler.h | 3 --- 2 files changed, 2 insertions(+), 11 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d184074729b..06feeadb892 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -365,12 +365,6 @@ void TCPHandler::runImpl() try { - /// If a user passed query-local timeouts, reset socket to initial state at the end of the query - SCOPE_EXIT({ - std::scoped_lock lock(in_mutex, out_mutex); - state.timeout_setter.reset(); - }); - /** If Query - process it. If Ping or Cancel - go back to the beginning. * There may come settings for a separate query that modify `query_context`. * It's possible to receive part uuids packet before the query, so then receivePacket has to be called twice. @@ -633,6 +627,8 @@ void TCPHandler::runImpl() state.io.onException(); exception.reset(e.clone()); + state.timeout_setter.reset(); + if (e.code() == ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT) throw; @@ -782,8 +778,6 @@ void TCPHandler::extractConnectionSettingsFromContext(const ContextPtr & context bool TCPHandler::readDataNext() { - std::scoped_lock lock(in_mutex); - Stopwatch watch(CLOCK_MONOTONIC_COARSE); /// Poll interval should not be greater than receive_timeout diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 50ef6bcf20d..74afb5a14a5 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -229,9 +229,6 @@ private: /// `out_mutex` protects `out` (WriteBuffer). /// So it is used for method sendData(), sendProgress(), sendLogs(), etc. std::mutex out_mutex; - /// `in_mutex` protects `in` (ReadBuffer) - /// Used in readDataNext() and to protect socket timeout settings - std::mutex in_mutex; /// `task_callback_mutex` protects tasks callbacks. /// Inside these callbacks we might also change cancellation status, /// so it also protects cancellation status checks. From b0e6b3e88930d3ca493dddb688235c64cec1d893 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 30 Jul 2024 06:30:12 +0000 Subject: [PATCH 230/644] Kick off CI build From 0c9fa155d4993220c00e4b41c0354b20d3312f33 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 30 Jul 2024 11:58:35 +0200 Subject: [PATCH 231/644] revert last commit --- src/Storages/VirtualColumnUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 4edab01925d..f16eff7edb6 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -131,7 +131,7 @@ NameSet getVirtualNamesForFileLikeStorage() std::unordered_map parseHivePartitioningKeysAndValues(const String & path, const ColumnsDescription & storage_columns) { - std::string pattern = "([^/])=([^/]+)/"; + std::string pattern = "([^/]+)=([^/]+)/"; re2::StringPiece input_piece(path); std::unordered_map key_values; From d3830e0a4fd96c286e5b91a78b1a9583f5eb6291 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 30 Jul 2024 14:54:42 +0000 Subject: [PATCH 232/644] Add perf tests for Dynamic and Variant --- .../insert_select_squashing_dynamic.xml | 59 +++++++++++++++++++ .../insert_select_squashing_variant.xml | 34 +++++++++++ 2 files changed, 93 insertions(+) create mode 100644 tests/performance/insert_select_squashing_dynamic.xml create mode 100644 tests/performance/insert_select_squashing_variant.xml diff --git a/tests/performance/insert_select_squashing_dynamic.xml b/tests/performance/insert_select_squashing_dynamic.xml new file mode 100644 index 00000000000..f7f600fd8bd --- /dev/null +++ b/tests/performance/insert_select_squashing_dynamic.xml @@ -0,0 +1,59 @@ + + + 1000 + 0 + 1 + + + +CREATE TABLE dynamic_squash_performance_1 +( + d Dynamic +) +ENGINE = Null; + + + +CREATE TABLE dynamic_squash_performance_2 +( + d Dynamic(max_types=6) +) +ENGINE = Null; + + + +CREATE TABLE src_dynamic_squash_performance_1 +( + d Dynamic +) +ENGINE = Memory; + + + +CREATE TABLE src_dynamic_squash_performance_2 +( + d Dynamic(max_types=6) +) +ENGINE = Memory; + + + + + + + + + + + + INSERT INTO dynamic_squash_performance_1 SELECT number::Dynamic FROM numbers(10000000) + INSERT INTO dynamic_squash_performance_1 SELECT range(number % 100)::Dynamic FROM numbers(2000000) + INSERT INTO dynamic_squash_performance_1 SELECT * FROM src_dynamic_squash_performance_1 + INSERT INTO dynamic_squash_performance_2 SELECT * FROM src_dynamic_squash_performance_2 + + DROP TABLE IF EXISTS dynamic_squash_performance_1 + DROP TABLE IF EXISTS dynamic_squash_performance_2 + DROP TABLE IF EXISTS src_dynamic_squash_performance_1 + DROP TABLE IF EXISTS src_dynamic_squash_performance_2 + + diff --git a/tests/performance/insert_select_squashing_variant.xml b/tests/performance/insert_select_squashing_variant.xml new file mode 100644 index 00000000000..5c59fc7b50f --- /dev/null +++ b/tests/performance/insert_select_squashing_variant.xml @@ -0,0 +1,34 @@ + + + 1000 + 0 + 1 + 1 + + + +CREATE TABLE variant_squash_performance +( + v Variant(Tuple(v1 Array(UInt64)), Tuple(v2 Array(UInt64)), Tuple(v3 Array(UInt64)), Tuple(v4 Array(UInt64)), Tuple(v5 Array(UInt64))) +) +ENGINE = Null; + + + +CREATE TABLE src_variant_squash_performance +( + v Variant(Tuple(v1 Array(UInt64)), Tuple(v2 Array(UInt64)), Tuple(v3 Array(UInt64)), Tuple(v4 Array(UInt64)), Tuple(v5 Array(UInt64))) +) +ENGINE = Memory; + + + + + + + INSERT INTO variant_squash_performance SELECT * FROM src_variant_squash_performance + + DROP TABLE IF EXISTS variant_squash_performance + DROP TABLE IF EXISTS src_variant_squash_performance + + From 9d0608ce001b35e17ff81c00d4965dbe4938b56b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 30 Jul 2024 16:55:37 +0200 Subject: [PATCH 233/644] Update Runner.cpp --- utils/keeper-bench/Runner.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index d99f2645a31..59761d827e1 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -545,8 +545,7 @@ struct ZooKeeperRequestFromLogReader file_read_buf = DB::wrapReadBufferWithCompressionMethod(std::move(file_read_buf), compression_method); DB::SingleReadBufferIterator read_buffer_iterator(std::move(file_read_buf)); - std::string sample_path; - auto [columns_description, format] = DB::detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); + auto [columns_description, format] = DB::detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); DB::ColumnsWithTypeAndName columns; columns.reserve(columns_description.size()); From bd83ba88b03b7de15fbc7530d6c827156674f3b5 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 30 Jul 2024 18:48:34 +0000 Subject: [PATCH 234/644] add comments --- src/Server/TCPHandler.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 06feeadb892..13ec2ab102e 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -394,7 +394,8 @@ void TCPHandler::runImpl() /// So it's better to update the connection settings for flexibility. extractConnectionSettingsFromContext(query_context); - /// Sync timeouts on client and server during current query to avoid dangling queries on server + /// Sync timeouts on client and server during current query to avoid dangling queries on server. + /// It should be reset at the end of query. state.timeout_setter = std::make_unique(socket(), send_timeout, receive_timeout); /// Should we send internal logs to client? @@ -602,6 +603,7 @@ void TCPHandler::runImpl() /// QueryState should be cleared before QueryScope, since otherwise /// the MemoryTracker will be wrong for possible deallocations. /// (i.e. deallocations from the Aggregator with two-level aggregation) + /// Also it resets socket's timeouts. state.reset(); last_sent_snapshots = ProfileEvents::ThreadIdToCountersSnapshot{}; query_scope.reset(); @@ -627,6 +629,7 @@ void TCPHandler::runImpl() state.io.onException(); exception.reset(e.clone()); + /// In case of exception state was not reset, so socket's timouts must be reset explicitly state.timeout_setter.reset(); if (e.code() == ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT) From 08ecf6c6642f7fba6f5503f5121c832b982de945 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 30 Jul 2024 20:07:37 +0000 Subject: [PATCH 235/644] Reset formatter after buffer is restarted Formatter can use buffer to initialize some of its members. If buffer is restarted after those members are initialized, then the buffer restart might invalidate pointers/references/iterators hold into buffer. --- src/Storages/MessageQueueSink.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MessageQueueSink.cpp b/src/Storages/MessageQueueSink.cpp index 36899011e33..d4dabd60ef8 100644 --- a/src/Storages/MessageQueueSink.cpp +++ b/src/Storages/MessageQueueSink.cpp @@ -60,12 +60,12 @@ void MessageQueueSink::consume(Chunk & chunk) row_format->writeRow(columns, row); } row_format->finalize(); - row_format->resetFormatter(); producer->produce(buffer->str(), i, columns, row - 1); /// Reallocate buffer if it's capacity is large then DBMS_DEFAULT_BUFFER_SIZE, /// because most likely in this case we serialized abnormally large row /// and won't need this large allocated buffer anymore. buffer->restart(DBMS_DEFAULT_BUFFER_SIZE); + row_format->resetFormatter(); } } else @@ -73,8 +73,8 @@ void MessageQueueSink::consume(Chunk & chunk) format->write(getHeader().cloneWithColumns(chunk.detachColumns())); format->finalize(); producer->produce(buffer->str(), chunk.getNumRows(), columns, chunk.getNumRows() - 1); - format->resetFormatter(); buffer->restart(); + format->resetFormatter(); } } From 956f8762fef7473804f7d82d63f076e09736f42c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 31 Jul 2024 05:11:34 +0000 Subject: [PATCH 236/644] fix after merge --- src/Client/ClientApplicationBase.cpp | 37 ++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp index 9f133616d2e..1b2ae16a479 100644 --- a/src/Client/ClientApplicationBase.cpp +++ b/src/Client/ClientApplicationBase.cpp @@ -158,6 +158,8 @@ void ClientApplicationBase::init(int argc, char ** argv) ("config-file,C", po::value(), "config-file path") + ("proto_caps", po::value(), "enable/disable chunked protocol: chunked_optional, notchunked, notchunked_optional, send_chunked, send_chunked_optional, send_notchunked, send_notchunked_optional, recv_chunked, recv_chunked_optional, recv_notchunked, recv_notchunked_optional") + ("query,q", po::value>()->multitoken(), R"(Query. Can be specified multiple times (--query "SELECT 1" --query "SELECT 2") or once with multiple comma-separated queries (--query "SELECT 1; SELECT 2;"). In the latter case, INSERT queries with non-VALUE format must be separated by empty lines.)") ("queries-file", po::value>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)") ("multiquery,n", "Obsolete, does nothing") @@ -339,6 +341,41 @@ void ClientApplicationBase::init(int argc, char ** argv) if (options.count("server_logs_file")) server_logs_file = options["server_logs_file"].as(); + if (options.count("proto_caps")) + { + std::string proto_caps_str = options["proto_caps"].as(); + + std::vector proto_caps; + splitInto<','>(proto_caps, proto_caps_str); + + for (auto cap_str : proto_caps) + { + std::string direction; + + if (cap_str.starts_with("send_")) + { + direction = "send"; + cap_str = cap_str.substr(std::string_view("send_").size()); + } + else if (cap_str.starts_with("recv_")) + { + direction = "recv"; + cap_str = cap_str.substr(std::string_view("recv_").size()); + } + + if (cap_str != "chunked" && cap_str != "notchunked" && cap_str != "chunked_optional" && cap_str != "notchunked_optional") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "proto_caps option is incorrect ({})", proto_caps_str); + + if (direction.empty()) + { + config().setString("proto_caps.send", std::string(cap_str)); + config().setString("proto_caps.recv", std::string(cap_str)); + } + else + config().setString("proto_caps." + direction, std::string(cap_str)); + } + } + query_processing_stage = QueryProcessingStage::fromString(options["stage"].as()); query_kind = parseQueryKind(options["query_kind"].as()); profile_events.print = options.count("print-profile-events"); From 9fb610ae10da22f521a1ab2e4442c78766d5be37 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 31 Jul 2024 11:52:36 +0200 Subject: [PATCH 237/644] fix tests --- .../queries/0_stateless/03203_hive_style_partitioning.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index 430a3582f65..a4a2e48e046 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -118,4 +118,3 @@ Eva Schmidt Elizabeth Schmidt Samuel Schmidt Elizabeth Schmidt Eva Schmidt Elizabeth Samuel Schmidt Elizabeth -Elizabeth Gordon Elizabeth Gordon From 0cd37533a1e9873632cff7dc6debbbf802a29742 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 31 Jul 2024 10:10:44 +0000 Subject: [PATCH 238/644] fix after merge --- src/Client/ClientBase.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index f8c2fb0d6bc..0c26b77bcec 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1039,10 +1039,10 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa connection_parameters.timeouts, query, query_parameters, - global_context->getCurrentQueryId(), + client_context->getCurrentQueryId(), query_processing_stage, - &global_context->getSettingsRef(), - &global_context->getClientInfo(), + &client_context->getSettingsRef(), + &client_context->getClientInfo(), true, [&](const Progress & progress) { onProgress(progress); }); From 2a47bb5d2ba777a7220ce6fb2afd3eebdb40afce Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 31 Jul 2024 18:36:46 +0200 Subject: [PATCH 239/644] Fix build --- src/Storages/Kafka/StorageKafkaCommon.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp index 13713ef6c43..883eae95a7f 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.cpp +++ b/src/Storages/Kafka/StorageKafkaCommon.cpp @@ -767,8 +767,6 @@ void drainConsumer( void eraseMessageErrors(Messages & messages, const LoggerPtr & log, ErrorHandler error_handler) { - assert(current == messages.begin()); - size_t skipped = std::erase_if( messages, [&](auto & message) From 646b7e53d7c865bd16a5017d1a0d2e71f0a39cab Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 31 Jul 2024 17:06:30 +0000 Subject: [PATCH 240/644] add timeout reset --- src/Server/TCPHandler.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 13ec2ab102e..76295a9f45e 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -690,6 +690,9 @@ void TCPHandler::runImpl() exception = std::make_unique(Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception")); } + /// In case of exception state was not reset, so socket's timouts must be reset explicitly + state.timeout_setter.reset(); + try { if (exception) From e7fc206069796f662bb31aab671cbf75b95984ec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 31 Jul 2024 17:20:01 +0000 Subject: [PATCH 241/644] Add test --- .../format_schemas/string_key_value.capnp | 6 + .../format_schemas/string_key_value.format | 1 + .../format_schemas/string_key_value.proto | 6 + .../clickhouse_path/format_schemas/test.capnp | 2 +- tests/integration/test_storage_kafka/test.py | 129 ++++++++++++++++++ 5 files changed, 143 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.capnp create mode 100644 tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.format create mode 100644 tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.proto diff --git a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.capnp b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.capnp new file mode 100644 index 00000000000..4f3eabe22f0 --- /dev/null +++ b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.capnp @@ -0,0 +1,6 @@ +@0x99f75f775fe63dae; + +struct StringKeyValuePair { + key@0 : Text; + value@1 : Text; +} diff --git a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.format b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.format new file mode 100644 index 00000000000..83dff6ce401 --- /dev/null +++ b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.format @@ -0,0 +1 @@ +(key = ${key:CSV}, value = ${value:CSV}) diff --git a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.proto b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.proto new file mode 100644 index 00000000000..71905c63bdf --- /dev/null +++ b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.proto @@ -0,0 +1,6 @@ +syntax = "proto3"; + +message StringKeyValuePair { + string key = 1; + string value = 2; +} diff --git a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp index 44f1961205b..247e7b9ceca 100644 --- a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp +++ b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp @@ -7,4 +7,4 @@ struct TestRecordStruct val1 @2 : Text; val2 @3 : Float32; val3 @4 : UInt8; -} \ No newline at end of file +} diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 8393e88db88..596933c1566 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -5097,6 +5097,135 @@ def test_multiple_read_in_materialized_views(kafka_cluster, max_retries=15): ) +def test_kafka_produce_http_interface_row_based_format(kafka_cluster): + # reproduction of #https://github.com/ClickHouse/ClickHouse/issues/61060 with validating the written messages + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + + topic_prefix = "http_row_" + + # It is important to have: + # - long enough messages + # - enough messages + # I don't know the exact requirement for message sizes, but it doesn't reproduce with short messages + # For the number of messages it seems like at least 3 messages is necessary + expected_key = "01234567890123456789" + expected_value = "aaaaabbbbbccccc" + + insert_query_end = f"(key, value) VALUES ('{expected_key}', '{expected_value}'), ('{expected_key}', '{expected_value}'), ('{expected_key}', '{expected_value}')" + insert_query_template = "INSERT INTO {table_name} " + insert_query_end + + extra_settings = { + "Protobuf": ", kafka_schema = 'string_key_value.proto:StringKeyValuePair'", + "CapnProto": ", kafka_schema='string_key_value:StringKeyValuePair'", + "Template": ", format_template_row='string_key_value.format'" + } + + # Only the formats that can be used both and input and output format are tested + # Reasons to exclude following formats: + # - JSONStrings: not actually an input format + # - ProtobufSingle: I cannot make it work to parse the messages. Probably something is broken, + # because the producer can write multiple rows into a same message, which makes them impossible to parse properly. Should added after #67549 is fixed. + # - ProtobufList: I didn't want to deal with the envelope and stuff + # - Npy: supports only single column + # - LineAsString: supports only single column + # - RawBLOB: supports only single column + formats_to_test = [ + "TabSeparated", + "TabSeparatedRaw", + "TabSeparatedWithNames", + "TabSeparatedWithNamesAndTypes", + "TabSeparatedRawWithNames", + "TabSeparatedRawWithNamesAndTypes", + "Template", + "CSV", + "CSVWithNames", + "CSVWithNamesAndTypes", + "CustomSeparated", + "CustomSeparatedWithNames", + "CustomSeparatedWithNamesAndTypes", + "Values", + "JSON", + "JSONColumns", + "JSONColumnsWithMetadata", + "JSONCompact", + "JSONCompactColumns", + "JSONEachRow", + "JSONStringsEachRow", + "JSONCompactEachRow", + "JSONCompactEachRowWithNames", + "JSONCompactEachRowWithNamesAndTypes", + "JSONCompactStringsEachRow", + "JSONCompactStringsEachRowWithNames", + "JSONCompactStringsEachRowWithNamesAndTypes", + "JSONObjectEachRow", + "BSONEachRow", + "TSKV", + "Protobuf", + "Avro", + "Parquet", + "Arrow", + "ArrowStream", + "ORC", + "RowBinary", + "RowBinaryWithNames", + "RowBinaryWithNamesAndTypes", + "Native", + "CapnProto", + "MsgPack", + ] + for format in formats_to_test: + logging.debug(f"Creating tables and writing messages to {format}") + topic = topic_prefix + format + kafka_create_topic(admin_client, topic) + + extra_setting = extra_settings.get(format, "") + + # kafka_max_rows_per_message is set to 2 to make sure every format produces at least 2 messages, thus increasing the chance of catching a bug + instance.query( + f""" + DROP TABLE IF EXISTS test.view_{topic}; + DROP TABLE IF EXISTS test.consumer_{topic}; + CREATE TABLE test.kafka_writer_{topic} (key String, value String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic}', + kafka_group_name = '{topic}', + kafka_format = '{format}', + kafka_max_rows_per_message = 2 {extra_setting}; + + CREATE TABLE test.kafka_{topic} (key String, value String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic}', + kafka_group_name = '{topic}', + kafka_format = '{format}' {extra_setting}; + + CREATE MATERIALIZED VIEW test.view_{topic} Engine=Log AS + SELECT key, value FROM test.kafka_{topic}; + """ + ) + + instance.http_query(insert_query_template.format(table_name="test.kafka_writer_"+topic), method="POST") + + expected = f"""\ +{expected_key}\t{expected_value} +{expected_key}\t{expected_value} +{expected_key}\t{expected_value} +""" + # give some times for the readers to read the messages + for format in formats_to_test: + logging.debug(f"Checking result for {format}") + topic = topic_prefix + format + + result = instance.query_with_retry(f"SELECT * FROM test.view_{topic}", check_callback=lambda res: res.count("\n") == 3) + + assert TSV(result) == TSV(expected) + + kafka_delete_topic(admin_client, topic) + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From bada9ea9be4888ab4ef1b3f7fcdf015e11d994d1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 31 Jul 2024 17:30:24 +0000 Subject: [PATCH 242/644] Automatic style fix --- tests/integration/test_storage_kafka/test.py | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 596933c1566..dd0bf1bf28f 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -5103,7 +5103,6 @@ def test_kafka_produce_http_interface_row_based_format(kafka_cluster): bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) - topic_prefix = "http_row_" # It is important to have: @@ -5120,7 +5119,7 @@ def test_kafka_produce_http_interface_row_based_format(kafka_cluster): extra_settings = { "Protobuf": ", kafka_schema = 'string_key_value.proto:StringKeyValuePair'", "CapnProto": ", kafka_schema='string_key_value:StringKeyValuePair'", - "Template": ", format_template_row='string_key_value.format'" + "Template": ", format_template_row='string_key_value.format'", } # Only the formats that can be used both and input and output format are tested @@ -5208,7 +5207,10 @@ def test_kafka_produce_http_interface_row_based_format(kafka_cluster): """ ) - instance.http_query(insert_query_template.format(table_name="test.kafka_writer_"+topic), method="POST") + instance.http_query( + insert_query_template.format(table_name="test.kafka_writer_" + topic), + method="POST", + ) expected = f"""\ {expected_key}\t{expected_value} @@ -5220,12 +5222,16 @@ def test_kafka_produce_http_interface_row_based_format(kafka_cluster): logging.debug(f"Checking result for {format}") topic = topic_prefix + format - result = instance.query_with_retry(f"SELECT * FROM test.view_{topic}", check_callback=lambda res: res.count("\n") == 3) + result = instance.query_with_retry( + f"SELECT * FROM test.view_{topic}", + check_callback=lambda res: res.count("\n") == 3, + ) assert TSV(result) == TSV(expected) kafka_delete_topic(admin_client, topic) + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 23fa85e3ff269fac0ad6aa8a9122f90ff4d70d28 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 31 Jul 2024 19:30:58 +0200 Subject: [PATCH 243/644] Apply suggestions from code review Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- docs/en/engines/table-engines/integrations/kafka.md | 2 +- src/Storages/Kafka/StorageKafka2.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index 389bb6c9029..38a9d696067 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -284,7 +284,7 @@ SETTINGS allow_experimental_kafka_offsets_storage_in_keeper=1; ### Known limitations As the new engine is experimental, it is not production ready yet. There are few known limitations of the implementation: - - The biggest limitation is the engine doesn't support direct reading from Kafka topic (insertion works, but reading doesn't), thus the direct `SELECT` queries will fail. + - The biggest limitation is the engine doesn't support direct reading. Reading from the engine using materialized views and writing to the engine work, but direct reading doesn't. As a result, all direct `SELECT` queries will fail. - Rapidly dropping and recreating the table or specifying the same ClickHouse Keeper path to different engines might cause issues. As best practice you can use the `{uuid}` in `kafka_keeper_path` to avoid clashing paths. - To make repeatable reads, messages cannot be consumed from multiple partitions on a single thread. On the other hand, the Kafka consumers have to be polled regularly to keep them alive. As a result of these two objectives, we decided to only allow creating multiple consumers if `kafka_thread_per_consumer` is enabled, otherwise it is too complicated to avoid issues regarding polling consumers regularly. - Consumers created by the new storage engine do not show up in [`system.kafka_consumers`](../../../operations/system-tables/kafka_consumers.md) table. diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index 058bff18f56..318c04f1f91 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -40,7 +40,7 @@ using KafkaConsumer2Ptr = std::shared_ptr; /// It is similar to the already existing StorageKafka, it instead of storing the offsets /// in Kafka, its main source of information about offsets is Keeper. On top of the /// offsets, it also stores the number of messages (intent size) it tried to insert from -/// each topic. By storing the intent sizes it possible to retry the same batch of +/// each topic. By storing the intent sizes it is possible to retry the same batch of /// messages in case of any errors and giving deduplication a chance to deduplicate /// blocks. /// From beb5d02cdc1f5fae58a8ee43fadb1c581868b894 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 31 Jul 2024 17:58:20 +0000 Subject: [PATCH 244/644] Move THROW back to InterpreterDelete. --- src/Interpreters/InterpreterDeleteQuery.cpp | 13 +++++++++++++ src/Interpreters/MutationsInterpreter.cpp | 10 ---------- src/Storages/MergeTree/MutateTask.cpp | 12 ++++++++---- .../03161_lightweight_delete_projection.sql | 4 ++-- 4 files changed, 23 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 0e988e7d031..3000292f047 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -85,6 +86,18 @@ BlockIO InterpreterDeleteQuery::execute() "Lightweight delete mutate is disabled. " "Set `enable_lightweight_delete` setting to enable it"); + if (metadata_snapshot->hasProjections()) + { + if (const auto * merge_tree_data = dynamic_cast(table.get())) + if (merge_tree_data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "DELETE query is not allowed for table {} because as it has projections and setting " + "lightweight_mutation_projection_mode is set to THROW. " + "User should change lightweight_mutation_projection_mode OR " + "drop all the projections manually before running the query", + table_id.getFullTableName()); + } + /// Build "ALTER ... UPDATE _row_exists = 0 WHERE predicate" query String alter_query = "ALTER TABLE " + table->getStorageID().getFullTableName() diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index db4ea9c0754..480c6736bc5 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -491,16 +491,6 @@ static void validateUpdateColumns( { if (!source.supportsLightweightDelete()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); - - if (const MergeTreeData * merge_tree_data = source.getMergeTreeData(); merge_tree_data != nullptr) - { - if (merge_tree_data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW - && merge_tree_data->hasProjection()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "DELETE query is not supported for table {} as it has projections. " - "User should drop all the projections manually before running the query", - source.getStorage()->getStorageID().getFullTableName()); - } } else if (virtual_columns.tryGet(column_name)) { diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 6245d80508b..8b5829eb058 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2320,10 +2320,14 @@ bool MutateTask::prepare() ctx->context, ctx->materialized_indices); - bool lightweight_delete_projection_drop = lightweight_delete_mode - && ctx->data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::DROP; + auto lightweight_mutation_projection_mode = ctx->data->getSettings()->lightweight_mutation_projection_mode; + bool lightweight_delete_drops_projections = + lightweight_mutation_projection_mode == LightweightMutationProjectionMode::DROP + || lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW; + + bool should_create_projections = !(lightweight_delete_mode && lightweight_delete_drops_projections); /// Under lightweight delete mode, if option is drop, projections_to_recalc should be empty. - if (!lightweight_delete_projection_drop) + if (should_create_projections) { ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate( ctx->source_part, @@ -2342,7 +2346,7 @@ bool MutateTask::prepare() ctx->projections_to_recalc, ctx->stats_to_recalc, ctx->metadata_snapshot, - lightweight_delete_projection_drop); + !should_create_projections); ctx->files_to_rename = MutationHelpers::collectFilesForRenames( ctx->source_part, diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index f33653fc652..02b880d620a 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -21,7 +21,7 @@ SELECT 'testing throw default mode'; ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; -DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } +DELETE FROM users WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } SELECT 'testing drop mode'; ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; @@ -85,7 +85,7 @@ INSERT INTO users VALUES (1231, 'John', 33); SELECT 'testing throw default mode'; ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; -DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } +DELETE FROM users WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } SELECT 'testing drop mode'; ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; From 7aff8748b027b43c48f670446c06e704b1767a35 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 31 Jul 2024 18:06:09 +0000 Subject: [PATCH 245/644] Address small review comments --- .../table-engines/integrations/kafka.md | 2 +- src/Storages/Kafka/KafkaConsumer2.cpp | 4 +-- src/Storages/Kafka/StorageKafka2.cpp | 28 ------------------- src/Storages/Kafka/StorageKafkaCommon.cpp | 2 +- 4 files changed, 4 insertions(+), 32 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index 38a9d696067..de6492e8ea7 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -257,7 +257,7 @@ If `allow_experimental_kafka_offsets_storage_in_keeper` is enabled, then two mor - `kafka_keeper_path` specifies the path to the table in ClickHouse Keeper - `kafka_replica_name` specifies the replica name in ClickHouse Keeper -Either both of the settings must be specified or neither of them. When both of them are specified, then a new, experimental Kafka engine will be used. The new engine doesn't depend on storing the committed offsets in Kafka,but stores them in ClickHouse Keeper. It still tries to commit the offsets to Kafka, but it only depends on those offsets when the table is created. In any other circumstances (table is restarted, or recovered after some error) the offsets stored in ClickHouse Keeper will be used to consume messages from. Apart from the committed offset, it also stores how many messages were consumed in the last batch, so if the insert fails, the same amount of messages will be consumed, thus enabling deduplication if necessary. +Either both of the settings must be specified or neither of them. When both of them are specified, then a new, experimental Kafka engine will be used. The new engine doesn't depend on storing the committed offsets in Kafka, but stores them in ClickHouse Keeper. It still tries to commit the offsets to Kafka, but it only depends on those offsets when the table is created. In any other circumstances (table is restarted, or recovered after some error) the offsets stored in ClickHouse Keeper will be used as an offset to continue consuming messages from. Apart from the committed offset, it also stores how many messages were consumed in the last batch, so if the insert fails, the same amount of messages will be consumed, thus enabling deduplication if necessary. Example: diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 8659465a805..761186fbbdd 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -326,7 +326,7 @@ void KafkaConsumer2::commit(const TopicPartition & topic_partition) if (e.get_error() == RD_KAFKA_RESP_ERR__NO_OFFSET) committed = true; else - LOG_WARNING(log, "Exception during commit attempt: {}", e.what()); + LOG_ERROR(log, "Exception during attempt to commit to Kafka: {}", e.what()); } } @@ -334,7 +334,7 @@ void KafkaConsumer2::commit(const TopicPartition & topic_partition) { // The failure is not the biggest issue, it only counts when a table is dropped and recreated, otherwise the offsets are taken from keeper. ProfileEvents::increment(ProfileEvents::KafkaCommitFailures); - LOG_INFO(log, "All commit attempts failed"); + LOG_ERROR(log, "All commit attempts failed"); } else { diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index dc8d0f8a7df..42f7419def3 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -319,34 +319,6 @@ void StorageKafka2::assertActive() const throw Exception(ErrorCodes::LOGICAL_ERROR, "Table is not active (replica path: {})", replica_path); } -SettingsChanges StorageKafka2::createSettingsAdjustments() -{ - SettingsChanges result; - // Needed for backward compatibility - if (!kafka_settings->input_format_skip_unknown_fields.changed) - { - // Always skip unknown fields regardless of the context (JSON or TSKV) - kafka_settings->input_format_skip_unknown_fields = true; - } - - if (!kafka_settings->input_format_allow_errors_ratio.changed) - kafka_settings->input_format_allow_errors_ratio = 0.; - - if (!kafka_settings->input_format_allow_errors_num.changed) - kafka_settings->input_format_allow_errors_num = kafka_settings->kafka_skip_broken_messages.value; - - if (!schema_name.empty()) - result.emplace_back("format_schema", schema_name); - - for (const auto & setting : *kafka_settings) - { - const auto & name = setting.getName(); - if (name.find("kafka_") == std::string::npos) - result.emplace_back(name, setting.getValue()); - } - return result; -} - Pipe StorageKafka2::read( const Names & /*column_names */, diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp index 883eae95a7f..801c3a18a39 100644 --- a/src/Storages/Kafka/StorageKafkaCommon.cpp +++ b/src/Storages/Kafka/StorageKafkaCommon.cpp @@ -114,7 +114,7 @@ StorageKafkaInterceptors::rdKafkaOnThreadExit(rd_kafka_t *, rd_ka self->thread_statuses.erase(it); - return RD_KAFKA_RESP_ERR_NO_ERROR; + return RD_KAFKA_RESP_ERR_NO_ERROR; } template From 3c20297ae1aaad3c5c5e4bac18c898243d05fece Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 31 Jul 2024 18:07:33 +0000 Subject: [PATCH 246/644] Make sure consumers are created even when initial subscribe fails --- src/Storages/Kafka/KafkaConsumer2.cpp | 12 ++++++++++-- src/Storages/Kafka/KafkaConsumer2.h | 3 +++ src/Storages/Kafka/StorageKafka2.cpp | 17 ++++++++--------- src/Storages/Kafka/StorageKafka2.h | 1 - 4 files changed, 21 insertions(+), 12 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 761186fbbdd..a94afd2c8da 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -122,8 +122,6 @@ KafkaConsumer2::KafkaConsumer2( LOG_ERROR(log, "Rebalance error: {}", err); ProfileEvents::increment(ProfileEvents::KafkaRebalanceErrors); }); - - consumer->subscribe(topics); } KafkaConsumer2::~KafkaConsumer2() @@ -342,6 +340,16 @@ void KafkaConsumer2::commit(const TopicPartition & topic_partition) } } +void KafkaConsumer2::subscribeIfNotSubscribedYet() +{ + if (likely(is_subscribed)) + return; + + consumer->subscribe(topics); + is_subscribed = true; + LOG_DEBUG(log, "Subscribed."); +} + ReadBufferPtr KafkaConsumer2::getNextMessage() { if (current != messages.end()) diff --git a/src/Storages/Kafka/KafkaConsumer2.h b/src/Storages/Kafka/KafkaConsumer2.h index 3c91df8a02f..dd2cfe87aa0 100644 --- a/src/Storages/Kafka/KafkaConsumer2.h +++ b/src/Storages/Kafka/KafkaConsumer2.h @@ -115,6 +115,8 @@ public: const auto & currentHeaderList() const { return current[-1].get_header_list(); } String currentPayload() const { return current[-1].get_payload(); } + void subscribeIfNotSubscribedYet(); + private: using Messages = std::vector; CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaConsumers}; @@ -136,6 +138,7 @@ private: StalledStatus stalled_status = StalledStatus::NO_MESSAGES_RETURNED; const std::atomic & stopped; + bool is_subscribed = false; // order is important, need to be destructed before consumer Messages messages; diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 42f7419def3..e70be7d7a2a 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -372,6 +372,8 @@ void StorageKafka2::startup() consumers.push_back(ConsumerAndAssignmentInfo{.consumer = createConsumer(i), .keeper = getZooKeeper()}); LOG_DEBUG(log, "Created #{} consumer", num_created_consumers); ++num_created_consumers; + + consumers.back().consumer->subscribeIfNotSubscribedYet(); } catch (const cppkafka::Exception &) { @@ -404,16 +406,11 @@ KafkaConsumer2Ptr StorageKafka2::createConsumer(size_t consumer_number) consumer_impl->set_destroy_flags(RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE); /// NOTE: we pass |stream_cancelled| by reference here, so the buffers should not outlive the storage. - if (thread_per_consumer) - { - // call subscribe; - auto & stream_cancelled = tasks[consumer_number]->stream_cancelled; - return std::make_shared( - consumer_impl, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), stream_cancelled, topics); - } - + chassert((thread_per_consumer || num_consumers == 1) && "StorageKafka2 cannot handle multiple consumers on a single thread"); + auto & stream_cancelled = tasks[consumer_number]->stream_cancelled; return std::make_shared( - consumer_impl, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), tasks.back()->stream_cancelled, topics); + consumer_impl, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), stream_cancelled, topics); + } @@ -1067,6 +1064,8 @@ std::optional StorageKafka2::streamToViews(size_t id auto & consumer_info = consumers[idx]; consumer_info.watch.restart(); auto & consumer = consumer_info.consumer; + // In case the initial subscribe in startup failed, let's subscribe now + consumer->subscribeIfNotSubscribedYet(); // To keep the consumer alive const auto wait_for_assignment = consumer_info.locks.empty(); diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index 318c04f1f91..e2565100879 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -181,7 +181,6 @@ private: void partialShutdown(); void assertActive() const; - SettingsChanges createSettingsAdjustments(); KafkaConsumer2Ptr createConsumer(size_t consumer_number); // Returns full consumer related configuration, also the configuration // contains global kafka properties. From be65f7be5133d16c162b679cbacad94e582b1493 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 31 Jul 2024 18:38:16 +0000 Subject: [PATCH 247/644] Fix build --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 06fb47779d7..992b1220201 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -58,7 +58,7 @@ String ClickHouseVersion::toString() const static std::initializer_list> settings_changes_history_initializer = { {"24.8", {{"allow_experimental_kafka_offsets_storage_in_keeper", false, false, "Allow the usage of experimental Kafka storage engine that stores the committed offsets in ClickHouse Keeper"}, - }} + }}, {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, From 557f9dbe3fb02e3bce62adbeb1fd5056f2d36b6c Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 31 Jul 2024 18:51:27 +0000 Subject: [PATCH 248/644] fix test --- .../0_stateless/02319_lightweight_delete_on_merge_tree.sql | 2 +- tests/queries/0_stateless/02792_drop_projection_lwd.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql index f82f79dbe44..6491253cd5f 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql @@ -102,7 +102,7 @@ ALTER TABLE t_proj ADD PROJECTION p_1 (SELECT avg(a), avg(b), count()) SETTINGS INSERT INTO t_proj SELECT number + 1, number + 1 FROM numbers(1000); -DELETE FROM t_proj WHERE a < 100; -- { serverError NOT_IMPLEMENTED } +DELETE FROM t_proj WHERE a < 100; -- { serverError SUPPORT_IS_DISABLED } SELECT avg(a), avg(b), count() FROM t_proj; diff --git a/tests/queries/0_stateless/02792_drop_projection_lwd.sql b/tests/queries/0_stateless/02792_drop_projection_lwd.sql index dcde7dcc600..dad7f7cd028 100644 --- a/tests/queries/0_stateless/02792_drop_projection_lwd.sql +++ b/tests/queries/0_stateless/02792_drop_projection_lwd.sql @@ -7,7 +7,7 @@ CREATE TABLE t_projections_lwd (a UInt32, b UInt32, PROJECTION p (SELECT * ORDER INSERT INTO t_projections_lwd SELECT number, number FROM numbers(100); -- LWD does not work, as expected -DELETE FROM t_projections_lwd WHERE a = 1; -- { serverError NOT_IMPLEMENTED } +DELETE FROM t_projections_lwd WHERE a = 1; -- { serverError SUPPORT_IS_DISABLED } KILL MUTATION WHERE database = currentDatabase() AND table = 't_projections_lwd' SYNC FORMAT Null; -- drop projection From 91e48d8b1b1029af937433eca377b9187b03d49a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 31 Jul 2024 20:57:57 +0000 Subject: [PATCH 249/644] Split `StorageKafkaCommon` --- src/Storages/Kafka/KafkaConfigLoader.cpp | 475 ++++++++++++ src/Storages/Kafka/KafkaConfigLoader.h | 54 ++ src/Storages/Kafka/KafkaConsumer.cpp | 2 +- src/Storages/Kafka/KafkaConsumer2.cpp | 2 +- src/Storages/Kafka/StorageKafka.cpp | 3 +- src/Storages/Kafka/StorageKafka.h | 6 +- src/Storages/Kafka/StorageKafka2.cpp | 3 +- src/Storages/Kafka/StorageKafka2.h | 6 +- src/Storages/Kafka/StorageKafkaCommon.cpp | 875 ---------------------- src/Storages/Kafka/StorageKafkaCommon.h | 116 --- src/Storages/Kafka/StorageKafkaUtils.cpp | 458 +++++++++++ src/Storages/Kafka/StorageKafkaUtils.h | 61 ++ 12 files changed, 1060 insertions(+), 1001 deletions(-) create mode 100644 src/Storages/Kafka/KafkaConfigLoader.cpp create mode 100644 src/Storages/Kafka/KafkaConfigLoader.h delete mode 100644 src/Storages/Kafka/StorageKafkaCommon.cpp delete mode 100644 src/Storages/Kafka/StorageKafkaCommon.h create mode 100644 src/Storages/Kafka/StorageKafkaUtils.cpp create mode 100644 src/Storages/Kafka/StorageKafkaUtils.h diff --git a/src/Storages/Kafka/KafkaConfigLoader.cpp b/src/Storages/Kafka/KafkaConfigLoader.cpp new file mode 100644 index 00000000000..3d31a987395 --- /dev/null +++ b/src/Storages/Kafka/KafkaConfigLoader.cpp @@ -0,0 +1,475 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace CurrentMetrics +{ +extern const Metric KafkaLibrdkafkaThreads; +} + +namespace DB +{ + +template +struct KafkaInterceptors +{ + static rd_kafka_resp_err_t rdKafkaOnThreadStart(rd_kafka_t *, rd_kafka_thread_type_t thread_type, const char *, void * ctx); + + static rd_kafka_resp_err_t rdKafkaOnThreadExit(rd_kafka_t *, rd_kafka_thread_type_t, const char *, void * ctx); + + static rd_kafka_resp_err_t + rdKafkaOnNew(rd_kafka_t * rk, const rd_kafka_conf_t *, void * ctx, char * /*errstr*/, size_t /*errstr_size*/); + + static rd_kafka_resp_err_t rdKafkaOnConfDup( + rd_kafka_conf_t * new_conf, const rd_kafka_conf_t * /*old_conf*/, size_t /*filter_cnt*/, const char ** /*filter*/, void * ctx); +}; + +template +rd_kafka_resp_err_t +KafkaInterceptors::rdKafkaOnThreadStart(rd_kafka_t *, rd_kafka_thread_type_t thread_type, const char *, void * ctx) +{ + TStorageKafka * self = reinterpret_cast(ctx); + CurrentMetrics::add(CurrentMetrics::KafkaLibrdkafkaThreads, 1); + + const auto & storage_id = self->getStorageID(); + const auto & table = storage_id.getTableName(); + + switch (thread_type) + { + case RD_KAFKA_THREAD_MAIN: + setThreadName(("rdk:m/" + table.substr(0, 9)).c_str()); + break; + case RD_KAFKA_THREAD_BACKGROUND: + setThreadName(("rdk:bg/" + table.substr(0, 8)).c_str()); + break; + case RD_KAFKA_THREAD_BROKER: + setThreadName(("rdk:b/" + table.substr(0, 9)).c_str()); + break; + } + + /// Create ThreadStatus to track memory allocations from librdkafka threads. + // + /// And store them in a separate list (thread_statuses) to make sure that they will be destroyed, + /// regardless how librdkafka calls the hooks. + /// But this can trigger use-after-free if librdkafka will not destroy threads after rd_kafka_wait_destroyed() + auto thread_status = std::make_shared(); + std::lock_guard lock(self->thread_statuses_mutex); + self->thread_statuses.emplace_back(std::move(thread_status)); + + return RD_KAFKA_RESP_ERR_NO_ERROR; +} + +template +rd_kafka_resp_err_t KafkaInterceptors::rdKafkaOnThreadExit(rd_kafka_t *, rd_kafka_thread_type_t, const char *, void * ctx) +{ + TStorageKafka * self = reinterpret_cast(ctx); + CurrentMetrics::sub(CurrentMetrics::KafkaLibrdkafkaThreads, 1); + + std::lock_guard lock(self->thread_statuses_mutex); + const auto it = std::find_if( + self->thread_statuses.begin(), + self->thread_statuses.end(), + [](const auto & thread_status_ptr) { return thread_status_ptr.get() == current_thread; }); + if (it == self->thread_statuses.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread status for this librdkafka thread."); + + self->thread_statuses.erase(it); + + return RD_KAFKA_RESP_ERR_NO_ERROR; +} + +template +rd_kafka_resp_err_t KafkaInterceptors::rdKafkaOnNew( + rd_kafka_t * rk, const rd_kafka_conf_t *, void * ctx, char * /*errstr*/, size_t /*errstr_size*/) +{ + TStorageKafka * self = reinterpret_cast(ctx); + rd_kafka_resp_err_t status; + + status = rd_kafka_interceptor_add_on_thread_start(rk, "init-thread", rdKafkaOnThreadStart, ctx); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + { + LOG_ERROR(self->log, "Cannot set on thread start interceptor due to {} error", status); + return status; + } + + status = rd_kafka_interceptor_add_on_thread_exit(rk, "exit-thread", rdKafkaOnThreadExit, ctx); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(self->log, "Cannot set on thread exit interceptor due to {} error", status); + + return status; +} + +template +rd_kafka_resp_err_t KafkaInterceptors::rdKafkaOnConfDup( + rd_kafka_conf_t * new_conf, const rd_kafka_conf_t * /*old_conf*/, size_t /*filter_cnt*/, const char ** /*filter*/, void * ctx) +{ + TStorageKafka * self = reinterpret_cast(ctx); + rd_kafka_resp_err_t status; + + // cppkafka copies configuration multiple times + status = rd_kafka_conf_interceptor_add_on_conf_dup(new_conf, "init", rdKafkaOnConfDup, ctx); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + { + LOG_ERROR(self->log, "Cannot set on conf dup interceptor due to {} error", status); + return status; + } + + status = rd_kafka_conf_interceptor_add_on_new(new_conf, "init", rdKafkaOnNew, ctx); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(self->log, "Cannot set on conf new interceptor due to {} error", status); + + return status; +} + +template struct KafkaInterceptors; +template struct KafkaInterceptors; + +namespace +{ + +void setKafkaConfigValue(cppkafka::Configuration & kafka_config, const String & key, const String & value) +{ + /// "log_level" has valid underscore, the remaining librdkafka setting use dot.separated.format which isn't acceptable for XML. + /// See https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md + const String setting_name_in_kafka_config = (key == "log_level") ? key : boost::replace_all_copy(key, "_", "."); + kafka_config.set(setting_name_in_kafka_config, value); +} + +void loadConfigProperty( + cppkafka::Configuration & kafka_config, + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + const String & tag) +{ + const String property_path = config_prefix + "." + tag; + const String property_value = config.getString(property_path); + + setKafkaConfigValue(kafka_config, tag, property_value); +} + +void loadNamedCollectionConfig(cppkafka::Configuration & kafka_config, const String & collection_name, const String & config_prefix) +{ + const auto & collection = NamedCollectionFactory::instance().get(collection_name); + for (const auto & key : collection->getKeys(-1, config_prefix)) + { + // Cut prefix with '.' before actual config tag. + const auto param_name = key.substr(config_prefix.size() + 1); + setKafkaConfigValue(kafka_config, param_name, collection->get(key)); + } +} + +void loadLegacyTopicConfig( + cppkafka::Configuration & kafka_config, + const Poco::Util::AbstractConfiguration & config, + const String & collection_name, + const String & config_prefix) +{ + if (!collection_name.empty()) + { + loadNamedCollectionConfig(kafka_config, collection_name, config_prefix); + return; + } + + Poco::Util::AbstractConfiguration::Keys tags; + config.keys(config_prefix, tags); + + for (const auto & tag : tags) + { + loadConfigProperty(kafka_config, config, config_prefix, tag); + } +} + +/// Read server configuration into cppkafa configuration, used by new per-topic configuration +void loadTopicConfig( + cppkafka::Configuration & kafka_config, + const Poco::Util::AbstractConfiguration & config, + const String & collection_name, + const String & config_prefix, + const String & topic) +{ + if (!collection_name.empty()) + { + const auto topic_prefix = fmt::format("{}.{}", config_prefix, KafkaConfigLoader::CONFIG_KAFKA_TOPIC_TAG); + const auto & collection = NamedCollectionFactory::instance().get(collection_name); + for (const auto & key : collection->getKeys(1, config_prefix)) + { + /// Only consider key . Multiple occurrences given as "kafka_topic", "kafka_topic[1]", etc. + if (!key.starts_with(topic_prefix)) + continue; + + const String kafka_topic_path = config_prefix + "." + key; + const String kafka_topic_name_path = kafka_topic_path + "." + KafkaConfigLoader::CONFIG_NAME_TAG; + if (topic == collection->get(kafka_topic_name_path)) + /// Found it! Now read the per-topic configuration into cppkafka. + loadNamedCollectionConfig(kafka_config, collection_name, kafka_topic_path); + } + } + else + { + /// Read all tags one level below + Poco::Util::AbstractConfiguration::Keys tags; + config.keys(config_prefix, tags); + + for (const auto & tag : tags) + { + if (tag == KafkaConfigLoader::CONFIG_NAME_TAG) + continue; // ignore , it is used to match topic configurations + loadConfigProperty(kafka_config, config, config_prefix, tag); + } + } +} + +/// Read server configuration into cppkafka configuration, used by global configuration and by legacy per-topic configuration +void loadFromConfig( + cppkafka::Configuration & kafka_config, const KafkaConfigLoader::LoadConfigParams & params, const String & config_prefix) +{ + if (!params.collection_name.empty()) + { + loadNamedCollectionConfig(kafka_config, params.collection_name, config_prefix); + return; + } + + /// Read all tags one level below + Poco::Util::AbstractConfiguration::Keys tags; + params.config.keys(config_prefix, tags); + + for (const auto & tag : tags) + { + if (tag == KafkaConfigLoader::CONFIG_KAFKA_PRODUCER_TAG || tag == KafkaConfigLoader::CONFIG_KAFKA_CONSUMER_TAG) + /// Do not load consumer/producer properties, since they should be separated by different configuration objects. + continue; + + if (tag.starts_with( + KafkaConfigLoader::CONFIG_KAFKA_TOPIC_TAG)) /// multiple occurrences given as "kafka_topic", "kafka_topic[1]", etc. + { + // Update consumer topic-specific configuration (new syntax). Example with topics "football" and "baseball": + // + // + // football + // 250 + // 5000 + // + // + // baseball + // 300 + // 2000 + // + // + // Advantages: The period restriction no longer applies (e.g. sports.football will work), everything + // Kafka-related is below . + for (const auto & topic : params.topics) + { + /// Read topic name between ... + const String kafka_topic_path = config_prefix + "." + tag; + const String kafka_topic_name_path = kafka_topic_path + "." + KafkaConfigLoader::CONFIG_NAME_TAG; + const String topic_name = params.config.getString(kafka_topic_name_path); + + if (topic_name != topic) + continue; + loadTopicConfig(kafka_config, params.config, params.collection_name, kafka_topic_path, topic); + } + continue; + } + if (tag.starts_with(KafkaConfigLoader::CONFIG_KAFKA_TAG)) + /// skip legacy configuration per topic e.g. . + /// it will be processed is a separate function + continue; + // Update configuration from the configuration. Example: + // + // 250 + // 100000 + // + loadConfigProperty(kafka_config, params.config, config_prefix, tag); + } +} + +void loadLegacyConfigSyntax( + cppkafka::Configuration & kafka_config, + const Poco::Util::AbstractConfiguration & config, + const String & collection_name, + const Names & topics) +{ + for (const auto & topic : topics) + { + const String kafka_topic_path = KafkaConfigLoader::CONFIG_KAFKA_TAG + "." + KafkaConfigLoader::CONFIG_KAFKA_TAG + "_" + topic; + loadLegacyTopicConfig(kafka_config, config, collection_name, kafka_topic_path); + } +} + +void loadConsumerConfig(cppkafka::Configuration & kafka_config, const KafkaConfigLoader::LoadConfigParams & params) +{ + const String consumer_path = KafkaConfigLoader::CONFIG_KAFKA_TAG + "." + KafkaConfigLoader::CONFIG_KAFKA_CONSUMER_TAG; + loadLegacyConfigSyntax(kafka_config, params.config, params.collection_name, params.topics); + // A new syntax has higher priority + loadFromConfig(kafka_config, params, consumer_path); +} + +void loadProducerConfig(cppkafka::Configuration & kafka_config, const KafkaConfigLoader::LoadConfigParams & params) +{ + const String producer_path = KafkaConfigLoader::CONFIG_KAFKA_TAG + "." + KafkaConfigLoader::CONFIG_KAFKA_PRODUCER_TAG; + loadLegacyConfigSyntax(kafka_config, params.config, params.collection_name, params.topics); + // A new syntax has higher priority + loadFromConfig(kafka_config, params, producer_path); +} + +template +void updateGlobalConfiguration( + cppkafka::Configuration & kafka_config, TKafkaStorage & storage, const KafkaConfigLoader::LoadConfigParams & params) +{ + loadFromConfig(kafka_config, params, KafkaConfigLoader::CONFIG_KAFKA_TAG); + +#if USE_KRB5 + if (kafka_config.has_property("sasl.kerberos.kinit.cmd")) + LOG_WARNING(params.log, "sasl.kerberos.kinit.cmd configuration parameter is ignored."); + + kafka_config.set("sasl.kerberos.kinit.cmd", ""); + kafka_config.set("sasl.kerberos.min.time.before.relogin", "0"); + + if (kafka_config.has_property("sasl.kerberos.keytab") && kafka_config.has_property("sasl.kerberos.principal")) + { + String keytab = kafka_config.get("sasl.kerberos.keytab"); + String principal = kafka_config.get("sasl.kerberos.principal"); + LOG_DEBUG(params.log, "Running KerberosInit"); + try + { + kerberosInit(keytab, principal); + } + catch (const Exception & e) + { + LOG_ERROR(params.log, "KerberosInit failure: {}", getExceptionMessage(e, false)); + } + LOG_DEBUG(params.log, "Finished KerberosInit"); + } +#else // USE_KRB5 + if (kafka_config.has_property("sasl.kerberos.keytab") || kafka_config.has_property("sasl.kerberos.principal")) + LOG_WARNING(log, "Ignoring Kerberos-related parameters because ClickHouse was built without krb5 library support."); +#endif // USE_KRB5 + // No need to add any prefix, messages can be distinguished + kafka_config.set_log_callback( + [log = params.log](cppkafka::KafkaHandleBase & handle, int level, const std::string & facility, const std::string & message) + { + auto [poco_level, client_logs_level] = parseSyslogLevel(level); + const auto & kafka_object_config = handle.get_configuration(); + const std::string client_id_key{"client.id"}; + chassert(kafka_object_config.has_property(client_id_key) && "Kafka configuration doesn't have expected client.id set"); + LOG_IMPL( + log, + client_logs_level, + poco_level, + "[client.id:{}] [rdk:{}] {}", + kafka_object_config.get(client_id_key), + facility, + message); + }); + + /// NOTE: statistics should be consumed, otherwise it creates too much + /// entries in the queue, that leads to memory leak and slow shutdown. + if (!kafka_config.has_property("statistics.interval.ms")) + { + // every 3 seconds by default. set to 0 to disable. + kafka_config.set("statistics.interval.ms", "3000"); + } + // Configure interceptor to change thread name + // + // TODO: add interceptors support into the cppkafka. + // XXX: rdkafka uses pthread_set_name_np(), but glibc-compatibility overrides it to noop. + { + // This should be safe, since we wait the rdkafka object anyway. + void * self = static_cast(&storage); + + int status; + + status + = rd_kafka_conf_interceptor_add_on_new(kafka_config.get_handle(), "init", KafkaInterceptors::rdKafkaOnNew, self); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(params.log, "Cannot set new interceptor due to {} error", status); + + // cppkafka always copy the configuration + status = rd_kafka_conf_interceptor_add_on_conf_dup( + kafka_config.get_handle(), "init", KafkaInterceptors::rdKafkaOnConfDup, self); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(params.log, "Cannot set dup conf interceptor due to {} error", status); + } +} + +} + +template +cppkafka::Configuration KafkaConfigLoader::getConsumerConfiguration(TKafkaStorage & storage, const ConsumerConfigParams & params) +{ + cppkafka::Configuration conf; + + conf.set("metadata.broker.list", params.brokers); + conf.set("group.id", params.group); + if (params.multiple_consumers) + conf.set("client.id", fmt::format("{}-{}", params.client_id, params.consumer_number)); + else + conf.set("client.id", params.client_id); + conf.set("client.software.name", VERSION_NAME); + conf.set("client.software.version", VERSION_DESCRIBE); + conf.set("auto.offset.reset", "earliest"); // If no offset stored for this group, read all messages from the start + + // that allows to prevent fast draining of the librdkafka queue + // during building of single insert block. Improves performance + // significantly, but may lead to bigger memory consumption. + size_t default_queued_min_messages = 100000; // must be greater than or equal to default + size_t max_allowed_queued_min_messages = 10000000; // must be less than or equal to max allowed value + conf.set( + "queued.min.messages", std::min(std::max(params.max_block_size, default_queued_min_messages), max_allowed_queued_min_messages)); + + updateGlobalConfiguration(conf, storage, params); + loadConsumerConfig(conf, params); + + // those settings should not be changed by users. + conf.set("enable.auto.commit", "false"); // We manually commit offsets after a stream successfully finished + conf.set("enable.auto.offset.store", "false"); // Update offset automatically - to commit them all at once. + conf.set("enable.partition.eof", "false"); // Ignore EOF messages + + for (auto & property : conf.get_all()) + { + LOG_TRACE(params.log, "Consumer set property {}:{}", property.first, property.second); + } + + return conf; +} + +template cppkafka::Configuration +KafkaConfigLoader::getConsumerConfiguration(StorageKafka & storage, const ConsumerConfigParams & params); +template cppkafka::Configuration +KafkaConfigLoader::getConsumerConfiguration(StorageKafka2 & storage, const ConsumerConfigParams & params); + +template +cppkafka::Configuration KafkaConfigLoader::getProducerConfiguration(TKafkaStorage & storage, const ProducerConfigParams & params) +{ + cppkafka::Configuration conf; + conf.set("metadata.broker.list", params.brokers); + conf.set("client.id", params.client_id); + conf.set("client.software.name", VERSION_NAME); + conf.set("client.software.version", VERSION_DESCRIBE); + + updateGlobalConfiguration(conf, storage, params); + loadProducerConfig(conf, params); + + for (auto & property : conf.get_all()) + { + LOG_TRACE(params.log, "Producer set property {}:{}", property.first, property.second); + } + + return conf; +} + +template cppkafka::Configuration +KafkaConfigLoader::getProducerConfiguration(StorageKafka & storage, const ProducerConfigParams & params); +template cppkafka::Configuration +KafkaConfigLoader::getProducerConfiguration(StorageKafka2 & storage, const ProducerConfigParams & params); + +} diff --git a/src/Storages/Kafka/KafkaConfigLoader.h b/src/Storages/Kafka/KafkaConfigLoader.h new file mode 100644 index 00000000000..f18683c17f0 --- /dev/null +++ b/src/Storages/Kafka/KafkaConfigLoader.h @@ -0,0 +1,54 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ +struct KafkaSettings; +class VirtualColumnsDescription; + +struct KafkaConfigLoader +{ + static inline const String CONFIG_KAFKA_TAG = "kafka"; + static inline const String CONFIG_KAFKA_TOPIC_TAG = "kafka_topic"; + static inline const String CONFIG_NAME_TAG = "name"; + static inline const String CONFIG_KAFKA_CONSUMER_TAG = "consumer"; + static inline const String CONFIG_KAFKA_PRODUCER_TAG = "producer"; + using LogCallback = cppkafka::Configuration::LogCallback; + + + struct LoadConfigParams + { + const Poco::Util::AbstractConfiguration & config; + String & collection_name; + const Names & topics; + LoggerPtr & log; + }; + + struct ConsumerConfigParams : public LoadConfigParams + { + String brokers; + String group; + bool multiple_consumers; + size_t consumer_number; + String client_id; + size_t max_block_size; + }; + + struct ProducerConfigParams : public LoadConfigParams + { + String brokers; + String client_id; + }; + + template + static cppkafka::Configuration getConsumerConfiguration(TKafkaStorage & storage, const ConsumerConfigParams & params); + + template + static cppkafka::Configuration getProducerConfiguration(TKafkaStorage & storage, const ProducerConfigParams & params); +}; +} diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index 1affbbaf8fd..d9256cf39ce 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index a94afd2c8da..8581398aa90 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 0902ef838b4..f4f641d1c68 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -14,13 +14,14 @@ #include #include #include +#include #include #include #include #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index d02d86b468c..966d818d675 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -24,7 +24,7 @@ class StorageSystemKafkaConsumers; class ThreadStatus; template -struct StorageKafkaInterceptors; +struct KafkaInterceptors; using KafkaConsumerPtr = std::shared_ptr; using ConsumerPtr = std::shared_ptr; @@ -34,8 +34,8 @@ using ConsumerPtr = std::shared_ptr; */ class StorageKafka final : public IStorage, WithContext { - using StorageKafkaInterceptors = StorageKafkaInterceptors; - friend StorageKafkaInterceptors; + using KafkaInterceptors = KafkaInterceptors; + friend KafkaInterceptors; public: StorageKafka( diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index e70be7d7a2a..f58d629dd9b 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -18,10 +18,11 @@ #include #include #include +#include #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h index e2565100879..f85fedb316a 100644 --- a/src/Storages/Kafka/StorageKafka2.h +++ b/src/Storages/Kafka/StorageKafka2.h @@ -30,7 +30,7 @@ namespace DB { template -struct StorageKafkaInterceptors; +struct KafkaInterceptors; using KafkaConsumer2Ptr = std::shared_ptr; @@ -51,8 +51,8 @@ using KafkaConsumer2Ptr = std::shared_ptr; /// hashes for deduplication. class StorageKafka2 final : public IStorage, WithContext { - using StorageKafkaInterceptors = StorageKafkaInterceptors; - friend StorageKafkaInterceptors; + using KafkaInterceptors = KafkaInterceptors; + friend KafkaInterceptors; public: StorageKafka2( diff --git a/src/Storages/Kafka/StorageKafkaCommon.cpp b/src/Storages/Kafka/StorageKafkaCommon.cpp deleted file mode 100644 index 801c3a18a39..00000000000 --- a/src/Storages/Kafka/StorageKafkaCommon.cpp +++ /dev/null @@ -1,875 +0,0 @@ -#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 - -#if USE_KRB5 -# include -#endif // USE_KRB5 - -namespace CurrentMetrics -{ -extern const Metric KafkaLibrdkafkaThreads; -} - -namespace ProfileEvents -{ -extern const Event KafkaConsumerErrors; -} - -namespace DB -{ - -using namespace std::chrono_literals; - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int SUPPORT_IS_DISABLED; -} - -template -rd_kafka_resp_err_t -StorageKafkaInterceptors::rdKafkaOnThreadStart(rd_kafka_t *, rd_kafka_thread_type_t thread_type, const char *, void * ctx) -{ - TStorageKafka * self = reinterpret_cast(ctx); - CurrentMetrics::add(CurrentMetrics::KafkaLibrdkafkaThreads, 1); - - const auto & storage_id = self->getStorageID(); - const auto & table = storage_id.getTableName(); - - switch (thread_type) - { - case RD_KAFKA_THREAD_MAIN: - setThreadName(("rdk:m/" + table.substr(0, 9)).c_str()); - break; - case RD_KAFKA_THREAD_BACKGROUND: - setThreadName(("rdk:bg/" + table.substr(0, 8)).c_str()); - break; - case RD_KAFKA_THREAD_BROKER: - setThreadName(("rdk:b/" + table.substr(0, 9)).c_str()); - break; - } - - /// Create ThreadStatus to track memory allocations from librdkafka threads. - // - /// And store them in a separate list (thread_statuses) to make sure that they will be destroyed, - /// regardless how librdkafka calls the hooks. - /// But this can trigger use-after-free if librdkafka will not destroy threads after rd_kafka_wait_destroyed() - auto thread_status = std::make_shared(); - std::lock_guard lock(self->thread_statuses_mutex); - self->thread_statuses.emplace_back(std::move(thread_status)); - - return RD_KAFKA_RESP_ERR_NO_ERROR; -} - -template -rd_kafka_resp_err_t -StorageKafkaInterceptors::rdKafkaOnThreadExit(rd_kafka_t *, rd_kafka_thread_type_t, const char *, void * ctx) -{ - TStorageKafka * self = reinterpret_cast(ctx); - CurrentMetrics::sub(CurrentMetrics::KafkaLibrdkafkaThreads, 1); - - std::lock_guard lock(self->thread_statuses_mutex); - const auto it = std::find_if(self->thread_statuses.begin(), self->thread_statuses.end(), [](const auto & thread_status_ptr) - { - return thread_status_ptr.get() == current_thread; - }); - if (it == self->thread_statuses.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread status for this librdkafka thread."); - - self->thread_statuses.erase(it); - - return RD_KAFKA_RESP_ERR_NO_ERROR; -} - -template -rd_kafka_resp_err_t StorageKafkaInterceptors::rdKafkaOnNew( - rd_kafka_t * rk, const rd_kafka_conf_t *, void * ctx, char * /*errstr*/, size_t /*errstr_size*/) -{ - TStorageKafka * self = reinterpret_cast(ctx); - rd_kafka_resp_err_t status; - - status = rd_kafka_interceptor_add_on_thread_start(rk, "init-thread", rdKafkaOnThreadStart, ctx); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - { - LOG_ERROR(self->log, "Cannot set on thread start interceptor due to {} error", status); - return status; - } - - status = rd_kafka_interceptor_add_on_thread_exit(rk, "exit-thread", rdKafkaOnThreadExit, ctx); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - LOG_ERROR(self->log, "Cannot set on thread exit interceptor due to {} error", status); - - return status; -} - -template -rd_kafka_resp_err_t StorageKafkaInterceptors::rdKafkaOnConfDup( - rd_kafka_conf_t * new_conf, const rd_kafka_conf_t * /*old_conf*/, size_t /*filter_cnt*/, const char ** /*filter*/, void * ctx) -{ - TStorageKafka * self = reinterpret_cast(ctx); - rd_kafka_resp_err_t status; - - // cppkafka copies configuration multiple times - status = rd_kafka_conf_interceptor_add_on_conf_dup(new_conf, "init", rdKafkaOnConfDup, ctx); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - { - LOG_ERROR(self->log, "Cannot set on conf dup interceptor due to {} error", status); - return status; - } - - status = rd_kafka_conf_interceptor_add_on_new(new_conf, "init", rdKafkaOnNew, ctx); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - LOG_ERROR(self->log, "Cannot set on conf new interceptor due to {} error", status); - - return status; -} - -void setKafkaConfigValue(cppkafka::Configuration & kafka_config, const String & key, const String & value) -{ - /// "log_level" has valid underscore, the remaining librdkafka setting use dot.separated.format which isn't acceptable for XML. - /// See https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md - const String setting_name_in_kafka_config = (key == "log_level") ? key : boost::replace_all_copy(key, "_", "."); - kafka_config.set(setting_name_in_kafka_config, value); -} - -void loadConfigProperty(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const String & tag) -{ - const String property_path = config_prefix + "." + tag; - const String property_value = config.getString(property_path); - - setKafkaConfigValue(kafka_config, tag, property_value); -} - -void loadNamedCollectionConfig(cppkafka::Configuration & kafka_config, const String & collection_name, const String & config_prefix) -{ - const auto & collection = NamedCollectionFactory::instance().get(collection_name); - for (const auto & key : collection->getKeys(-1, config_prefix)) - { - // Cut prefix with '.' before actual config tag. - const auto param_name = key.substr(config_prefix.size() + 1); - setKafkaConfigValue(kafka_config, param_name, collection->get(key)); - } -} - -void loadLegacyTopicConfig(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & collection_name, const String & config_prefix) -{ - if (!collection_name.empty()) - { - loadNamedCollectionConfig(kafka_config, collection_name, config_prefix); - return; - } - - Poco::Util::AbstractConfiguration::Keys tags; - config.keys(config_prefix, tags); - - for (const auto & tag : tags) - { - loadConfigProperty(kafka_config, config, config_prefix, tag); - } -} - -/// Read server configuration into cppkafa configuration, used by new per-topic configuration -void loadTopicConfig(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & collection_name, const String & config_prefix, const String & topic) -{ - if (!collection_name.empty()) - { - const auto topic_prefix = fmt::format("{}.{}", config_prefix, KafkaConfigLoader::CONFIG_KAFKA_TOPIC_TAG); - const auto & collection = NamedCollectionFactory::instance().get(collection_name); - for (const auto & key : collection->getKeys(1, config_prefix)) - { - /// Only consider key . Multiple occurrences given as "kafka_topic", "kafka_topic[1]", etc. - if (!key.starts_with(topic_prefix)) - continue; - - const String kafka_topic_path = config_prefix + "." + key; - const String kafka_topic_name_path = kafka_topic_path + "." + KafkaConfigLoader::CONFIG_NAME_TAG; - if (topic == collection->get(kafka_topic_name_path)) - /// Found it! Now read the per-topic configuration into cppkafka. - loadNamedCollectionConfig(kafka_config, collection_name, kafka_topic_path); - } - } - else - { - /// Read all tags one level below - Poco::Util::AbstractConfiguration::Keys tags; - config.keys(config_prefix, tags); - - for (const auto & tag : tags) - { - if (tag == KafkaConfigLoader::CONFIG_NAME_TAG) - continue; // ignore , it is used to match topic configurations - loadConfigProperty(kafka_config, config, config_prefix, tag); - } - } -} - -/// Read server configuration into cppkafka configuration, used by global configuration and by legacy per-topic configuration -static void -loadFromConfig(cppkafka::Configuration & kafka_config, const KafkaConfigLoader::LoadConfigParams & params, const String & config_prefix) -{ - if (!params.collection_name.empty()) - { - loadNamedCollectionConfig(kafka_config, params.collection_name, config_prefix); - return; - } - - /// Read all tags one level below - Poco::Util::AbstractConfiguration::Keys tags; - params.config.keys(config_prefix, tags); - - for (const auto & tag : tags) - { - if (tag == KafkaConfigLoader::CONFIG_KAFKA_PRODUCER_TAG || tag == KafkaConfigLoader::CONFIG_KAFKA_CONSUMER_TAG) - /// Do not load consumer/producer properties, since they should be separated by different configuration objects. - continue; - - if (tag.starts_with(KafkaConfigLoader::CONFIG_KAFKA_TOPIC_TAG)) /// multiple occurrences given as "kafka_topic", "kafka_topic[1]", etc. - { - // Update consumer topic-specific configuration (new syntax). Example with topics "football" and "baseball": - // - // - // football - // 250 - // 5000 - // - // - // baseball - // 300 - // 2000 - // - // - // Advantages: The period restriction no longer applies (e.g. sports.football will work), everything - // Kafka-related is below . - for (const auto & topic : params.topics) - { - /// Read topic name between ... - const String kafka_topic_path = config_prefix + "." + tag; - const String kafka_topic_name_path = kafka_topic_path + "." + KafkaConfigLoader::CONFIG_NAME_TAG; - const String topic_name = params.config.getString(kafka_topic_name_path); - - if (topic_name != topic) - continue; - loadTopicConfig(kafka_config, params.config, params.collection_name, kafka_topic_path, topic); - } - continue; - } - if (tag.starts_with(KafkaConfigLoader::CONFIG_KAFKA_TAG)) - /// skip legacy configuration per topic e.g. . - /// it will be processed is a separate function - continue; - // Update configuration from the configuration. Example: - // - // 250 - // 100000 - // - loadConfigProperty(kafka_config, params.config, config_prefix, tag); - } -} - -void loadLegacyConfigSyntax( - cppkafka::Configuration & kafka_config, - const Poco::Util::AbstractConfiguration & config, - const String & collection_name, - const Names & topics) -{ - for (const auto & topic : topics) - { - const String kafka_topic_path = KafkaConfigLoader::CONFIG_KAFKA_TAG + "." + KafkaConfigLoader::CONFIG_KAFKA_TAG + "_" + topic; - loadLegacyTopicConfig(kafka_config, config, collection_name, kafka_topic_path); - } -} - -static void loadConsumerConfig(cppkafka::Configuration & kafka_config, const KafkaConfigLoader::LoadConfigParams & params) -{ - const String consumer_path = KafkaConfigLoader::CONFIG_KAFKA_TAG + "." + KafkaConfigLoader::CONFIG_KAFKA_CONSUMER_TAG; - loadLegacyConfigSyntax(kafka_config, params.config, params.collection_name, params.topics); - // A new syntax has higher priority - loadFromConfig(kafka_config, params, consumer_path); -} - -static void loadProducerConfig(cppkafka::Configuration & kafka_config, const KafkaConfigLoader::LoadConfigParams & params) -{ - const String producer_path = KafkaConfigLoader::CONFIG_KAFKA_TAG + "." + KafkaConfigLoader::CONFIG_KAFKA_PRODUCER_TAG; - loadLegacyConfigSyntax(kafka_config, params.config, params.collection_name, params.topics); - // A new syntax has higher priority - loadFromConfig(kafka_config, params, producer_path); -} - -template -static void updateGlobalConfiguration( - cppkafka::Configuration & kafka_config, TKafkaStorage & storage, const KafkaConfigLoader::LoadConfigParams & params) -{ - loadFromConfig(kafka_config, params, KafkaConfigLoader::CONFIG_KAFKA_TAG); - -#if USE_KRB5 - if (kafka_config.has_property("sasl.kerberos.kinit.cmd")) - LOG_WARNING(params.log, "sasl.kerberos.kinit.cmd configuration parameter is ignored."); - - kafka_config.set("sasl.kerberos.kinit.cmd", ""); - kafka_config.set("sasl.kerberos.min.time.before.relogin", "0"); - - if (kafka_config.has_property("sasl.kerberos.keytab") && kafka_config.has_property("sasl.kerberos.principal")) - { - String keytab = kafka_config.get("sasl.kerberos.keytab"); - String principal = kafka_config.get("sasl.kerberos.principal"); - LOG_DEBUG(params.log, "Running KerberosInit"); - try - { - kerberosInit(keytab, principal); - } - catch (const Exception & e) - { - LOG_ERROR(params.log, "KerberosInit failure: {}", getExceptionMessage(e, false)); - } - LOG_DEBUG(params.log, "Finished KerberosInit"); - } -#else // USE_KRB5 - if (kafka_config.has_property("sasl.kerberos.keytab") || kafka_config.has_property("sasl.kerberos.principal")) - LOG_WARNING(log, "Ignoring Kerberos-related parameters because ClickHouse was built without krb5 library support."); -#endif // USE_KRB5 - // No need to add any prefix, messages can be distinguished - kafka_config.set_log_callback( - [log = params.log](cppkafka::KafkaHandleBase & handle, int level, const std::string & facility, const std::string & message) - { - auto [poco_level, client_logs_level] = parseSyslogLevel(level); - const auto & kafka_object_config = handle.get_configuration(); - const std::string client_id_key{"client.id"}; - chassert(kafka_object_config.has_property(client_id_key) && "Kafka configuration doesn't have expected client.id set"); - LOG_IMPL( - log, - client_logs_level, - poco_level, - "[client.id:{}] [rdk:{}] {}", - kafka_object_config.get(client_id_key), - facility, - message); - }); - - /// NOTE: statistics should be consumed, otherwise it creates too much - /// entries in the queue, that leads to memory leak and slow shutdown. - if (!kafka_config.has_property("statistics.interval.ms")) - { - // every 3 seconds by default. set to 0 to disable. - kafka_config.set("statistics.interval.ms", "3000"); - } - // Configure interceptor to change thread name - // - // TODO: add interceptors support into the cppkafka. - // XXX: rdkafka uses pthread_set_name_np(), but glibc-compatibility overrides it to noop. - { - // This should be safe, since we wait the rdkafka object anyway. - void * self = static_cast(&storage); - - int status; - - status = rd_kafka_conf_interceptor_add_on_new( - kafka_config.get_handle(), "init", StorageKafkaInterceptors::rdKafkaOnNew, self); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - LOG_ERROR(params.log, "Cannot set new interceptor due to {} error", status); - - // cppkafka always copy the configuration - status = rd_kafka_conf_interceptor_add_on_conf_dup( - kafka_config.get_handle(), "init", StorageKafkaInterceptors::rdKafkaOnConfDup, self); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - LOG_ERROR(params.log, "Cannot set dup conf interceptor due to {} error", status); - } -} - -template -cppkafka::Configuration KafkaConfigLoader::getConsumerConfiguration(TKafkaStorage & storage, const ConsumerConfigParams & params) -{ - cppkafka::Configuration conf; - - conf.set("metadata.broker.list", params.brokers); - conf.set("group.id", params.group); - if (params.multiple_consumers) - conf.set("client.id", fmt::format("{}-{}", params.client_id, params.consumer_number)); - else - conf.set("client.id", params.client_id); - conf.set("client.software.name", VERSION_NAME); - conf.set("client.software.version", VERSION_DESCRIBE); - conf.set("auto.offset.reset", "earliest"); // If no offset stored for this group, read all messages from the start - - // that allows to prevent fast draining of the librdkafka queue - // during building of single insert block. Improves performance - // significantly, but may lead to bigger memory consumption. - size_t default_queued_min_messages = 100000; // must be greater than or equal to default - size_t max_allowed_queued_min_messages = 10000000; // must be less than or equal to max allowed value - conf.set( - "queued.min.messages", std::min(std::max(params.max_block_size, default_queued_min_messages), max_allowed_queued_min_messages)); - - updateGlobalConfiguration(conf, storage, params); - loadConsumerConfig(conf, params); - - // those settings should not be changed by users. - conf.set("enable.auto.commit", "false"); // We manually commit offsets after a stream successfully finished - conf.set("enable.auto.offset.store", "false"); // Update offset automatically - to commit them all at once. - conf.set("enable.partition.eof", "false"); // Ignore EOF messages - - for (auto & property : conf.get_all()) - { - LOG_TRACE(params.log, "Consumer set property {}:{}", property.first, property.second); - } - - return conf; -} - -template cppkafka::Configuration KafkaConfigLoader::getConsumerConfiguration(StorageKafka & storage, const ConsumerConfigParams & params); -template cppkafka::Configuration KafkaConfigLoader::getConsumerConfiguration(StorageKafka2 & storage, const ConsumerConfigParams & params); - -template -cppkafka::Configuration KafkaConfigLoader::getProducerConfiguration(TKafkaStorage & storage, const ProducerConfigParams & params) -{ - cppkafka::Configuration conf; - conf.set("metadata.broker.list", params.brokers); - conf.set("client.id", params.client_id); - conf.set("client.software.name", VERSION_NAME); - conf.set("client.software.version", VERSION_DESCRIBE); - - updateGlobalConfiguration(conf, storage, params); - loadProducerConfig(conf, params); - - for (auto & property : conf.get_all()) - { - LOG_TRACE(params.log, "Producer set property {}:{}", property.first, property.second); - } - - return conf; -} - -template cppkafka::Configuration KafkaConfigLoader::getProducerConfiguration(StorageKafka & storage, const ProducerConfigParams & params); -template cppkafka::Configuration KafkaConfigLoader::getProducerConfiguration(StorageKafka2 & storage, const ProducerConfigParams & params); - -void registerStorageKafka(StorageFactory & factory) -{ - auto creator_fn = [](const StorageFactory::Arguments & args) -> std::shared_ptr - { - ASTs & engine_args = args.engine_args; - size_t args_count = engine_args.size(); - const bool has_settings = args.storage_def->settings; - - auto kafka_settings = std::make_unique(); - String collection_name; - if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args, args.getLocalContext())) - { - for (const auto & setting : kafka_settings->all()) - { - const auto & setting_name = setting.getName(); - if (named_collection->has(setting_name)) - kafka_settings->set(setting_name, named_collection->get(setting_name)); - } - collection_name = assert_cast(args.engine_args[0].get())->name(); - } - - if (has_settings) - { - kafka_settings->loadFromQuery(*args.storage_def); - } - -// Check arguments and settings -#define CHECK_KAFKA_STORAGE_ARGUMENT(ARG_NUM, PAR_NAME, EVAL) \ - /* One of the four required arguments is not specified */ \ - if (args_count < (ARG_NUM) && (ARG_NUM) <= 4 && !kafka_settings->PAR_NAME.changed) \ - { \ - throw Exception( \ - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, \ - "Required parameter '{}' " \ - "for storage Kafka not specified", \ - #PAR_NAME); \ - } \ - if (args_count >= (ARG_NUM)) \ - { \ - /* The same argument is given in two places */ \ - if (has_settings && kafka_settings->PAR_NAME.changed) \ - { \ - throw Exception( \ - ErrorCodes::BAD_ARGUMENTS, \ - "The argument №{} of storage Kafka " \ - "and the parameter '{}' " \ - "in SETTINGS cannot be specified at the same time", \ - #ARG_NUM, \ - #PAR_NAME); \ - } \ - /* move engine args to settings */ \ - else \ - { \ - if constexpr ((EVAL) == 1) \ - { \ - engine_args[(ARG_NUM)-1] = evaluateConstantExpressionAsLiteral(engine_args[(ARG_NUM)-1], args.getLocalContext()); \ - } \ - if constexpr ((EVAL) == 2) \ - { \ - engine_args[(ARG_NUM)-1] \ - = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[(ARG_NUM)-1], args.getLocalContext()); \ - } \ - kafka_settings->PAR_NAME = engine_args[(ARG_NUM)-1]->as().value; \ - } \ - } - - /** Arguments of engine is following: - * - Kafka broker list - * - List of topics - * - Group ID (may be a constant expression with a string result) - * - Message format (string) - * - Row delimiter - * - Schema (optional, if the format supports it) - * - Number of consumers - * - Max block size for background consumption - * - Skip (at least) unreadable messages number - * - Do intermediate commits when the batch consumed and handled - */ - - /* 0 = raw, 1 = evaluateConstantExpressionAsLiteral, 2=evaluateConstantExpressionOrIdentifierAsLiteral */ - /// In case of named collection we already validated the arguments. - if (collection_name.empty()) - { - CHECK_KAFKA_STORAGE_ARGUMENT(1, kafka_broker_list, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(2, kafka_topic_list, 1) - CHECK_KAFKA_STORAGE_ARGUMENT(3, kafka_group_name, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(4, kafka_format, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(5, kafka_row_delimiter, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(6, kafka_schema, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(7, kafka_num_consumers, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(10, kafka_commit_every_batch, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(11, kafka_client_id, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(12, kafka_poll_timeout_ms, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(13, kafka_flush_interval_ms, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(14, kafka_thread_per_consumer, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(15, kafka_handle_error_mode, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(16, kafka_commit_on_select, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(17, kafka_max_rows_per_message, 0) - } - -#undef CHECK_KAFKA_STORAGE_ARGUMENT - - auto num_consumers = kafka_settings->kafka_num_consumers.value; - auto max_consumers = std::max(getNumberOfPhysicalCPUCores(), 16); - - if (!args.getLocalContext()->getSettingsRef().kafka_disable_num_consumers_limit && num_consumers > max_consumers) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "The number of consumers can not be bigger than {}. " - "A single consumer can read any number of partitions. " - "Extra consumers are relatively expensive, " - "and using a lot of them can lead to high memory and CPU usage. " - "To achieve better performance " - "of getting data from Kafka, consider using a setting kafka_thread_per_consumer=1, " - "and ensure you have enough threads " - "in MessageBrokerSchedulePool (background_message_broker_schedule_pool_size). " - "See also https://clickhouse.com/docs/integrations/kafka/kafka-table-engine#tuning-performance", - max_consumers); - } - else if (num_consumers < 1) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Number of consumers can not be lower than 1"); - } - - if (kafka_settings->kafka_max_block_size.changed && kafka_settings->kafka_max_block_size.value < 1) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "kafka_max_block_size can not be lower than 1"); - } - - if (kafka_settings->kafka_poll_max_batch_size.changed && kafka_settings->kafka_poll_max_batch_size.value < 1) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "kafka_poll_max_batch_size can not be lower than 1"); - } - NamesAndTypesList supported_columns; - for (const auto & column : args.columns) - { - if (column.default_desc.kind == ColumnDefaultKind::Alias) - supported_columns.emplace_back(column.name, column.type); - if (column.default_desc.kind == ColumnDefaultKind::Default && !column.default_desc.expression) - supported_columns.emplace_back(column.name, column.type); - } - // Kafka engine allows only ordinary columns without default expression or alias columns. - if (args.columns.getAll() != supported_columns) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "KafkaEngine doesn't support DEFAULT/MATERIALIZED/EPHEMERAL expressions for columns. " - "See https://clickhouse.com/docs/en/engines/table-engines/integrations/kafka/#configuration"); - } - - const auto has_keeper_path = kafka_settings->kafka_keeper_path.changed && !kafka_settings->kafka_keeper_path.value.empty(); - const auto has_replica_name = kafka_settings->kafka_replica_name.changed && !kafka_settings->kafka_replica_name.value.empty(); - - if (!has_keeper_path && !has_replica_name) - return std::make_shared( - args.table_id, args.getContext(), args.columns, args.comment, std::move(kafka_settings), collection_name); - - if (!args.getLocalContext()->getSettingsRef().allow_experimental_kafka_offsets_storage_in_keeper && !args.query.attach) - throw Exception( - ErrorCodes::SUPPORT_IS_DISABLED, - "Storing the Kafka offsets in Keeper is experimental. Set `allow_experimental_kafka_offsets_storage_in_keeper` setting " - "to enable it"); - - if (!has_keeper_path || !has_replica_name) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Either specify both zookeeper path and replica name or none of them"); - - const auto is_on_cluster = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; - const auto is_replicated_database = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY - && DatabaseCatalog::instance().getDatabase(args.table_id.database_name)->getEngineName() == "Replicated"; - - // UUID macro is only allowed: - // - with Atomic database only with ON CLUSTER queries, otherwise it is easy to misuse: each replica would have separate uuid generated. - // - with Replicated database - // - with attach queries, as those are used on server startup - const auto allow_uuid_macro = is_on_cluster || is_replicated_database || args.query.attach; - - auto context = args.getContext(); - // Unfold {database} and {table} macro on table creation, so table can be renamed. - if (args.mode < LoadingStrictnessLevel::ATTACH) - { - Macros::MacroExpansionInfo info; - /// NOTE: it's not recursive - info.expand_special_macros_only = true; - info.table_id = args.table_id; - // We could probably unfold UUID here too, but let's keep it similar to ReplicatedMergeTree, which doesn't do the unfolding. - info.table_id.uuid = UUIDHelpers::Nil; - kafka_settings->kafka_keeper_path.value = context->getMacros()->expand(kafka_settings->kafka_keeper_path.value, info); - - info.level = 0; - kafka_settings->kafka_replica_name.value = context->getMacros()->expand(kafka_settings->kafka_replica_name.value, info); - } - - - auto * settings_query = args.storage_def->settings; - chassert(has_settings && "Unexpected settings query in StorageKafka"); - - settings_query->changes.setSetting("kafka_keeper_path", kafka_settings->kafka_keeper_path.value); - settings_query->changes.setSetting("kafka_replica_name", kafka_settings->kafka_replica_name.value); - - // Expand other macros (such as {replica}). We do not expand them on previous step to make possible copying metadata files between replicas. - // Disable expanding {shard} macro, because it can lead to incorrect behavior and it doesn't make sense to shard Kafka tables. - Macros::MacroExpansionInfo info; - info.table_id = args.table_id; - if (is_replicated_database) - { - auto database = DatabaseCatalog::instance().getDatabase(args.table_id.database_name); - info.shard.reset(); - info.replica = getReplicatedDatabaseReplicaName(database); - } - if (!allow_uuid_macro) - info.table_id.uuid = UUIDHelpers::Nil; - kafka_settings->kafka_keeper_path.value = context->getMacros()->expand(kafka_settings->kafka_keeper_path.value, info); - - info.level = 0; - info.table_id.uuid = UUIDHelpers::Nil; - kafka_settings->kafka_replica_name.value = context->getMacros()->expand(kafka_settings->kafka_replica_name.value, info); - - return std::make_shared( - args.table_id, args.getContext(), args.columns, args.comment, std::move(kafka_settings), collection_name); - }; - - factory.registerStorage( - "Kafka", - creator_fn, - StorageFactory::StorageFeatures{ - .supports_settings = true, - }); -} - -namespace StorageKafkaUtils -{ -Names parseTopics(String topic_list) -{ - Names result; - boost::split(result, topic_list, [](char c) { return c == ','; }); - for (String & topic : result) - boost::trim(topic); - return result; -} - -String getDefaultClientId(const StorageID & table_id) -{ - return fmt::format("{}-{}-{}-{}", VERSION_NAME, getFQDNOrHostName(), table_id.database_name, table_id.table_name); -} - -void drainConsumer( - cppkafka::Consumer & consumer, const std::chrono::milliseconds drain_timeout, const LoggerPtr & log, ErrorHandler error_handler) -{ - auto start_time = std::chrono::steady_clock::now(); - cppkafka::Error last_error(RD_KAFKA_RESP_ERR_NO_ERROR); - - while (true) - { - auto msg = consumer.poll(100ms); - if (!msg) - break; - - auto error = msg.get_error(); - - if (error) - { - if (msg.is_eof() || error == last_error) - { - break; - } - else - { - LOG_ERROR(log, "Error during draining: {}", error); - error_handler(error); - } - } - - // i don't stop draining on first error, - // only if it repeats once again sequentially - last_error = error; - - auto ts = std::chrono::steady_clock::now(); - if (std::chrono::duration_cast(ts - start_time) > drain_timeout) - { - LOG_ERROR(log, "Timeout during draining."); - break; - } - } -} - -void eraseMessageErrors(Messages & messages, const LoggerPtr & log, ErrorHandler error_handler) -{ - size_t skipped = std::erase_if( - messages, - [&](auto & message) - { - if (auto error = message.get_error()) - { - ProfileEvents::increment(ProfileEvents::KafkaConsumerErrors); - LOG_ERROR(log, "Consumer error: {}", error); - error_handler(error); - return true; - } - return false; - }); - - if (skipped) - LOG_ERROR(log, "There were {} messages with an error", skipped); -} - -SettingsChanges createSettingsAdjustments(KafkaSettings & kafka_settings, const String & schema_name) -{ - SettingsChanges result; - // Needed for backward compatibility - if (!kafka_settings.input_format_skip_unknown_fields.changed) - { - // Always skip unknown fields regardless of the context (JSON or TSKV) - kafka_settings.input_format_skip_unknown_fields = true; - } - - if (!kafka_settings.input_format_allow_errors_ratio.changed) - { - kafka_settings.input_format_allow_errors_ratio = 0.; - } - - if (!kafka_settings.input_format_allow_errors_num.changed) - { - kafka_settings.input_format_allow_errors_num = kafka_settings.kafka_skip_broken_messages.value; - } - - if (!schema_name.empty()) - result.emplace_back("format_schema", schema_name); - - for (const auto & setting : kafka_settings) - { - const auto & name = setting.getName(); - if (name.find("kafka_") == std::string::npos) - result.emplace_back(name, setting.getValue()); - } - return result; -} - - -bool checkDependencies(const StorageID & table_id, const ContextPtr& context) -{ - // Check if all dependencies are attached - auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); - if (view_ids.empty()) - return true; - - // Check the dependencies are ready? - for (const auto & view_id : view_ids) - { - auto view = DatabaseCatalog::instance().tryGetTable(view_id, context); - if (!view) - return false; - - // If it materialized view, check it's target table - auto * materialized_view = dynamic_cast(view.get()); - if (materialized_view && !materialized_view->tryGetTargetTable()) - return false; - - // Check all its dependencies - if (!checkDependencies(view_id, context)) - return false; - } - - return true; -} - - -VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode) -{ - VirtualColumnsDescription desc; - - desc.addEphemeral("_topic", std::make_shared(std::make_shared()), ""); - desc.addEphemeral("_key", std::make_shared(), ""); - desc.addEphemeral("_offset", std::make_shared(), ""); - desc.addEphemeral("_partition", std::make_shared(), ""); - desc.addEphemeral("_timestamp", std::make_shared(std::make_shared()), ""); - desc.addEphemeral("_timestamp_ms", std::make_shared(std::make_shared(3)), ""); - desc.addEphemeral("_headers.name", std::make_shared(std::make_shared()), ""); - desc.addEphemeral("_headers.value", std::make_shared(std::make_shared()), ""); - - if (handle_error_mode == StreamingHandleErrorMode::STREAM) - { - desc.addEphemeral("_raw_message", std::make_shared(), ""); - desc.addEphemeral("_error", std::make_shared(), ""); - } - - return desc; -} -} - -template struct StorageKafkaInterceptors; -template struct StorageKafkaInterceptors; - -} diff --git a/src/Storages/Kafka/StorageKafkaCommon.h b/src/Storages/Kafka/StorageKafkaCommon.h deleted file mode 100644 index dd38ee69675..00000000000 --- a/src/Storages/Kafka/StorageKafkaCommon.h +++ /dev/null @@ -1,116 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace Poco -{ -namespace Util -{ - class AbstractConfiguration; -} -} - -namespace DB -{ - -struct KafkaSettings; -class VirtualColumnsDescription; - -template -struct StorageKafkaInterceptors -{ - static rd_kafka_resp_err_t rdKafkaOnThreadStart(rd_kafka_t *, rd_kafka_thread_type_t thread_type, const char *, void * ctx); - - static rd_kafka_resp_err_t rdKafkaOnThreadExit(rd_kafka_t *, rd_kafka_thread_type_t, const char *, void * ctx); - - static rd_kafka_resp_err_t - rdKafkaOnNew(rd_kafka_t * rk, const rd_kafka_conf_t *, void * ctx, char * /*errstr*/, size_t /*errstr_size*/); - - static rd_kafka_resp_err_t rdKafkaOnConfDup( - rd_kafka_conf_t * new_conf, const rd_kafka_conf_t * /*old_conf*/, size_t /*filter_cnt*/, const char ** /*filter*/, void * ctx); -}; - -struct KafkaConfigLoader -{ - static inline const String CONFIG_KAFKA_TAG = "kafka"; - static inline const String CONFIG_KAFKA_TOPIC_TAG = "kafka_topic"; - static inline const String CONFIG_NAME_TAG = "name"; - static inline const String CONFIG_KAFKA_CONSUMER_TAG = "consumer"; - static inline const String CONFIG_KAFKA_PRODUCER_TAG = "producer"; - using LogCallback = cppkafka::Configuration::LogCallback; - - - struct LoadConfigParams - { - const Poco::Util::AbstractConfiguration & config; - String & collection_name; - const Names & topics; - LoggerPtr & log; - }; - - struct ConsumerConfigParams : public LoadConfigParams - { - String brokers; - String group; - bool multiple_consumers; - size_t consumer_number; - String client_id; - size_t max_block_size; - }; - - struct ProducerConfigParams : public LoadConfigParams - { - String brokers; - String client_id; - }; - - template - static cppkafka::Configuration getConsumerConfiguration(TKafkaStorage & storage, const ConsumerConfigParams & params); - - template - static cppkafka::Configuration getProducerConfiguration(TKafkaStorage & storage, const ProducerConfigParams & params); -}; - -namespace StorageKafkaUtils -{ -Names parseTopics(String topic_list); -String getDefaultClientId(const StorageID & table_id); - -using ErrorHandler = std::function; - -void drainConsumer( - cppkafka::Consumer & consumer, - std::chrono::milliseconds drain_timeout, - const LoggerPtr & log, - ErrorHandler error_handler = [](const cppkafka::Error & /*err*/) {}); - -using Messages = std::vector; -void eraseMessageErrors(Messages & messages, const LoggerPtr & log, ErrorHandler error_handler = [](const cppkafka::Error & /*err*/) {}); - -SettingsChanges createSettingsAdjustments(KafkaSettings & kafka_settings, const String & schema_name); - -bool checkDependencies(const StorageID & table_id, const ContextPtr& context); - -VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode); -} -} - -template <> -struct fmt::formatter : fmt::ostream_formatter -{ -}; -template <> -struct fmt::formatter : fmt::ostream_formatter -{ -}; diff --git a/src/Storages/Kafka/StorageKafkaUtils.cpp b/src/Storages/Kafka/StorageKafkaUtils.cpp new file mode 100644 index 00000000000..c510303f45e --- /dev/null +++ b/src/Storages/Kafka/StorageKafkaUtils.cpp @@ -0,0 +1,458 @@ +#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 + +#if USE_KRB5 +# include +#endif // USE_KRB5 + +namespace CurrentMetrics +{ +extern const Metric KafkaLibrdkafkaThreads; +} + +namespace ProfileEvents +{ +extern const Event KafkaConsumerErrors; +} + +namespace DB +{ + +using namespace std::chrono_literals; + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int SUPPORT_IS_DISABLED; +} + + +void registerStorageKafka(StorageFactory & factory) +{ + auto creator_fn = [](const StorageFactory::Arguments & args) -> std::shared_ptr + { + ASTs & engine_args = args.engine_args; + size_t args_count = engine_args.size(); + const bool has_settings = args.storage_def->settings; + + auto kafka_settings = std::make_unique(); + String collection_name; + if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args, args.getLocalContext())) + { + for (const auto & setting : kafka_settings->all()) + { + const auto & setting_name = setting.getName(); + if (named_collection->has(setting_name)) + kafka_settings->set(setting_name, named_collection->get(setting_name)); + } + collection_name = assert_cast(args.engine_args[0].get())->name(); + } + + if (has_settings) + { + kafka_settings->loadFromQuery(*args.storage_def); + } + +// Check arguments and settings +#define CHECK_KAFKA_STORAGE_ARGUMENT(ARG_NUM, PAR_NAME, EVAL) \ + /* One of the four required arguments is not specified */ \ + if (args_count < (ARG_NUM) && (ARG_NUM) <= 4 && !kafka_settings->PAR_NAME.changed) \ + { \ + throw Exception( \ + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, \ + "Required parameter '{}' " \ + "for storage Kafka not specified", \ + #PAR_NAME); \ + } \ + if (args_count >= (ARG_NUM)) \ + { \ + /* The same argument is given in two places */ \ + if (has_settings && kafka_settings->PAR_NAME.changed) \ + { \ + throw Exception( \ + ErrorCodes::BAD_ARGUMENTS, \ + "The argument №{} of storage Kafka " \ + "and the parameter '{}' " \ + "in SETTINGS cannot be specified at the same time", \ + #ARG_NUM, \ + #PAR_NAME); \ + } \ + /* move engine args to settings */ \ + else \ + { \ + if constexpr ((EVAL) == 1) \ + { \ + engine_args[(ARG_NUM)-1] = evaluateConstantExpressionAsLiteral(engine_args[(ARG_NUM)-1], args.getLocalContext()); \ + } \ + if constexpr ((EVAL) == 2) \ + { \ + engine_args[(ARG_NUM)-1] \ + = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[(ARG_NUM)-1], args.getLocalContext()); \ + } \ + kafka_settings->PAR_NAME = engine_args[(ARG_NUM)-1]->as().value; \ + } \ + } + + /** Arguments of engine is following: + * - Kafka broker list + * - List of topics + * - Group ID (may be a constant expression with a string result) + * - Message format (string) + * - Row delimiter + * - Schema (optional, if the format supports it) + * - Number of consumers + * - Max block size for background consumption + * - Skip (at least) unreadable messages number + * - Do intermediate commits when the batch consumed and handled + */ + + /* 0 = raw, 1 = evaluateConstantExpressionAsLiteral, 2=evaluateConstantExpressionOrIdentifierAsLiteral */ + /// In case of named collection we already validated the arguments. + if (collection_name.empty()) + { + CHECK_KAFKA_STORAGE_ARGUMENT(1, kafka_broker_list, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(2, kafka_topic_list, 1) + CHECK_KAFKA_STORAGE_ARGUMENT(3, kafka_group_name, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(4, kafka_format, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(5, kafka_row_delimiter, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(6, kafka_schema, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(7, kafka_num_consumers, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(10, kafka_commit_every_batch, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(11, kafka_client_id, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(12, kafka_poll_timeout_ms, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(13, kafka_flush_interval_ms, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(14, kafka_thread_per_consumer, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(15, kafka_handle_error_mode, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(16, kafka_commit_on_select, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(17, kafka_max_rows_per_message, 0) + } + +#undef CHECK_KAFKA_STORAGE_ARGUMENT + + auto num_consumers = kafka_settings->kafka_num_consumers.value; + auto max_consumers = std::max(getNumberOfPhysicalCPUCores(), 16); + + if (!args.getLocalContext()->getSettingsRef().kafka_disable_num_consumers_limit && num_consumers > max_consumers) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "The number of consumers can not be bigger than {}. " + "A single consumer can read any number of partitions. " + "Extra consumers are relatively expensive, " + "and using a lot of them can lead to high memory and CPU usage. " + "To achieve better performance " + "of getting data from Kafka, consider using a setting kafka_thread_per_consumer=1, " + "and ensure you have enough threads " + "in MessageBrokerSchedulePool (background_message_broker_schedule_pool_size). " + "See also https://clickhouse.com/docs/integrations/kafka/kafka-table-engine#tuning-performance", + max_consumers); + } + else if (num_consumers < 1) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Number of consumers can not be lower than 1"); + } + + if (kafka_settings->kafka_max_block_size.changed && kafka_settings->kafka_max_block_size.value < 1) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "kafka_max_block_size can not be lower than 1"); + } + + if (kafka_settings->kafka_poll_max_batch_size.changed && kafka_settings->kafka_poll_max_batch_size.value < 1) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "kafka_poll_max_batch_size can not be lower than 1"); + } + NamesAndTypesList supported_columns; + for (const auto & column : args.columns) + { + if (column.default_desc.kind == ColumnDefaultKind::Alias) + supported_columns.emplace_back(column.name, column.type); + if (column.default_desc.kind == ColumnDefaultKind::Default && !column.default_desc.expression) + supported_columns.emplace_back(column.name, column.type); + } + // Kafka engine allows only ordinary columns without default expression or alias columns. + if (args.columns.getAll() != supported_columns) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "KafkaEngine doesn't support DEFAULT/MATERIALIZED/EPHEMERAL expressions for columns. " + "See https://clickhouse.com/docs/en/engines/table-engines/integrations/kafka/#configuration"); + } + + const auto has_keeper_path = kafka_settings->kafka_keeper_path.changed && !kafka_settings->kafka_keeper_path.value.empty(); + const auto has_replica_name = kafka_settings->kafka_replica_name.changed && !kafka_settings->kafka_replica_name.value.empty(); + + if (!has_keeper_path && !has_replica_name) + return std::make_shared( + args.table_id, args.getContext(), args.columns, args.comment, std::move(kafka_settings), collection_name); + + if (!args.getLocalContext()->getSettingsRef().allow_experimental_kafka_offsets_storage_in_keeper && !args.query.attach) + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "Storing the Kafka offsets in Keeper is experimental. Set `allow_experimental_kafka_offsets_storage_in_keeper` setting " + "to enable it"); + + if (!has_keeper_path || !has_replica_name) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Either specify both zookeeper path and replica name or none of them"); + + const auto is_on_cluster = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; + const auto is_replicated_database = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY + && DatabaseCatalog::instance().getDatabase(args.table_id.database_name)->getEngineName() == "Replicated"; + + // UUID macro is only allowed: + // - with Atomic database only with ON CLUSTER queries, otherwise it is easy to misuse: each replica would have separate uuid generated. + // - with Replicated database + // - with attach queries, as those are used on server startup + const auto allow_uuid_macro = is_on_cluster || is_replicated_database || args.query.attach; + + auto context = args.getContext(); + // Unfold {database} and {table} macro on table creation, so table can be renamed. + if (args.mode < LoadingStrictnessLevel::ATTACH) + { + Macros::MacroExpansionInfo info; + /// NOTE: it's not recursive + info.expand_special_macros_only = true; + info.table_id = args.table_id; + // We could probably unfold UUID here too, but let's keep it similar to ReplicatedMergeTree, which doesn't do the unfolding. + info.table_id.uuid = UUIDHelpers::Nil; + kafka_settings->kafka_keeper_path.value = context->getMacros()->expand(kafka_settings->kafka_keeper_path.value, info); + + info.level = 0; + kafka_settings->kafka_replica_name.value = context->getMacros()->expand(kafka_settings->kafka_replica_name.value, info); + } + + + auto * settings_query = args.storage_def->settings; + chassert(has_settings && "Unexpected settings query in StorageKafka"); + + settings_query->changes.setSetting("kafka_keeper_path", kafka_settings->kafka_keeper_path.value); + settings_query->changes.setSetting("kafka_replica_name", kafka_settings->kafka_replica_name.value); + + // Expand other macros (such as {replica}). We do not expand them on previous step to make possible copying metadata files between replicas. + // Disable expanding {shard} macro, because it can lead to incorrect behavior and it doesn't make sense to shard Kafka tables. + Macros::MacroExpansionInfo info; + info.table_id = args.table_id; + if (is_replicated_database) + { + auto database = DatabaseCatalog::instance().getDatabase(args.table_id.database_name); + info.shard.reset(); + info.replica = getReplicatedDatabaseReplicaName(database); + } + if (!allow_uuid_macro) + info.table_id.uuid = UUIDHelpers::Nil; + kafka_settings->kafka_keeper_path.value = context->getMacros()->expand(kafka_settings->kafka_keeper_path.value, info); + + info.level = 0; + info.table_id.uuid = UUIDHelpers::Nil; + kafka_settings->kafka_replica_name.value = context->getMacros()->expand(kafka_settings->kafka_replica_name.value, info); + + return std::make_shared( + args.table_id, args.getContext(), args.columns, args.comment, std::move(kafka_settings), collection_name); + }; + + factory.registerStorage( + "Kafka", + creator_fn, + StorageFactory::StorageFeatures{ + .supports_settings = true, + }); +} + +namespace StorageKafkaUtils +{ +Names parseTopics(String topic_list) +{ + Names result; + boost::split(result, topic_list, [](char c) { return c == ','; }); + for (String & topic : result) + boost::trim(topic); + return result; +} + +String getDefaultClientId(const StorageID & table_id) +{ + return fmt::format("{}-{}-{}-{}", VERSION_NAME, getFQDNOrHostName(), table_id.database_name, table_id.table_name); +} + +void drainConsumer( + cppkafka::Consumer & consumer, const std::chrono::milliseconds drain_timeout, const LoggerPtr & log, ErrorHandler error_handler) +{ + auto start_time = std::chrono::steady_clock::now(); + cppkafka::Error last_error(RD_KAFKA_RESP_ERR_NO_ERROR); + + while (true) + { + auto msg = consumer.poll(100ms); + if (!msg) + break; + + auto error = msg.get_error(); + + if (error) + { + if (msg.is_eof() || error == last_error) + { + break; + } + else + { + LOG_ERROR(log, "Error during draining: {}", error); + error_handler(error); + } + } + + // i don't stop draining on first error, + // only if it repeats once again sequentially + last_error = error; + + auto ts = std::chrono::steady_clock::now(); + if (std::chrono::duration_cast(ts - start_time) > drain_timeout) + { + LOG_ERROR(log, "Timeout during draining."); + break; + } + } +} + +void eraseMessageErrors(Messages & messages, const LoggerPtr & log, ErrorHandler error_handler) +{ + size_t skipped = std::erase_if( + messages, + [&](auto & message) + { + if (auto error = message.get_error()) + { + ProfileEvents::increment(ProfileEvents::KafkaConsumerErrors); + LOG_ERROR(log, "Consumer error: {}", error); + error_handler(error); + return true; + } + return false; + }); + + if (skipped) + LOG_ERROR(log, "There were {} messages with an error", skipped); +} + +SettingsChanges createSettingsAdjustments(KafkaSettings & kafka_settings, const String & schema_name) +{ + SettingsChanges result; + // Needed for backward compatibility + if (!kafka_settings.input_format_skip_unknown_fields.changed) + { + // Always skip unknown fields regardless of the context (JSON or TSKV) + kafka_settings.input_format_skip_unknown_fields = true; + } + + if (!kafka_settings.input_format_allow_errors_ratio.changed) + { + kafka_settings.input_format_allow_errors_ratio = 0.; + } + + if (!kafka_settings.input_format_allow_errors_num.changed) + { + kafka_settings.input_format_allow_errors_num = kafka_settings.kafka_skip_broken_messages.value; + } + + if (!schema_name.empty()) + result.emplace_back("format_schema", schema_name); + + for (const auto & setting : kafka_settings) + { + const auto & name = setting.getName(); + if (name.find("kafka_") == std::string::npos) + result.emplace_back(name, setting.getValue()); + } + return result; +} + + +bool checkDependencies(const StorageID & table_id, const ContextPtr& context) +{ + // Check if all dependencies are attached + auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); + if (view_ids.empty()) + return true; + + // Check the dependencies are ready? + for (const auto & view_id : view_ids) + { + auto view = DatabaseCatalog::instance().tryGetTable(view_id, context); + if (!view) + return false; + + // If it materialized view, check it's target table + auto * materialized_view = dynamic_cast(view.get()); + if (materialized_view && !materialized_view->tryGetTargetTable()) + return false; + + // Check all its dependencies + if (!checkDependencies(view_id, context)) + return false; + } + + return true; +} + + +VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode) +{ + VirtualColumnsDescription desc; + + desc.addEphemeral("_topic", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_key", std::make_shared(), ""); + desc.addEphemeral("_offset", std::make_shared(), ""); + desc.addEphemeral("_partition", std::make_shared(), ""); + desc.addEphemeral("_timestamp", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_timestamp_ms", std::make_shared(std::make_shared(3)), ""); + desc.addEphemeral("_headers.name", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_headers.value", std::make_shared(std::make_shared()), ""); + + if (handle_error_mode == StreamingHandleErrorMode::STREAM) + { + desc.addEphemeral("_raw_message", std::make_shared(), ""); + desc.addEphemeral("_error", std::make_shared(), ""); + } + + return desc; +} +} +} diff --git a/src/Storages/Kafka/StorageKafkaUtils.h b/src/Storages/Kafka/StorageKafkaUtils.h new file mode 100644 index 00000000000..cc956dde78d --- /dev/null +++ b/src/Storages/Kafka/StorageKafkaUtils.h @@ -0,0 +1,61 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace Poco +{ +namespace Util +{ + class AbstractConfiguration; +} +} + +namespace DB +{ + +class VirtualColumnsDescription; +struct KafkaSettings; + +namespace StorageKafkaUtils +{ +Names parseTopics(String topic_list); +String getDefaultClientId(const StorageID & table_id); + +using ErrorHandler = std::function; + +void drainConsumer( + cppkafka::Consumer & consumer, + std::chrono::milliseconds drain_timeout, + const LoggerPtr & log, + ErrorHandler error_handler = [](const cppkafka::Error & /*err*/) {}); + +using Messages = std::vector; +void eraseMessageErrors(Messages & messages, const LoggerPtr & log, ErrorHandler error_handler = [](const cppkafka::Error & /*err*/) {}); + +SettingsChanges createSettingsAdjustments(KafkaSettings & kafka_settings, const String & schema_name); + +bool checkDependencies(const StorageID & table_id, const ContextPtr& context); + +VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode); +} +} + +template <> +struct fmt::formatter : fmt::ostream_formatter +{ +}; +template <> +struct fmt::formatter : fmt::ostream_formatter +{ +}; From d83c0c1b3b189a78833afec5f87e7004b0f934e3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 1 Aug 2024 07:36:53 +0000 Subject: [PATCH 250/644] prevent normalization of WITH RECURSIVE alias --- src/Interpreters/AddDefaultDatabaseVisitor.h | 8 ++++++++ .../0_stateless/03215_view_with_recursive.reference | 1 + .../0_stateless/03215_view_with_recursive.sql | 13 +++++++++++++ 3 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/03215_view_with_recursive.reference create mode 100644 tests/queries/0_stateless/03215_view_with_recursive.sql diff --git a/src/Interpreters/AddDefaultDatabaseVisitor.h b/src/Interpreters/AddDefaultDatabaseVisitor.h index 356bffa75e9..ced94963d5b 100644 --- a/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -100,6 +101,7 @@ private: const String database_name; std::set external_tables; + mutable String with_alias; bool only_replace_current_database_function = false; bool only_replace_in_join = false; @@ -117,6 +119,9 @@ private: void visit(ASTSelectQuery & select, ASTPtr &) const { + if (auto with = select.with()) + with_alias = with->children[0]->as()->name; + if (select.tables()) tryVisit(select.refTables()); @@ -165,6 +170,9 @@ private: /// There is temporary table with such name, should not be rewritten. if (external_tables.contains(identifier.shortName())) return; + /// This is WITH RECURSIVE alias. + if (identifier.name() == with_alias) + return; auto qualified_identifier = std::make_shared(database_name, identifier.name()); if (!identifier.alias.empty()) diff --git a/tests/queries/0_stateless/03215_view_with_recursive.reference b/tests/queries/0_stateless/03215_view_with_recursive.reference new file mode 100644 index 00000000000..c3ac783e702 --- /dev/null +++ b/tests/queries/0_stateless/03215_view_with_recursive.reference @@ -0,0 +1 @@ +5050 diff --git a/tests/queries/0_stateless/03215_view_with_recursive.sql b/tests/queries/0_stateless/03215_view_with_recursive.sql new file mode 100644 index 00000000000..cac47124d51 --- /dev/null +++ b/tests/queries/0_stateless/03215_view_with_recursive.sql @@ -0,0 +1,13 @@ +CREATE VIEW 03215_test_v +AS WITH RECURSIVE test_table AS + ( + SELECT 1 AS number + UNION ALL + SELECT number + 1 + FROM test_table + WHERE number < 100 + ) +SELECT sum(number) +FROM test_table; + +SELECT * FROM 03215_test_v; From a78b1ddd976f98b1952e7ee350b1fcd1935ede19 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 1 Aug 2024 08:44:49 +0000 Subject: [PATCH 251/644] fix --- src/Interpreters/AddDefaultDatabaseVisitor.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/AddDefaultDatabaseVisitor.h b/src/Interpreters/AddDefaultDatabaseVisitor.h index ced94963d5b..efe39702fea 100644 --- a/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -120,7 +120,8 @@ private: void visit(ASTSelectQuery & select, ASTPtr &) const { if (auto with = select.with()) - with_alias = with->children[0]->as()->name; + if (auto with_element = with->children[0]->as()) + with_alias = with_element->name; if (select.tables()) tryVisit(select.refTables()); @@ -171,7 +172,7 @@ private: if (external_tables.contains(identifier.shortName())) return; /// This is WITH RECURSIVE alias. - if (identifier.name() == with_alias) + if (!with_alias.empty() && identifier.name() == with_alias) return; auto qualified_identifier = std::make_shared(database_name, identifier.name()); From 09b4d4ff6509eaa7b42c8f0f174e879731ddce0a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 1 Aug 2024 08:56:37 +0000 Subject: [PATCH 252/644] fix --- src/Interpreters/AddDefaultDatabaseVisitor.h | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/AddDefaultDatabaseVisitor.h b/src/Interpreters/AddDefaultDatabaseVisitor.h index efe39702fea..5e46a653efa 100644 --- a/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -119,9 +119,8 @@ private: void visit(ASTSelectQuery & select, ASTPtr &) const { - if (auto with = select.with()) - if (auto with_element = with->children[0]->as()) - with_alias = with_element->name; + if (select.recursive_with) + with_alias = select.with()->children[0]->as()->name; if (select.tables()) tryVisit(select.refTables()); From d2e0668d5129a4e60f462de5e5b683099f49bf4b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 1 Aug 2024 13:51:35 +0200 Subject: [PATCH 253/644] fix settingsChangesHistory after merge with master --- src/Core/SettingsChangesHistory.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index a01c5faaf10..28a732c6177 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -81,9 +81,6 @@ static std::initializer_list Date: Thu, 1 Aug 2024 20:16:10 +0200 Subject: [PATCH 254/644] fix merge with master --- docs/en/operations/settings/settings.md | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b880c42a45b..c621f2db5ae 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5615,18 +5615,6 @@ Disable all insert and mutations (alter table update / alter table delete / alte Default value: `false`. -## restore_replace_external_engines_to_null - -For testing purposes. Replaces all external engines to Null to not initiate external connections. - -Default value: `False` - -## restore_replace_external_table_functions_to_null - -For testing purposes. Replaces all external table functions to Null to not initiate external connections. - -Default value: `False` - ## use_hive_partitioning When enabled, ClickHouse will detect Hive-style partitioning in path (`/name=value/`) in file-like table engines [File](../../engines/table-engines/special/file.md#hive-style-partitioning)/[S3](../../engines/table-engines/integrations/s3.md#hive-style-partitioning)/[URL](../../engines/table-engines/special/url.md#hive-style-partitioning)/[HDFS](../../engines/table-engines/integrations/hdfs.md#hive-style-partitioning)/[AzureBlobStorage](../../engines/table-engines/integrations/azureBlobStorage.md#hive-style-partitioning) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. From a80c7c080cdbb3bec4662501686133298b7a4a2d Mon Sep 17 00:00:00 2001 From: Jacob Reckhard Date: Thu, 1 Aug 2024 18:23:11 -0600 Subject: [PATCH 255/644] Added support for reading multilinestring wkts --- docs/en/sql-reference/data-types/geo.md | 42 ++++++++++++ .../en/sql-reference/functions/geo/polygon.md | 42 ++++++++++-- src/DataTypes/DataTypeCustomGeo.cpp | 7 ++ src/DataTypes/DataTypeCustomGeo.h | 6 ++ src/Functions/geometryConverters.h | 64 +++++++++++++++++++ src/Functions/polygonsIntersection.cpp | 2 + src/Functions/polygonsSymDifference.cpp | 2 + src/Functions/polygonsUnion.cpp | 2 + src/Functions/polygonsWithin.cpp | 2 + src/Functions/readWkt.cpp | 30 +++++++++ .../03215_multilinestring_geometry.reference | 17 +++++ .../03215_multilinestring_geometry.sql | 12 ++++ .../aspell-ignore/en/aspell-dict.txt | 4 +- 13 files changed, 227 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03215_multilinestring_geometry.reference create mode 100644 tests/queries/0_stateless/03215_multilinestring_geometry.sql diff --git a/docs/en/sql-reference/data-types/geo.md b/docs/en/sql-reference/data-types/geo.md index 7ffc7447d96..8ce53bb2ef2 100644 --- a/docs/en/sql-reference/data-types/geo.md +++ b/docs/en/sql-reference/data-types/geo.md @@ -52,6 +52,48 @@ Result: └───────────────────────────────┴───────────────┘ ``` +## LineString + +`LineString` is a line stored as an array of points: [Array](array.md)([Point](#point)). + +**Example** + +Query: + +```sql +CREATE TABLE geo_linestring (l LineString) ENGINE = Memory(); +INSERT INTO geo_linestring VALUES([(0, 0), (10, 0), (10, 10), (0, 10)]); +SELECT l, toTypeName(l) FROM geo_linestring; +``` +Result: + +``` text +┌─r─────────────────────────────┬─toTypeName(r)─┐ +│ [(0,0),(10,0),(10,10),(0,10)] │ LineString │ +└───────────────────────────────┴───────────────┘ +``` + +## MultiLineString + +`MultiLineString` is multiple lines stored as an array of `LineString`: [Array](array.md)([LineString](#linestring)). + +**Example** + +Query: + +```sql +CREATE TABLE geo_multilinestring (l MultiLineString) ENGINE = Memory(); +INSERT INTO geo_multilinestring VALUES([[(0, 0), (10, 0), (10, 10), (0, 10)], [(1, 1), (2, 2), (3, 3)]]); +SELECT l, toTypeName(l) FROM geo_multilinestring; +``` +Result: + +``` text +┌─l───────────────────────────────────────────────────┬─toTypeName(l)───┐ +│ [[(0,0),(10,0),(10,10),(0,10)],[(1,1),(2,2),(3,3)]] │ MultiLineString │ +└─────────────────────────────────────────────────────┴─────────────────┘ +``` + ## Polygon `Polygon` is a polygon with holes stored as an array of rings: [Array](array.md)([Ring](#ring)). First element of outer array is the outer shape of polygon and all the following elements are holes. diff --git a/docs/en/sql-reference/functions/geo/polygon.md b/docs/en/sql-reference/functions/geo/polygon.md index 25a7a1fac8e..c054e05d39c 100644 --- a/docs/en/sql-reference/functions/geo/polygon.md +++ b/docs/en/sql-reference/functions/geo/polygon.md @@ -6,11 +6,13 @@ title: "Functions for Working with Polygons" ## WKT -Returns a WKT (Well Known Text) geometric object from various [Geo Data Types](../../data-types/geo.md). Supported WKT objects are: +Returns a WKT (Well Known Text) geometric object from various [Geo Data Types](../../data-types/geo.md). Supported WKT objects are: - POINT - POLYGON - MULTIPOLYGON +- LINESTRING +- MULTILINESTRING **Syntax** @@ -26,12 +28,16 @@ WKT(geo_data) - [Ring](../../data-types/geo.md#ring) - [Polygon](../../data-types/geo.md#polygon) - [MultiPolygon](../../data-types/geo.md#multipolygon) +- [LineString](../../data-types/geo.md#linestring) +- [MultiLineString](../../data-types/geo.md#multilinestring) **Returned value** - WKT geometric object `POINT` is returned for a Point. - WKT geometric object `POLYGON` is returned for a Polygon -- WKT geometric object `MULTIPOLYGON` is returned for a MultiPolygon. +- WKT geometric object `MULTIPOLYGON` is returned for a MultiPolygon. +- WKT geometric object `LINESTRING` is returned for a LineString. +- WKT geometric object `MULTILINESTRING` is returned for a MultiLineString. **Examples** @@ -84,7 +90,7 @@ SELECT ### Input parameters -String starting with `MULTIPOLYGON` +String starting with `MULTIPOLYGON` ### Returned value @@ -170,6 +176,34 @@ SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'); [(1,1),(2,2),(3,3),(1,1)] ``` +## readWKTMultiLineString + +Parses a Well-Known Text (WKT) representation of a MultiLineString geometry and returns it in the internal ClickHouse format. + +### Syntax + +```sql +readWKTMultiLineString(wkt_string) +``` + +### Arguments + +- `wkt_string`: The input WKT string representing a MultiLineString geometry. + +### Returned value + +The function returns a ClickHouse internal representation of the multilinestring geometry. + +### Example + +```sql +SELECT wkt(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3), (4 4, 5 5, 6 6))')); +``` + +```response +[[(1,1),(2,2),(3,3)],[(4,4),(5,5),(6,6)]] +``` + ## readWKTRing Parses a Well-Known Text (WKT) representation of a Polygon geometry and returns a ring (closed linestring) in the internal ClickHouse format. @@ -219,7 +253,7 @@ UInt8, 0 for false, 1 for true ## polygonsDistanceSpherical -Calculates the minimal distance between two points where one point belongs to the first polygon and the second to another polygon. Spherical means that coordinates are interpreted as coordinates on a pure and ideal sphere, which is not true for the Earth. Using this type of coordinate system speeds up execution, but of course is not precise. +Calculates the minimal distance between two points where one point belongs to the first polygon and the second to another polygon. Spherical means that coordinates are interpreted as coordinates on a pure and ideal sphere, which is not true for the Earth. Using this type of coordinate system speeds up execution, but of course is not precise. ### Example diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp index 0736d837d46..d72787647c3 100644 --- a/src/DataTypes/DataTypeCustomGeo.cpp +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -24,6 +24,13 @@ void registerDataTypeDomainGeo(DataTypeFactory & factory) std::make_unique(std::make_unique())); }); + // Custom type for mulitple lines stored as Array(LineString) + factory.registerSimpleDataTypeCustom("MultiLineString", [] + { + return std::make_pair(DataTypeFactory::instance().get("Array(LineString)"), + std::make_unique(std::make_unique())); + }); + // Custom type for simple polygon without holes stored as Array(Point) factory.registerSimpleDataTypeCustom("Ring", [] { diff --git a/src/DataTypes/DataTypeCustomGeo.h b/src/DataTypes/DataTypeCustomGeo.h index 0a1c83e4638..6a632f0d05c 100644 --- a/src/DataTypes/DataTypeCustomGeo.h +++ b/src/DataTypes/DataTypeCustomGeo.h @@ -17,6 +17,12 @@ public: DataTypeLineStringName() : DataTypeCustomFixedName("LineString") {} }; +class DataTypeMultiLineStringName : public DataTypeCustomFixedName +{ +public: + DataTypeMultiLineStringName() : DataTypeCustomFixedName("MultiLineString") {} +}; + class DataTypeRingName : public DataTypeCustomFixedName { public: diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index 03831d37e0c..bf975017a6d 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -31,6 +31,9 @@ namespace ErrorCodes template using LineString = boost::geometry::model::linestring; +template +using MultiLineString = boost::geometry::model::multi_linestring>; + template using Ring = boost::geometry::model::ring; @@ -42,12 +45,14 @@ using MultiPolygon = boost::geometry::model::multi_polygon>; using CartesianPoint = boost::geometry::model::d2::point_xy; using CartesianLineString = LineString; +using CartesianMultiLineString = MultiLineString; using CartesianRing = Ring; using CartesianPolygon = Polygon; using CartesianMultiPolygon = MultiPolygon; using SphericalPoint = boost::geometry::model::point>; using SphericalLineString = LineString; +using SphericalMultiLineString = MultiLineString; using SphericalRing = Ring; using SphericalPolygon = Polygon; using SphericalMultiPolygon = MultiPolygon; @@ -113,6 +118,28 @@ struct ColumnToLineStringsConverter } }; +/** + * Class which converts Column with type Array(Array(Tuple(Float64, Float64))) to a vector of boost multi_linestring type. +*/ +template +struct ColumnToMultiLineStringsConverter +{ + static std::vector> convert(ColumnPtr col) + { + const IColumn::Offsets & offsets = typeid_cast(*col).getOffsets(); + size_t prev_offset = 0; + std::vector> answer(offsets.size()); + auto all_linestrings = ColumnToLineStringsConverter::convert(typeid_cast(*col).getDataPtr()); + for (size_t iter = 0; iter < offsets.size() && iter < all_linestrings.size(); ++iter) + { + for (size_t linestring_iter = prev_offset; linestring_iter < offsets[iter]; ++linestring_iter) + answer[iter].emplace_back(std::move(all_linestrings[linestring_iter])); + prev_offset = offsets[iter]; + } + return answer; + } +}; + /** * Class which converts Column with type Array(Tuple(Float64, Float64)) to a vector of boost ring type. */ @@ -268,6 +295,38 @@ private: ColumnUInt64::MutablePtr offsets; }; +/// Serialize Point, MultiLineString as MultiLineString +template +class MultiLineStringSerializer +{ +public: + MultiLineStringSerializer() + : offsets(ColumnUInt64::create()) + {} + + explicit MultiLineStringSerializer(size_t n) + : offsets(ColumnUInt64::create(n)) + {} + + void add(const MultiLineString & multilinestring) + { + size += multilinestring.size(); + offsets->insertValue(size); + for (const auto & linestring : multilinestring) + linestring_serializer.add(linestring); + } + + ColumnPtr finalize() + { + return ColumnArray::create(linestring_serializer.finalize(), std::move(offsets)); + } + +private: + size_t size = 0; + LineStringSerializer linestring_serializer; + ColumnUInt64::MutablePtr offsets; +}; + /// Almost the same as LineStringSerializer /// Serialize Point, Ring as Ring template @@ -411,6 +470,11 @@ static void callOnGeometryDataType(DataTypePtr type, F && f) else if (factory.get("LineString")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "LineString") return f(ConverterType>()); + /// We should take the name into consideration to avoid ambiguity. + /// Because for example both MultiLineString and Polygon are resolved to Array(Tuple(Point)). + else if (factory.get("MultiLineString")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "MultiLineString") + return f(ConverterType>()); + /// For backward compatibility if we call this function not on a custom type, we will consider Array(Tuple(Point)) as type Ring. else if (factory.get("Ring")->equals(*type)) return f(ConverterType>()); diff --git a/src/Functions/polygonsIntersection.cpp b/src/Functions/polygonsIntersection.cpp index 329242e762e..43ab03f8c1f 100644 --- a/src/Functions/polygonsIntersection.cpp +++ b/src/Functions/polygonsIntersection.cpp @@ -75,6 +75,8 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be MultiLineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); diff --git a/src/Functions/polygonsSymDifference.cpp b/src/Functions/polygonsSymDifference.cpp index 3c219d0facb..6faec95bb7b 100644 --- a/src/Functions/polygonsSymDifference.cpp +++ b/src/Functions/polygonsSymDifference.cpp @@ -73,6 +73,8 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be MultiLineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); diff --git a/src/Functions/polygonsUnion.cpp b/src/Functions/polygonsUnion.cpp index 969eb2f78fb..5378ff636f8 100644 --- a/src/Functions/polygonsUnion.cpp +++ b/src/Functions/polygonsUnion.cpp @@ -73,6 +73,8 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be MultiLineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); diff --git a/src/Functions/polygonsWithin.cpp b/src/Functions/polygonsWithin.cpp index c63ad5ef868..dacd1c0e18f 100644 --- a/src/Functions/polygonsWithin.cpp +++ b/src/Functions/polygonsWithin.cpp @@ -77,6 +77,8 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be MultiLineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); diff --git a/src/Functions/readWkt.cpp b/src/Functions/readWkt.cpp index eb262777b0d..2010b5167e7 100644 --- a/src/Functions/readWkt.cpp +++ b/src/Functions/readWkt.cpp @@ -87,6 +87,11 @@ struct ReadWKTLineStringNameHolder static constexpr const char * name = "readWKTLineString"; }; +struct ReadWKTMultiLineStringNameHolder +{ + static constexpr const char * name = "readWKTMultiLineString"; +}; + struct ReadWKTRingNameHolder { static constexpr const char * name = "readWKTRing"; @@ -131,6 +136,31 @@ Parses a Well-Known Text (WKT) representation of a LineString geometry and retur }, .categories{"Unique identifiers"} }); + factory.registerFunction, ReadWKTMultiLineStringNameHolder>>(FunctionDocumentation + { + .description=R"( +Parses a Well-Known Text (WKT) representation of a MultiLineString geometry and returns it in the internal ClickHouse format. +)", + .syntax = "readWKTMultiLineString(wkt_string)", + .arguments{ + {"wkt_string", "The input WKT string representing a MultiLineString geometry."} + }, + .returned_value = "The function returns a ClickHouse internal representation of the multilinestring geometry.", + .examples{ + {"first call", "SELECT readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3), (4 4, 5 5, 6 6))');", R"( +┌─readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3), (4 4, 5 5, 6 6))')─┐ +│ [[(1,1),(2,2),(3,3)],[(4,4),(5,5),(6,6)]] │ +└──────────────────────────────────────────────────────────────────────────────┘ + + )"}, + {"second call", "SELECT toTypeName(readWKTLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'));", R"( +┌─toTypeName(readWKTLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'))─┐ +│ MultiLineString │ +└─────────────────────────────────────────────────────────────────────────┘ + )"}, + }, + .categories{"Unique identifiers"} + }); factory.registerFunction, ReadWKTRingNameHolder>>(); factory.registerFunction, ReadWKTPolygonNameHolder>>(); factory.registerFunction, ReadWKTMultiPolygonNameHolder>>(); diff --git a/tests/queries/0_stateless/03215_multilinestring_geometry.reference b/tests/queries/0_stateless/03215_multilinestring_geometry.reference new file mode 100644 index 00000000000..f4c5774018e --- /dev/null +++ b/tests/queries/0_stateless/03215_multilinestring_geometry.reference @@ -0,0 +1,17 @@ +-- { echoOn } +SELECT readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); +[[(1,1),(2,2),(3,3),(1,1)]] +SELECT toTypeName(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))')); +MultiLineString +SELECT wkt(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))')); +MULTILINESTRING((1 1,2 2,3 3,1 1)) +SELECT readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); +[[(1,1),(2,2),(3,3),(1,1)],[(1,0),(2,0),(3,0)]] +SELECT toTypeName(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))')); +MultiLineString +SELECT wkt(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))')); +MULTILINESTRING((1 1,2 2,3 3,1 1),(1 0,2 0,3 0)) +-- Native Array(Array(Tuple(Float64, Float64))) is treated as Polygon, not as MultiLineString. +WITH wkt(CAST([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'Array(Array(Tuple(Float64, Float64)))')) as x +SELECT x, toTypeName(x), readWKTPolygon(x) as y, toTypeName(y); +POLYGON((1 1,2 2,3 3,1 1)) String [[(1,1),(2,2),(3,3),(1,1)]] Polygon diff --git a/tests/queries/0_stateless/03215_multilinestring_geometry.sql b/tests/queries/0_stateless/03215_multilinestring_geometry.sql new file mode 100644 index 00000000000..71344920c52 --- /dev/null +++ b/tests/queries/0_stateless/03215_multilinestring_geometry.sql @@ -0,0 +1,12 @@ +-- { echoOn } +SELECT readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); +SELECT toTypeName(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))')); +SELECT wkt(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))')); + +SELECT readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); +SELECT toTypeName(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))')); +SELECT wkt(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))')); + +-- Native Array(Array(Tuple(Float64, Float64))) is treated as Polygon, not as MultiLineString. +WITH wkt(CAST([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'Array(Array(Tuple(Float64, Float64)))')) as x +SELECT x, toTypeName(x), readWKTPolygon(x) as y, toTypeName(y); diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index b21ae0764c6..3d7e77f213d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,4 +1,4 @@ -personal_ws-1.1 en 2942 +personal_ws-1.1 en 2942 AArch ACLs ALTERs @@ -561,6 +561,7 @@ MindsDB Mongodb Monotonicity MsgPack +MultiLineString MultiPolygon Multiline Multiqueries @@ -2361,6 +2362,7 @@ rankCorr rapidjson rawblob readWKTLineString +readWKTMultiLineString readWKTMultiPolygon readWKTPoint readWKTPolygon From c0d298781e72aaa1f34fc7fd610f5dbcaa9acf2d Mon Sep 17 00:00:00 2001 From: morning-color Date: Fri, 2 Aug 2024 10:36:15 +0800 Subject: [PATCH 256/644] Trigger test. --- .../queries/0_stateless/03174_exact_rows_before_aggregation.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql index 0afc0be4370..f9fd4ef5a7b 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql @@ -34,6 +34,7 @@ create table test (i int) engine MergeTree order by i; insert into test select arrayJoin(range(10000)); set optimize_aggregation_in_order=1; + select * from test where i < 10 group by i order by i FORMAT JSONCompact; select max(i) from test where i < 20 limit 1 FORMAT JSONCompact; From 1cb2904b447d631e14b7e3b7cc96e4be74947ef6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 2 Aug 2024 03:04:09 +0000 Subject: [PATCH 257/644] fix test --- tests/queries/0_stateless/03215_view_with_recursive.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03215_view_with_recursive.sql b/tests/queries/0_stateless/03215_view_with_recursive.sql index cac47124d51..ef7908612af 100644 --- a/tests/queries/0_stateless/03215_view_with_recursive.sql +++ b/tests/queries/0_stateless/03215_view_with_recursive.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + CREATE VIEW 03215_test_v AS WITH RECURSIVE test_table AS ( From c50ef37a03438003c21076c5700d9c1f52c1c435 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 2 Aug 2024 00:01:41 -0400 Subject: [PATCH 258/644] Fix inconsistent formatting for `GRANT CURRENT GRANTS` --- src/Parsers/Access/ASTGrantQuery.cpp | 21 +++------------------ 1 file changed, 3 insertions(+), 18 deletions(-) diff --git a/src/Parsers/Access/ASTGrantQuery.cpp b/src/Parsers/Access/ASTGrantQuery.cpp index f60fa7e4a23..eac88c75513 100644 --- a/src/Parsers/Access/ASTGrantQuery.cpp +++ b/src/Parsers/Access/ASTGrantQuery.cpp @@ -97,24 +97,9 @@ namespace void formatCurrentGrantsElements(const AccessRightsElements & elements, const IAST::FormatSettings & settings) { - for (size_t i = 0; i != elements.size(); ++i) - { - const auto & element = elements[i]; - - bool next_element_on_same_db_and_table = false; - if (i != elements.size() - 1) - { - const auto & next_element = elements[i + 1]; - if (element.sameDatabaseAndTableAndParameter(next_element)) - next_element_on_same_db_and_table = true; - } - - if (!next_element_on_same_db_and_table) - { - settings.ostr << " "; - formatONClause(element, settings); - } - } + settings.ostr << "("; + formatElementsWithoutOptions(elements, settings); + settings.ostr << ")"; } } From 0ec292a65f190f69c24420d5ca85d5658bffba0a Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 2 Aug 2024 00:32:01 -0400 Subject: [PATCH 259/644] Revert "Revert "FuzzQuery table function"" This reverts commit ff44b206 --- .../table-functions/fuzzQuery.md | 36 ++++ programs/client/Client.h | 5 +- src/{Client => Common}/QueryFuzzer.cpp | 50 ++++-- src/{Client => Common}/QueryFuzzer.h | 35 ++-- src/Storages/StorageFuzzQuery.cpp | 169 ++++++++++++++++++ src/Storages/StorageFuzzQuery.h | 88 +++++++++ src/Storages/registerStorages.cpp | 2 + src/TableFunctions/TableFunctionFuzzQuery.cpp | 54 ++++++ src/TableFunctions/TableFunctionFuzzQuery.h | 42 +++++ src/TableFunctions/registerTableFunctions.cpp | 1 + src/TableFunctions/registerTableFunctions.h | 1 + .../03031_table_function_fuzzquery.reference | 2 + .../03031_table_function_fuzzquery.sql | 18 ++ 13 files changed, 473 insertions(+), 30 deletions(-) create mode 100644 docs/en/sql-reference/table-functions/fuzzQuery.md rename src/{Client => Common}/QueryFuzzer.cpp (97%) rename src/{Client => Common}/QueryFuzzer.h (91%) create mode 100644 src/Storages/StorageFuzzQuery.cpp create mode 100644 src/Storages/StorageFuzzQuery.h create mode 100644 src/TableFunctions/TableFunctionFuzzQuery.cpp create mode 100644 src/TableFunctions/TableFunctionFuzzQuery.h create mode 100644 tests/queries/0_stateless/03031_table_function_fuzzquery.reference create mode 100644 tests/queries/0_stateless/03031_table_function_fuzzquery.sql diff --git a/docs/en/sql-reference/table-functions/fuzzQuery.md b/docs/en/sql-reference/table-functions/fuzzQuery.md new file mode 100644 index 00000000000..e15f8a40156 --- /dev/null +++ b/docs/en/sql-reference/table-functions/fuzzQuery.md @@ -0,0 +1,36 @@ +--- +slug: /en/sql-reference/table-functions/fuzzQuery +sidebar_position: 75 +sidebar_label: fuzzQuery +--- + +# fuzzQuery + +Perturbs the given query string with random variations. + +``` sql +fuzzQuery(query[, max_query_length[, random_seed]]) +``` + +**Arguments** + +- `query` (String) - The source query to perform the fuzzing on. +- `max_query_length` (UInt64) - A maximum length the query can get during the fuzzing process. +- `random_seed` (UInt64) - A random seed for producing stable results. + +**Returned Value** + +A table object with a single column containing perturbed query strings. + +## Usage Example + +``` sql +SELECT * FROM fuzzQuery('SELECT materialize(\'a\' AS key) GROUP BY key') LIMIT 2; +``` + +``` + ┌─query──────────────────────────────────────────────────────────┐ +1. │ SELECT 'a' AS key GROUP BY key │ +2. │ EXPLAIN PIPELINE compact = true SELECT 'a' AS key GROUP BY key │ + └────────────────────────────────────────────────────────────────┘ +``` diff --git a/programs/client/Client.h b/programs/client/Client.h index 7fdf77031ab..07a8e293b1a 100644 --- a/programs/client/Client.h +++ b/programs/client/Client.h @@ -11,7 +11,10 @@ class Client : public ClientApplicationBase public: using Arguments = ClientApplicationBase::Arguments; - Client() = default; + Client() + { + fuzzer = QueryFuzzer(randomSeed(), &std::cout, &std::cerr); + } void initialize(Poco::Util::Application & self) override; diff --git a/src/Client/QueryFuzzer.cpp b/src/Common/QueryFuzzer.cpp similarity index 97% rename from src/Client/QueryFuzzer.cpp rename to src/Common/QueryFuzzer.cpp index f5b700ea529..161c38f20e0 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Common/QueryFuzzer.cpp @@ -68,22 +68,21 @@ Field QueryFuzzer::getRandomField(int type) { case 0: { - return bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) - / sizeof(*bad_int64_values))]; + return bad_int64_values[fuzz_rand() % std::size(bad_int64_values)]; } case 1: { static constexpr double values[] = {NAN, INFINITY, -INFINITY, 0., -0., 0.0001, 0.5, 0.9999, 1., 1.0001, 2., 10.0001, 100.0001, 1000.0001, 1e10, 1e20, - FLT_MIN, FLT_MIN + FLT_EPSILON, FLT_MAX, FLT_MAX + FLT_EPSILON}; return values[fuzz_rand() % (sizeof(values) / sizeof(*values))]; + FLT_MIN, FLT_MIN + FLT_EPSILON, FLT_MAX, FLT_MAX + FLT_EPSILON}; return values[fuzz_rand() % std::size(values)]; } case 2: { static constexpr UInt64 scales[] = {0, 1, 2, 10}; return DecimalField( - bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) / sizeof(*bad_int64_values))], - static_cast(scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))]) + bad_int64_values[fuzz_rand() % std::size(bad_int64_values)], + static_cast(scales[fuzz_rand() % std::size(scales)]) ); } default: @@ -165,7 +164,8 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.erase(arr.begin() + pos); - std::cerr << "erased\n"; + if (debug_stream) + *debug_stream << "erased\n"; } if (fuzz_rand() % 5 == 0) @@ -174,12 +174,14 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.insert(arr.begin() + pos, fuzzField(arr[pos])); - std::cerr << fmt::format("inserted (pos {})\n", pos); + if (debug_stream) + *debug_stream << fmt::format("inserted (pos {})\n", pos); } else { arr.insert(arr.begin(), getRandomField(0)); - std::cerr << "inserted (0)\n"; + if (debug_stream) + *debug_stream << "inserted (0)\n"; } } @@ -197,7 +199,9 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.erase(arr.begin() + pos); - std::cerr << "erased\n"; + + if (debug_stream) + *debug_stream << "erased\n"; } if (fuzz_rand() % 5 == 0) @@ -206,12 +210,16 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.insert(arr.begin() + pos, fuzzField(arr[pos])); - std::cerr << fmt::format("inserted (pos {})\n", pos); + + if (debug_stream) + *debug_stream << fmt::format("inserted (pos {})\n", pos); } else { arr.insert(arr.begin(), getRandomField(0)); - std::cerr << "inserted (0)\n"; + + if (debug_stream) + *debug_stream << "inserted (0)\n"; } } @@ -344,7 +352,8 @@ void QueryFuzzer::fuzzOrderByList(IAST * ast) } else { - std::cerr << "No random column.\n"; + if (debug_stream) + *debug_stream << "No random column.\n"; } } @@ -378,7 +387,8 @@ void QueryFuzzer::fuzzColumnLikeExpressionList(IAST * ast) if (col) impl->children.insert(pos, col); else - std::cerr << "No random column.\n"; + if (debug_stream) + *debug_stream << "No random column.\n"; } // We don't have to recurse here to fuzz the children, this is handled by @@ -1361,11 +1371,15 @@ void QueryFuzzer::fuzzMain(ASTPtr & ast) collectFuzzInfoMain(ast); fuzz(ast); - std::cout << std::endl; - WriteBufferFromOStream ast_buf(std::cout, 4096); - formatAST(*ast, ast_buf, false /*highlight*/); - ast_buf.finalize(); - std::cout << std::endl << std::endl; + if (out_stream) + { + *out_stream << std::endl; + + WriteBufferFromOStream ast_buf(*out_stream, 4096); + formatAST(*ast, ast_buf, false /*highlight*/); + ast_buf.finalize(); + *out_stream << std::endl << std::endl; + } } } diff --git a/src/Client/QueryFuzzer.h b/src/Common/QueryFuzzer.h similarity index 91% rename from src/Client/QueryFuzzer.h rename to src/Common/QueryFuzzer.h index 6165e589cae..35d088809f2 100644 --- a/src/Client/QueryFuzzer.h +++ b/src/Common/QueryFuzzer.h @@ -35,9 +35,31 @@ struct ASTWindowDefinition; * queries, so you want to feed it a lot of queries to get some interesting mix * of them. Normally we feed SQL regression tests to it. */ -struct QueryFuzzer +class QueryFuzzer { - pcg64 fuzz_rand{randomSeed()}; +public: + explicit QueryFuzzer(pcg64 fuzz_rand_ = randomSeed(), std::ostream * out_stream_ = nullptr, std::ostream * debug_stream_ = nullptr) + : fuzz_rand(fuzz_rand_) + , out_stream(out_stream_) + , debug_stream(debug_stream_) + { + } + + // This is the only function you have to call -- it will modify the passed + // ASTPtr to point to new AST with some random changes. + void fuzzMain(ASTPtr & ast); + + ASTs getInsertQueriesForFuzzedTables(const String & full_query); + ASTs getDropQueriesForFuzzedTables(const ASTDropQuery & drop_query); + void notifyQueryFailed(ASTPtr ast); + + static bool isSuitableForFuzzing(const ASTCreateQuery & create); + +private: + pcg64 fuzz_rand; + + std::ostream * out_stream = nullptr; + std::ostream * debug_stream = nullptr; // We add elements to expression lists with fixed probability. Some elements // are so large, that the expected number of elements we add to them is @@ -66,10 +88,6 @@ struct QueryFuzzer std::unordered_map index_of_fuzzed_table; std::set created_tables_hashes; - // This is the only function you have to call -- it will modify the passed - // ASTPtr to point to new AST with some random changes. - void fuzzMain(ASTPtr & ast); - // Various helper functions follow, normally you shouldn't have to call them. Field getRandomField(int type); Field fuzzField(Field field); @@ -77,9 +95,6 @@ struct QueryFuzzer ASTPtr getRandomExpressionList(); DataTypePtr fuzzDataType(DataTypePtr type); DataTypePtr getRandomType(); - ASTs getInsertQueriesForFuzzedTables(const String & full_query); - ASTs getDropQueriesForFuzzedTables(const ASTDropQuery & drop_query); - void notifyQueryFailed(ASTPtr ast); void replaceWithColumnLike(ASTPtr & ast); void replaceWithTableLike(ASTPtr & ast); void fuzzOrderByElement(ASTOrderByElement * elem); @@ -102,8 +117,6 @@ struct QueryFuzzer void addTableLike(ASTPtr ast); void addColumnLike(ASTPtr ast); void collectFuzzInfoRecurse(ASTPtr ast); - - static bool isSuitableForFuzzing(const ASTCreateQuery & create); }; } diff --git a/src/Storages/StorageFuzzQuery.cpp b/src/Storages/StorageFuzzQuery.cpp new file mode 100644 index 00000000000..6e8f425f8dc --- /dev/null +++ b/src/Storages/StorageFuzzQuery.cpp @@ -0,0 +1,169 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +ColumnPtr FuzzQuerySource::createColumn() +{ + auto column = ColumnString::create(); + ColumnString::Chars & data_to = column->getChars(); + ColumnString::Offsets & offsets_to = column->getOffsets(); + + offsets_to.resize(block_size); + IColumn::Offset offset = 0; + + auto fuzz_base = query; + size_t row_num = 0; + + while (row_num < block_size) + { + ASTPtr new_query = fuzz_base->clone(); + + auto base_before_fuzz = fuzz_base->formatForErrorMessage(); + fuzzer.fuzzMain(new_query); + auto fuzzed_text = new_query->formatForErrorMessage(); + + if (base_before_fuzz == fuzzed_text) + continue; + + /// AST is too long, will start from the original query. + if (config.max_query_length > 500) + { + fuzz_base = query; + continue; + } + + IColumn::Offset next_offset = offset + fuzzed_text.size() + 1; + data_to.resize(next_offset); + + std::copy(fuzzed_text.begin(), fuzzed_text.end(), &data_to[offset]); + + data_to[offset + fuzzed_text.size()] = 0; + offsets_to[row_num] = next_offset; + + offset = next_offset; + fuzz_base = new_query; + ++row_num; + } + + return column; +} + +StorageFuzzQuery::StorageFuzzQuery( + const StorageID & table_id_, const ColumnsDescription & columns_, const String & comment_, const Configuration & config_) + : IStorage(table_id_), config(config_) +{ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setComment(comment_); + setInMemoryMetadata(storage_metadata); +} + +Pipe StorageFuzzQuery::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & /*query_info*/, + ContextPtr /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + size_t num_streams) +{ + storage_snapshot->check(column_names); + + Pipes pipes; + pipes.reserve(num_streams); + + const ColumnsDescription & our_columns = storage_snapshot->metadata->getColumns(); + Block block_header; + for (const auto & name : column_names) + { + const auto & name_type = our_columns.get(name); + MutableColumnPtr column = name_type.type->createColumn(); + block_header.insert({std::move(column), name_type.type, name_type.name}); + } + + const char * begin = config.query.data(); + const char * end = begin + config.query.size(); + + ParserQuery parser(end, false); + auto query = parseQuery(parser, begin, end, "", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); + + for (UInt64 i = 0; i < num_streams; ++i) + pipes.emplace_back(std::make_shared(max_block_size, block_header, config, query)); + + return Pipe::unitePipes(std::move(pipes)); +} + +StorageFuzzQuery::Configuration StorageFuzzQuery::getConfiguration(ASTs & engine_args, ContextPtr local_context) +{ + StorageFuzzQuery::Configuration configuration{}; + + // Supported signatures: + // + // FuzzQuery(query) + // FuzzQuery(query, max_query_length) + // FuzzQuery(query, max_query_length, random_seed) + if (engine_args.empty() || engine_args.size() > 3) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "FuzzQuery requires 1 to 3 arguments: query, max_query_length, random_seed"); + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); + + auto first_arg = checkAndGetLiteralArgument(engine_args[0], "query"); + configuration.query = std::move(first_arg); + + if (engine_args.size() >= 2) + { + const auto & literal = engine_args[1]->as(); + if (!literal.value.isNull()) + configuration.max_query_length = checkAndGetLiteralArgument(literal, "max_query_length"); + } + + if (engine_args.size() == 3) + { + const auto & literal = engine_args[2]->as(); + if (!literal.value.isNull()) + configuration.random_seed = checkAndGetLiteralArgument(literal, "random_seed"); + } + + return configuration; +} + +void registerStorageFuzzQuery(StorageFactory & factory) +{ + factory.registerStorage( + "FuzzQuery", + [](const StorageFactory::Arguments & args) -> std::shared_ptr + { + ASTs & engine_args = args.engine_args; + + if (engine_args.empty()) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage FuzzQuery must have arguments."); + + StorageFuzzQuery::Configuration configuration = StorageFuzzQuery::getConfiguration(engine_args, args.getLocalContext()); + + for (const auto& col : args.columns) + if (col.type->getTypeId() != TypeIndex::String) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "'StorageFuzzQuery' supports only columns of String type, got {}.", col.type->getName()); + + return std::make_shared(args.table_id, args.columns, args.comment, configuration); + }); +} + +} diff --git a/src/Storages/StorageFuzzQuery.h b/src/Storages/StorageFuzzQuery.h new file mode 100644 index 00000000000..125ef960e74 --- /dev/null +++ b/src/Storages/StorageFuzzQuery.h @@ -0,0 +1,88 @@ +#pragma once + +#include +#include +#include +#include + +#include "config.h" + +namespace DB +{ + +class NamedCollection; + +class StorageFuzzQuery final : public IStorage +{ +public: + struct Configuration : public StatelessTableEngineConfiguration + { + String query; + UInt64 max_query_length = 500; + UInt64 random_seed = randomSeed(); + }; + + StorageFuzzQuery( + const StorageID & table_id_, const ColumnsDescription & columns_, const String & comment_, const Configuration & config_); + + std::string getName() const override { return "FuzzQuery"; } + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + static StorageFuzzQuery::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context); + +private: + const Configuration config; +}; + + +class FuzzQuerySource : public ISource +{ +public: + FuzzQuerySource( + UInt64 block_size_, Block block_header_, const StorageFuzzQuery::Configuration & config_, ASTPtr query_) + : ISource(block_header_) + , block_size(block_size_) + , block_header(std::move(block_header_)) + , config(config_) + , query(query_) + , fuzzer(config_.random_seed) + { + } + + String getName() const override { return "FuzzQuery"; } + +protected: + Chunk generate() override + { + Columns columns; + columns.reserve(block_header.columns()); + for (const auto & col : block_header) + { + chassert(col.type->getTypeId() == TypeIndex::String); + columns.emplace_back(createColumn()); + } + + return {std::move(columns), block_size}; + } + +private: + ColumnPtr createColumn(); + + UInt64 block_size; + Block block_header; + + StorageFuzzQuery::Configuration config; + ASTPtr query; + + QueryFuzzer fuzzer; +}; + +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 8f33314397c..adc1074b1fe 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -26,6 +26,7 @@ void registerStorageGenerateRandom(StorageFactory & factory); void registerStorageExecutable(StorageFactory & factory); void registerStorageWindowView(StorageFactory & factory); void registerStorageLoop(StorageFactory & factory); +void registerStorageFuzzQuery(StorageFactory & factory); #if USE_RAPIDJSON || USE_SIMDJSON void registerStorageFuzzJSON(StorageFactory & factory); #endif @@ -126,6 +127,7 @@ void registerStorages() registerStorageExecutable(factory); registerStorageWindowView(factory); registerStorageLoop(factory); + registerStorageFuzzQuery(factory); #if USE_RAPIDJSON || USE_SIMDJSON registerStorageFuzzJSON(factory); #endif diff --git a/src/TableFunctions/TableFunctionFuzzQuery.cpp b/src/TableFunctions/TableFunctionFuzzQuery.cpp new file mode 100644 index 00000000000..224f6666556 --- /dev/null +++ b/src/TableFunctions/TableFunctionFuzzQuery.cpp @@ -0,0 +1,54 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +void TableFunctionFuzzQuery::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + ASTs & args_func = ast_function->children; + + if (args_func.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments", getName()); + + auto args = args_func.at(0)->children; + configuration = StorageFuzzQuery::getConfiguration(args, context); +} + +StoragePtr TableFunctionFuzzQuery::executeImpl( + const ASTPtr & /*ast_function*/, + ContextPtr context, + const std::string & table_name, + ColumnsDescription /*cached_columns*/, + bool is_insert_query) const +{ + ColumnsDescription columns = getActualTableStructure(context, is_insert_query); + auto res = std::make_shared( + StorageID(getDatabaseName(), table_name), + columns, + /* comment */ String{}, + configuration); + res->startup(); + return res; +} + +void registerTableFunctionFuzzQuery(TableFunctionFactory & factory) +{ + factory.registerFunction( + {.documentation + = {.description = "Perturbs a query string with random variations.", + .returned_value = "A table object with a single column containing perturbed query strings."}, + .allow_readonly = true}); +} + +} diff --git a/src/TableFunctions/TableFunctionFuzzQuery.h b/src/TableFunctions/TableFunctionFuzzQuery.h new file mode 100644 index 00000000000..22d10341c4d --- /dev/null +++ b/src/TableFunctions/TableFunctionFuzzQuery.h @@ -0,0 +1,42 @@ +#pragma once + +#include + +#include +#include +#include + +#include "config.h" + +namespace DB +{ + +class TableFunctionFuzzQuery : public ITableFunction +{ +public: + static constexpr auto name = "fuzzQuery"; + std::string getName() const override { return name; } + + void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; + + ColumnsDescription getActualTableStructure(ContextPtr /* context */, bool /* is_insert_query */) const override + { + return ColumnsDescription{{"query", std::make_shared()}}; + } + +private: + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const override; + + const char * getStorageTypeName() const override { return "fuzzQuery"; } + + String source; + std::optional random_seed; + StorageFuzzQuery::Configuration configuration; +}; + +} diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index ca4913898f9..a6c90872f12 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -26,6 +26,7 @@ void registerTableFunctions() registerTableFunctionMongoDB(factory); registerTableFunctionRedis(factory); registerTableFunctionMergeTreeIndex(factory); + registerTableFunctionFuzzQuery(factory); #if USE_RAPIDJSON || USE_SIMDJSON registerTableFunctionFuzzJSON(factory); #endif diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index efde4d6dcdc..2a8864a9bfd 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -23,6 +23,7 @@ void registerTableFunctionGenerate(TableFunctionFactory & factory); void registerTableFunctionMongoDB(TableFunctionFactory & factory); void registerTableFunctionRedis(TableFunctionFactory & factory); void registerTableFunctionMergeTreeIndex(TableFunctionFactory & factory); +void registerTableFunctionFuzzQuery(TableFunctionFactory & factory); #if USE_RAPIDJSON || USE_SIMDJSON void registerTableFunctionFuzzJSON(TableFunctionFactory & factory); #endif diff --git a/tests/queries/0_stateless/03031_table_function_fuzzquery.reference b/tests/queries/0_stateless/03031_table_function_fuzzquery.reference new file mode 100644 index 00000000000..202e4557a33 --- /dev/null +++ b/tests/queries/0_stateless/03031_table_function_fuzzquery.reference @@ -0,0 +1,2 @@ +query +String diff --git a/tests/queries/0_stateless/03031_table_function_fuzzquery.sql b/tests/queries/0_stateless/03031_table_function_fuzzquery.sql new file mode 100644 index 00000000000..b26096f7f0e --- /dev/null +++ b/tests/queries/0_stateless/03031_table_function_fuzzquery.sql @@ -0,0 +1,18 @@ + +SELECT * FROM fuzzQuery('SELECT 1', 500, 8956) LIMIT 0 FORMAT TSVWithNamesAndTypes; + +SELECT * FROM fuzzQuery('SELECT * +FROM ( + SELECT + ([toString(number % 2)] :: Array(LowCardinality(String))) AS item_id, + count() + FROM numbers(3) + GROUP BY item_id WITH TOTALS +) AS l FULL JOIN ( + SELECT + ([toString((number % 2) * 2)] :: Array(String)) AS item_id + FROM numbers(3) +) AS r +ON l.item_id = r.item_id +ORDER BY 1,2,3; +', 500, 8956) LIMIT 10 FORMAT NULL; From 51b39a6c745d61cb2e6feb39659ddb3cac57ad03 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 1 Aug 2024 18:24:45 +0100 Subject: [PATCH 260/644] some more --- tests/integration/test_executable_dictionary/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_executable_dictionary/test.py b/tests/integration/test_executable_dictionary/test.py index a1de429a235..2a6af75e751 100644 --- a/tests/integration/test_executable_dictionary/test.py +++ b/tests/integration/test_executable_dictionary/test.py @@ -235,6 +235,7 @@ def test_executable_implicit_input_signalled_python(started_cluster): ) +@pytest.mark.repeat(50) def test_executable_input_slow_python(started_cluster): skip_test_msan(node) assert node.query_and_get_error( From 77a2eb61ef965a6460bbdb74447aa3871cb1d0c7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Aug 2024 17:43:33 +0000 Subject: [PATCH 261/644] Update test. --- ...61_lightweight_delete_projection.reference | 70 ++++++++++++++++++ .../03161_lightweight_delete_projection.sql | 74 ++++++++++--------- 2 files changed, 111 insertions(+), 33 deletions(-) diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index 960fa1dcc33..eef0c5a41b5 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -1,20 +1,90 @@ compact part testing throw default mode +-- { echoOn } + +ALTER TABLE users_compact MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; +DELETE FROM users_compact WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } +SELECT 'testing drop mode'; testing drop mode +ALTER TABLE users_compact MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; +DELETE FROM users_compact WHERE uid = 1231; +SELECT * FROM users_compact ORDER BY uid; +SYSTEM FLUSH LOGS; +-- all_1_1_0_2 +SELECT + name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); all_1_1_0_2 +-- expecting no projection +SELECT + name, parent_name +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); +SELECT 'testing rebuild mode'; testing rebuild mode +INSERT INTO users_compact VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); +ALTER TABLE users_compact MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; +DELETE FROM users_compact WHERE uid = 6666; +SELECT * FROM users_compact ORDER BY uid; 8888 Alice 50 +SYSTEM FLUSH LOGS; +-- all_1_1_0_4, all_3_3_0_4 +SELECT + name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); all_1_1_0_4 all_3_3_0_4 +-- expecting projection p1, p2 +SELECT + name, parent_name +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); p1 all_3_3_0_4 p2 all_3_3_0_4 wide part testing throw default mode +-- { echoOn } + +ALTER TABLE users_wide MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; +DELETE FROM users_wide WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } +SELECT 'testing drop mode'; testing drop mode +ALTER TABLE users_wide MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; +DELETE FROM users_wide WHERE uid = 1231; +SELECT * FROM users_wide ORDER BY uid; +SYSTEM FLUSH LOGS; +-- all_1_1_0_2 +SELECT + name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); all_1_1_0_2 +-- expecting no projection +SELECT + name, parent_name +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); +SELECT 'testing rebuild mode'; testing rebuild mode +INSERT INTO users_wide VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); +ALTER TABLE users_wide MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; +DELETE FROM users_wide WHERE uid = 6666; +SELECT * FROM users_wide ORDER BY uid; 8888 Alice 50 +SYSTEM FLUSH LOGS; +-- all_1_1_0_4, all_3_3_0_4 +SELECT + name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); all_1_1_0_4 all_3_3_0_4 +-- expecting projection p1, p2 +SELECT + name, parent_name +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); p1 all_3_3_0_4 p2 all_3_3_0_4 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 02b880d620a..28e5612a529 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -1,12 +1,12 @@ SET lightweight_deletes_sync = 2, alter_sync = 2; -DROP TABLE IF EXISTS users; +DROP TABLE IF EXISTS users_compact; SELECT 'compact part'; -CREATE TABLE users ( +CREATE TABLE users_compact ( uid Int16, name String, age Int16, @@ -15,20 +15,22 @@ CREATE TABLE users ( ) ENGINE = MergeTree order by uid SETTINGS min_bytes_for_wide_part = 10485760; -INSERT INTO users VALUES (1231, 'John', 33); +INSERT INTO users_compact VALUES (1231, 'John', 33); SELECT 'testing throw default mode'; -ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; +-- { echoOn } -DELETE FROM users WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE users_compact MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; + +DELETE FROM users_compact WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } SELECT 'testing drop mode'; -ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; +ALTER TABLE users_compact MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; -DELETE FROM users WHERE uid = 1231; +DELETE FROM users_compact WHERE uid = 1231; -SELECT * FROM users ORDER BY uid; +SELECT * FROM users_compact ORDER BY uid; SYSTEM FLUSH LOGS; @@ -36,22 +38,22 @@ SYSTEM FLUSH LOGS; SELECT name FROM system.parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); -- expecting no projection SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); SELECT 'testing rebuild mode'; -INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); +INSERT INTO users_compact VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); -ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; +ALTER TABLE users_compact MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; -DELETE FROM users WHERE uid = 6666; +DELETE FROM users_compact WHERE uid = 6666; -SELECT * FROM users ORDER BY uid; +SELECT * FROM users_compact ORDER BY uid; SYSTEM FLUSH LOGS; @@ -59,19 +61,21 @@ SYSTEM FLUSH LOGS; SELECT name FROM system.parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); -- expecting projection p1, p2 SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); -DROP TABLE users; +-- { echoOff } + +DROP TABLE users_compact; SELECT 'wide part'; -CREATE TABLE users ( +CREATE TABLE users_wide ( uid Int16, name String, age Int16, @@ -80,19 +84,22 @@ CREATE TABLE users ( ) ENGINE = MergeTree order by uid SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO users VALUES (1231, 'John', 33); +INSERT INTO users_wide VALUES (1231, 'John', 33); SELECT 'testing throw default mode'; -ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; -DELETE FROM users WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } +-- { echoOn } + +ALTER TABLE users_wide MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; + +DELETE FROM users_wide WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } SELECT 'testing drop mode'; -ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; +ALTER TABLE users_wide MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; -DELETE FROM users WHERE uid = 1231; +DELETE FROM users_wide WHERE uid = 1231; -SELECT * FROM users ORDER BY uid; +SELECT * FROM users_wide ORDER BY uid; SYSTEM FLUSH LOGS; @@ -100,22 +107,22 @@ SYSTEM FLUSH LOGS; SELECT name FROM system.parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); -- expecting no projection SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); SELECT 'testing rebuild mode'; -INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); +INSERT INTO users_wide VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); -ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; +ALTER TABLE users_wide MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; -DELETE FROM users WHERE uid = 6666; +DELETE FROM users_wide WHERE uid = 6666; -SELECT * FROM users ORDER BY uid; +SELECT * FROM users_wide ORDER BY uid; SYSTEM FLUSH LOGS; @@ -123,13 +130,14 @@ SYSTEM FLUSH LOGS; SELECT name FROM system.parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); -- expecting projection p1, p2 SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); +-- { echoOff } -DROP TABLE users; \ No newline at end of file +DROP TABLE users_wide; From 08bde9cb44dce6fdf069527852faa8ec29a71b10 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 2 Aug 2024 18:28:33 -0400 Subject: [PATCH 262/644] fix conflict --- src/Client/ClientBase.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 1a23b6b1363..45251aea28a 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -2,7 +2,7 @@ #include -#include +#include #include #include #include From 51918dc080c9fa4b128a151cfbd0d28b294c56d3 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 3 Aug 2024 00:24:00 +0100 Subject: [PATCH 263/644] impl --- src/Common/ShellCommand.cpp | 9 ++++- src/Processors/Sources/ShellCommandSource.cpp | 39 +++++++++++++------ 2 files changed, 35 insertions(+), 13 deletions(-) diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index 98a21b43d76..79b0d667863 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -237,7 +237,14 @@ std::unique_ptr ShellCommand::executeImpl( res->write_fds.emplace(fd, fds.fds_rw[1]); } - LOG_TRACE(getLogger(), "Started shell command '{}' with pid {}", filename, pid); + LOG_TRACE( + getLogger(), + "Started shell command '{}' with pid {} and file descriptors: read {}, write {}", + filename, + pid, + res->out.getFD(), + res->err.getFD()); + return res; } diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 1659287c227..923bdfad8f8 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -75,6 +75,15 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond while (true) { Stopwatch watch; + +#if defined(DEBUG_OR_SANITIZER_BUILD) + auto describe_fd = [](const auto & pollfd) { return fmt::format("(fd={}, flags={})", pollfd.fd, fcntl(pollfd.fd, F_GETFL)); }; + LOG_TRACE( + getLogger("TimeoutReadBufferFromFileDescriptor"), + "Polling descriptors: {}", + fmt::join(std::span(pfds, pfds + num) | std::views::transform(describe_fd), ", ")); +#endif + res = poll(pfds, static_cast(num), static_cast(timeout_milliseconds)); if (res < 0) @@ -84,7 +93,16 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond const auto elapsed = watch.elapsedMilliseconds(); if (timeout_milliseconds <= elapsed) + { +#if defined(DEBUG_OR_SANITIZER_BUILD) + LOG_TRACE( + getLogger("TimeoutReadBufferFromFileDescriptor"), + "Timeout exceeded: elapsed={}, timeout={}", + elapsed, + timeout_milliseconds); +#endif break; + } timeout_milliseconds -= elapsed; } else @@ -93,6 +111,15 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond } } +#if defined(DEBUG_OR_SANITIZER_BUILD) + auto describe_fd = [](const auto & pollfd) { return fmt::format("(fd={}, flags={})", pollfd.fd, fcntl(pollfd.fd, F_GETFL)); }; + LOG_TRACE( + getLogger("TimeoutReadBufferFromFileDescriptor"), + "Poll for descriptors: {} returned {}", + fmt::join(std::span(pfds, pfds + num) | std::views::transform(describe_fd), ", "), + res); +#endif + return res; } @@ -139,15 +166,9 @@ public: while (!bytes_read) { - LOG_TRACE( - getLogger("TimeoutReadBufferFromFileDescriptor"), - "Starting polling on descriptors ({}) with timeout {} ms", - fmt::join(std::span(pfds, pfds + num_pfds) | std::views::transform([](const auto & pollfd) { return pollfd.fd; }), ", "), - timeout_milliseconds); pfds[0].revents = 0; pfds[1].revents = 0; size_t num_events = pollWithTimeout(pfds, num_pfds, timeout_milliseconds); - LOG_TRACE(getLogger("TimeoutReadBufferFromFileDescriptor"), "Poll returned with num_events={}", num_events); if (0 == num_events) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Pipe read timeout exceeded {} milliseconds", timeout_milliseconds); @@ -208,12 +229,6 @@ public: return true; } - void reset() const - { - makeFdBlocking(stdout_fd); - makeFdBlocking(stderr_fd); - } - ~TimeoutReadBufferFromFileDescriptor() override { tryMakeFdBlocking(stdout_fd); From 2605bb36b66ccfb4621244a28475a242778b6cc4 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 3 Aug 2024 01:42:11 +0000 Subject: [PATCH 264/644] fix conflict --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8f73e10c44f..107a8e451c5 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -92,7 +92,7 @@ static std::initializer_list Date: Thu, 30 May 2024 17:15:29 +0200 Subject: [PATCH 265/644] Support constructing SnappyWriteBuffer from a reference to other write buffer. --- src/IO/SnappyWriteBuffer.cpp | 8 +++++++- src/IO/SnappyWriteBuffer.h | 10 +++++++++- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/src/IO/SnappyWriteBuffer.cpp b/src/IO/SnappyWriteBuffer.cpp index ca40d0656d1..0e02b48e1e0 100644 --- a/src/IO/SnappyWriteBuffer.cpp +++ b/src/IO/SnappyWriteBuffer.cpp @@ -16,7 +16,13 @@ namespace ErrorCodes } SnappyWriteBuffer::SnappyWriteBuffer(std::unique_ptr out_, size_t buf_size, char * existing_memory, size_t alignment) - : BufferWithOwnMemory(buf_size, existing_memory, alignment), out(std::move(out_)) + : SnappyWriteBuffer(*out_, buf_size, existing_memory, alignment) +{ + out_holder = std::move(out_); +} + +SnappyWriteBuffer::SnappyWriteBuffer(WriteBuffer & out_, size_t buf_size, char * existing_memory, size_t alignment) + : BufferWithOwnMemory(buf_size, existing_memory, alignment), out(&out_) { } diff --git a/src/IO/SnappyWriteBuffer.h b/src/IO/SnappyWriteBuffer.h index 2ff86fb64ef..b7a084d0f80 100644 --- a/src/IO/SnappyWriteBuffer.h +++ b/src/IO/SnappyWriteBuffer.h @@ -18,6 +18,12 @@ public: char * existing_memory = nullptr, size_t alignment = 0); + explicit SnappyWriteBuffer( + WriteBuffer & out_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0); + ~SnappyWriteBuffer() override; void finalizeImpl() override { finish(); } @@ -28,7 +34,9 @@ private: void finishImpl(); void finish(); - std::unique_ptr out; + WriteBuffer * out; + std::unique_ptr out_holder; + bool finished = false; String uncompress_buffer; From 2a6f498b7751df0e37f281136d524f542f7910c2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 12 Jul 2024 19:18:37 +0200 Subject: [PATCH 266/644] Add view targets Data, Tags, Metrics. --- src/Parsers/ASTCreateQuery.cpp | 7 +++++++ src/Parsers/ASTCreateQuery.h | 1 + src/Parsers/ASTViewTargets.cpp | 12 ++++++++++++ src/Parsers/ASTViewTargets.h | 11 ++++++++++- src/Parsers/CommonParsers.h | 6 ++++++ src/Parsers/CreateQueryUUIDs.cpp | 7 +++++++ src/Parsers/ParserCreateQuery.cpp | 9 +++++++++ 7 files changed, 52 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index f0f782c0a63..359e93ab269 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -483,6 +483,13 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat if (auto to_storage = getTargetInnerEngine(ViewTarget::To)) to_storage->formatImpl(settings, state, frame); + if (targets) + { + targets->formatTarget(ViewTarget::Data, settings, state, frame); + targets->formatTarget(ViewTarget::Tags, settings, state, frame); + targets->formatTarget(ViewTarget::Metrics, settings, state, frame); + } + if (dictionary) dictionary->formatImpl(settings, state, frame); diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index a95010aea31..6be0fa78903 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -97,6 +97,7 @@ public: bool is_materialized_view{false}; bool is_live_view{false}; bool is_window_view{false}; + bool is_time_series_table{false}; /// CREATE TABLE ... ENGINE=TimeSeries() ... bool is_populate{false}; bool is_create_empty{false}; /// CREATE TABLE ... EMPTY AS SELECT ... bool replace_view{false}; /// CREATE OR REPLACE VIEW diff --git a/src/Parsers/ASTViewTargets.cpp b/src/Parsers/ASTViewTargets.cpp index 8ee98e704df..ffd746cc38a 100644 --- a/src/Parsers/ASTViewTargets.cpp +++ b/src/Parsers/ASTViewTargets.cpp @@ -21,6 +21,9 @@ std::string_view toString(ViewTarget::Kind kind) { case ViewTarget::To: return "to"; case ViewTarget::Inner: return "inner"; + case ViewTarget::Data: return "data"; + case ViewTarget::Tags: return "tags"; + case ViewTarget::Metrics: return "metrics"; } throw Exception(ErrorCodes::LOGICAL_ERROR, "{} doesn't support kind {}", __FUNCTION__, kind); } @@ -254,6 +257,9 @@ std::optional ASTViewTargets::getKeywordForTableID(ViewTarget::Kind kin { case ViewTarget::To: return Keyword::TO; /// TO mydb.mydata case ViewTarget::Inner: return std::nullopt; + case ViewTarget::Data: return Keyword::DATA; /// DATA mydb.mydata + case ViewTarget::Tags: return Keyword::TAGS; /// TAGS mydb.mytags + case ViewTarget::Metrics: return Keyword::METRICS; /// METRICS mydb.mymetrics } UNREACHABLE(); } @@ -264,6 +270,9 @@ std::optional ASTViewTargets::getKeywordForInnerStorage(ViewTarget::Kin { case ViewTarget::To: return std::nullopt; /// ENGINE = MergeTree() case ViewTarget::Inner: return Keyword::INNER; /// INNER ENGINE = MergeTree() + case ViewTarget::Data: return Keyword::DATA; /// DATA ENGINE = MergeTree() + case ViewTarget::Tags: return Keyword::TAGS; /// TAGS ENGINE = MergeTree() + case ViewTarget::Metrics: return Keyword::METRICS; /// METRICS ENGINE = MergeTree() } UNREACHABLE(); } @@ -274,6 +283,9 @@ std::optional ASTViewTargets::getKeywordForInnerUUID(ViewTarget::Kind k { case ViewTarget::To: return Keyword::TO_INNER_UUID; /// TO INNER UUID 'XXX' case ViewTarget::Inner: return std::nullopt; + case ViewTarget::Data: return Keyword::DATA_INNER_UUID; /// DATA INNER UUID 'XXX' + case ViewTarget::Tags: return Keyword::TAGS_INNER_UUID; /// TAGS INNER UUID 'XXX' + case ViewTarget::Metrics: return Keyword::METRICS_INNER_UUID; /// METRICS INNER UUID 'XXX' } UNREACHABLE(); } diff --git a/src/Parsers/ASTViewTargets.h b/src/Parsers/ASTViewTargets.h index 12182919f0e..7814dd5249c 100644 --- a/src/Parsers/ASTViewTargets.h +++ b/src/Parsers/ASTViewTargets.h @@ -9,7 +9,7 @@ namespace DB class ASTStorage; enum class Keyword : size_t; -/// Information about target tables (external or inner) of a materialized view or a window view. +/// Information about target tables (external or inner) of a materialized view or a window view or a TimeSeries table. /// See ASTViewTargets for more details. struct ViewTarget { @@ -24,6 +24,15 @@ struct ViewTarget /// If `kind == ViewTarget::Inner` then `ViewTarget` contains information about the "INNER" table of a window view: /// CREATE WINDOW VIEW db.wv_name {INNER ENGINE inner_engine} AS SELECT ... Inner, + + /// The "data" table for a TimeSeries table, contains time series. + Data, + + /// The "tags" table for a TimeSeries table, contains identifiers for each combination of a metric name and tags (labels). + Tags, + + /// The "metrics" table for a TimeSeries table, contains general information (metadata) about metrics. + Metrics, }; Kind kind = To; diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 0ae9ee4833c..34df4b135bb 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -116,6 +116,8 @@ namespace DB MR_MACROS(CURRENT_TRANSACTION, "CURRENT TRANSACTION") \ MR_MACROS(CURRENTUSER, "CURRENTUSER") \ MR_MACROS(D, "D") \ + MR_MACROS(DATA, "DATA") \ + MR_MACROS(DATA_INNER_UUID, "DATA INNER UUID") \ MR_MACROS(DATABASE, "DATABASE") \ MR_MACROS(DATABASES, "DATABASES") \ MR_MACROS(DATE, "DATE") \ @@ -288,6 +290,8 @@ namespace DB MR_MACROS(MCS, "MCS") \ MR_MACROS(MEMORY, "MEMORY") \ MR_MACROS(MERGES, "MERGES") \ + MR_MACROS(METRICS, "METRICS") \ + MR_MACROS(METRICS_INNER_UUID, "METRICS INNER UUID") \ MR_MACROS(MI, "MI") \ MR_MACROS(MICROSECOND, "MICROSECOND") \ MR_MACROS(MICROSECONDS, "MICROSECONDS") \ @@ -464,6 +468,8 @@ namespace DB MR_MACROS(TABLE_OVERRIDE, "TABLE OVERRIDE") \ MR_MACROS(TABLE, "TABLE") \ MR_MACROS(TABLES, "TABLES") \ + MR_MACROS(TAGS, "TAGS") \ + MR_MACROS(TAGS_INNER_UUID, "TAGS INNER UUID") \ MR_MACROS(TEMPORARY_TABLE, "TEMPORARY TABLE") \ MR_MACROS(TEMPORARY, "TEMPORARY") \ MR_MACROS(TEST, "TEST") \ diff --git a/src/Parsers/CreateQueryUUIDs.cpp b/src/Parsers/CreateQueryUUIDs.cpp index 4dfee67b537..fbdc6161408 100644 --- a/src/Parsers/CreateQueryUUIDs.cpp +++ b/src/Parsers/CreateQueryUUIDs.cpp @@ -45,6 +45,13 @@ CreateQueryUUIDs::CreateQueryUUIDs(const ASTCreateQuery & query, bool generate_r /// then MV will create inner table. We should generate UUID of inner table here. if (query.is_materialized_view) generate_target_uuid(ViewTarget::To); + + if (query.is_time_series_table) + { + generate_target_uuid(ViewTarget::Data); + generate_target_uuid(ViewTarget::Tags); + generate_target_uuid(ViewTarget::Metrics); + } } } } diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index a592975613b..66965903ab0 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -696,6 +696,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ASTPtr table; ASTPtr columns_list; std::shared_ptr storage; + bool is_time_series_table = false; ASTPtr targets; ASTPtr as_database; ASTPtr as_table; @@ -784,6 +785,13 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return false; storage = typeid_cast>(ast); + + if (storage && storage->engine && (storage->engine->name == "TimeSeries")) + { + is_time_series_table = true; + ParserViewTargets({ViewTarget::Data, ViewTarget::Tags, ViewTarget::Metrics}).parse(pos, targets, expected); + } + return true; }; @@ -873,6 +881,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->create_or_replace = or_replace; query->if_not_exists = if_not_exists; query->temporary = is_temporary; + query->is_time_series_table = is_time_series_table; query->database = table_id->getDatabase(); query->table = table_id->getTable(); From 50604e8ea95b1d91850b8896ac21617b219b0309 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 6 May 2024 16:15:39 +0200 Subject: [PATCH 267/644] Add new table engine TimeSeries. --- src/CMakeLists.txt | 1 + src/Interpreters/InterpreterCreateQuery.cpp | 6 + src/Parsers/ASTFunction.h | 16 + src/Storages/StorageTimeSeries.cpp | 445 ++++++++++++++++++ src/Storages/StorageTimeSeries.h | 107 +++++ .../TimeSeries/TimeSeriesColumnNames.h | 34 ++ .../TimeSeries/TimeSeriesColumnsValidator.cpp | 246 ++++++++++ .../TimeSeries/TimeSeriesColumnsValidator.h | 51 ++ .../TimeSeriesDefinitionNormalizer.cpp | 416 ++++++++++++++++ .../TimeSeriesDefinitionNormalizer.h | 55 +++ .../TimeSeriesInnerTablesCreator.cpp | 163 +++++++ .../TimeSeries/TimeSeriesInnerTablesCreator.h | 47 ++ .../TimeSeries/TimeSeriesSettings.cpp | 34 ++ src/Storages/TimeSeries/TimeSeriesSettings.h | 25 + src/Storages/registerStorages.cpp | 4 + 15 files changed, 1650 insertions(+) create mode 100644 src/Storages/StorageTimeSeries.cpp create mode 100644 src/Storages/StorageTimeSeries.h create mode 100644 src/Storages/TimeSeries/TimeSeriesColumnNames.h create mode 100644 src/Storages/TimeSeries/TimeSeriesColumnsValidator.cpp create mode 100644 src/Storages/TimeSeries/TimeSeriesColumnsValidator.h create mode 100644 src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp create mode 100644 src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.h create mode 100644 src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp create mode 100644 src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.h create mode 100644 src/Storages/TimeSeries/TimeSeriesSettings.cpp create mode 100644 src/Storages/TimeSeries/TimeSeriesSettings.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 8c133971785..cc10fdf9646 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -225,6 +225,7 @@ add_object_library(clickhouse_storages_liveview Storages/LiveView) add_object_library(clickhouse_storages_windowview Storages/WindowView) add_object_library(clickhouse_storages_s3queue Storages/ObjectStorageQueue) add_object_library(clickhouse_storages_materializedview Storages/MaterializedView) +add_object_library(clickhouse_storages_time_series Storages/TimeSeries) add_object_library(clickhouse_client Client) # Always compile this file with the highest possible level of optimizations, even in Debug builds. # https://github.com/ClickHouse/ClickHouse/issues/65745 diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 971f90bd3cd..995ff8fac21 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include @@ -751,6 +752,10 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti if (create.storage && create.storage->engine) getContext()->checkAccess(AccessType::TABLE_ENGINE, create.storage->engine->name); + /// If this is a TimeSeries table then we need to normalize list of columns (add missing columns and reorder), and also set inner table engines. + if (create.is_time_series_table && (mode < LoadingStrictnessLevel::ATTACH)) + StorageTimeSeries::normalizeTableDefinition(create, getContext()); + TableProperties properties; TableLockHolder as_storage_lock; @@ -1058,6 +1063,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const else if (as_create.storage) { storage_def = typeid_cast>(as_create.storage->ptr()); + create.is_time_series_table = as_create.is_time_series_table; } else { diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index 1b4a5928d1c..5a14d66f31c 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -116,4 +116,20 @@ inline bool tryGetFunctionNameInto(const ASTPtr & ast, String & name) { return t /// Checks if function is a lambda function definition `lambda((x, y), x + y)` bool isASTLambdaFunction(const ASTFunction & function); +/// Makes an ASTFunction to represent a data type. +template +std::shared_ptr makeASTDataType(const String & type_name, Args &&... args) +{ + auto function = std::make_shared(); + function->name = type_name; + function->no_empty_args = true; + if (sizeof...(args)) + { + function->arguments = std::make_shared(); + function->children.push_back(function->arguments); + function->arguments->children = { std::forward(args)... }; + } + return function; +} + } diff --git a/src/Storages/StorageTimeSeries.cpp b/src/Storages/StorageTimeSeries.cpp new file mode 100644 index 00000000000..d85db53d78d --- /dev/null +++ b/src/Storages/StorageTimeSeries.cpp @@ -0,0 +1,445 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; + extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; +} + + +namespace +{ + namespace fs = std::filesystem; + + /// Loads TimeSeries storage settings from a create query. + std::shared_ptr getTimeSeriesSettingsFromQuery(const ASTCreateQuery & query) + { + auto storage_settings = std::make_shared(); + if (query.storage) + storage_settings->loadFromQuery(*query.storage); + return storage_settings; + } + + /// Creates an inner target table or just makes its storage ID. + /// This function is used by the constructor of StorageTimeSeries to find (or create) its target tables. + StorageID initTarget( + ViewTarget::Kind kind, + const ViewTarget * target_info, + const ContextPtr & context, + const StorageID & time_series_storage_id, + const ColumnsDescription & time_series_columns, + const TimeSeriesSettings & time_series_settings, + LoadingStrictnessLevel mode) + { + StorageID target_table_id = StorageID::createEmpty(); + + bool is_external_target = target_info && !target_info->table_id.empty(); + if (is_external_target) + { + /// A target table is specified. + target_table_id = target_info->table_id; + + if (mode < LoadingStrictnessLevel::ATTACH) + { + /// If it's not an ATTACH request then + /// check that the specified target table has all the required columns. + auto target_table = DatabaseCatalog::instance().getTable(target_table_id, context); + auto target_metadata = target_table->getInMemoryMetadataPtr(); + const auto & target_columns = target_metadata->columns; + TimeSeriesColumnsValidator validator{time_series_storage_id, time_series_settings}; + validator.validateTargetColumns(kind, target_table_id, target_columns); + } + } + else + { + TimeSeriesInnerTablesCreator inner_tables_creator{context, time_series_storage_id, time_series_columns, time_series_settings}; + auto inner_uuid = target_info ? target_info->inner_uuid : UUIDHelpers::Nil; + + /// An inner target table should be used. + if (mode >= LoadingStrictnessLevel::ATTACH) + { + /// If it's an ATTACH request, then the inner target table must be already created. + target_table_id = inner_tables_creator.getInnerTableID(kind, inner_uuid); + } + else + { + /// Create the inner target table. + auto inner_table_engine = target_info ? target_info->inner_engine : nullptr; + target_table_id = inner_tables_creator.createInnerTable(kind, inner_uuid, inner_table_engine); + } + } + + return target_table_id; + } +} + + +void StorageTimeSeries::normalizeTableDefinition(ASTCreateQuery & create_query, const ContextPtr & local_context) +{ + StorageID time_series_storage_id{create_query.getDatabase(), create_query.getTable()}; + TimeSeriesSettings time_series_settings; + if (create_query.storage) + time_series_settings.loadFromQuery(*create_query.storage); + std::shared_ptr as_create_query; + if (!create_query.as_table.empty()) + { + auto as_database = local_context->resolveDatabase(create_query.as_database); + as_create_query = typeid_cast>( + DatabaseCatalog::instance().getDatabase(as_database)->getCreateTableQuery(create_query.as_table, local_context)); + } + TimeSeriesDefinitionNormalizer normalizer{time_series_storage_id, time_series_settings, as_create_query.get()}; + normalizer.normalize(create_query); +} + + +StorageTimeSeries::StorageTimeSeries( + const StorageID & table_id, + const ContextPtr & local_context, + LoadingStrictnessLevel mode, + const ASTCreateQuery & query, + const ColumnsDescription & columns, + const String & comment) + : IStorage(table_id) + , WithContext(local_context->getGlobalContext()) +{ + storage_settings = getTimeSeriesSettingsFromQuery(query); + + if (mode < LoadingStrictnessLevel::ATTACH) + { + TimeSeriesColumnsValidator validator{table_id, *storage_settings}; + validator.validateColumns(columns); + } + + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns); + if (!comment.empty()) + storage_metadata.setComment(comment); + setInMemoryMetadata(storage_metadata); + + has_inner_tables = false; + + for (auto target_kind : {ViewTarget::Data, ViewTarget::Tags, ViewTarget::Metrics}) + { + const ViewTarget * target_info = query.targets ? query.targets->tryGetTarget(target_kind) : nullptr; + auto & target = targets.emplace_back(); + target.kind = target_kind; + target.table_id = initTarget(target_kind, target_info, local_context, getStorageID(), columns, *storage_settings, mode); + target.is_inner_table = target_info->table_id.empty(); + has_inner_tables |= target.is_inner_table; + } +} + + +StorageTimeSeries::~StorageTimeSeries() = default; + + +TimeSeriesSettings StorageTimeSeries::getStorageSettings() const +{ + return *getStorageSettingsPtr(); +} + +void StorageTimeSeries::startup() +{ +} + +void StorageTimeSeries::shutdown(bool) +{ +} + + +void StorageTimeSeries::drop() +{ + /// Sync flag and the setting make sense for Atomic databases only. + /// However, with Atomic databases, IStorage::drop() can be called only from a background task in DatabaseCatalog. + /// Running synchronous DROP from that task leads to deadlock. + dropInnerTableIfAny(/* sync= */ false, getContext()); +} + +void StorageTimeSeries::dropInnerTableIfAny(bool sync, ContextPtr local_context) +{ + if (!has_inner_tables) + return; + + for (const auto & target : targets) + { + if (target.is_inner_table && DatabaseCatalog::instance().tryGetTable(target.table_id, getContext())) + { + /// Best-effort to make them work: the inner table name is almost always less than the TimeSeries name (so it's safe to lock DDLGuard). + /// (See the comment in StorageMaterializedView::dropInnerTableIfAny.) + bool may_lock_ddl_guard = getStorageID().getQualifiedName() < target.table_id.getQualifiedName(); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target.table_id, + sync, /* ignore_sync_setting= */ true, may_lock_ddl_guard); + } + } +} + +void StorageTimeSeries::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) +{ + if (!has_inner_tables) + return; + + for (const auto & target : targets) + { + /// We truncate only inner tables here. + if (target.is_inner_table) + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Truncate, getContext(), local_context, target.table_id, /* sync= */ true); + } +} + + +StorageID StorageTimeSeries::getTargetTableId(ViewTarget::Kind target_kind) const +{ + for (const auto & target : targets) + { + if (target.kind == target_kind) + return target.table_id; + } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected target kind {}", toString(target_kind)); +} + +StoragePtr StorageTimeSeries::getTargetTable(ViewTarget::Kind target_kind, const ContextPtr & local_context) const +{ + return DatabaseCatalog::instance().getTable(getTargetTableId(target_kind), local_context); +} + +StoragePtr StorageTimeSeries::tryGetTargetTable(ViewTarget::Kind target_kind, const ContextPtr & local_context) const +{ + return DatabaseCatalog::instance().tryGetTable(getTargetTableId(target_kind), local_context); +} + + +std::optional StorageTimeSeries::totalRows(const Settings & settings) const +{ + UInt64 total_rows = 0; + if (has_inner_tables) + { + for (const auto & target : targets) + { + if (target.is_inner_table) + { + auto inner_table = DatabaseCatalog::instance().tryGetTable(target.table_id, getContext()); + if (!inner_table) + return std::nullopt; + + auto total_rows_in_inner_table = inner_table->totalRows(settings); + if (!total_rows_in_inner_table) + return std::nullopt; + + total_rows += *total_rows_in_inner_table; + } + } + } + return total_rows; +} + +std::optional StorageTimeSeries::totalBytes(const Settings & settings) const +{ + UInt64 total_bytes = 0; + if (has_inner_tables) + { + for (const auto & target : targets) + { + if (target.is_inner_table) + { + auto inner_table = DatabaseCatalog::instance().tryGetTable(target.table_id, getContext()); + if (!inner_table) + return std::nullopt; + + auto total_bytes_in_inner_table = inner_table->totalBytes(settings); + if (!total_bytes_in_inner_table) + return std::nullopt; + + total_bytes += *total_bytes_in_inner_table; + } + } + } + return total_bytes; +} + +std::optional StorageTimeSeries::totalBytesUncompressed(const Settings & settings) const +{ + UInt64 total_bytes = 0; + if (has_inner_tables) + { + for (const auto & target : targets) + { + if (target.is_inner_table) + { + auto inner_table = DatabaseCatalog::instance().tryGetTable(target.table_id, getContext()); + if (!inner_table) + return std::nullopt; + + auto total_bytes_in_inner_table = inner_table->totalBytesUncompressed(settings); + if (!total_bytes_in_inner_table) + return std::nullopt; + + total_bytes += *total_bytes_in_inner_table; + } + } + } + return total_bytes; +} + +Strings StorageTimeSeries::getDataPaths() const +{ + Strings data_paths; + for (const auto & target : targets) + { + auto table = DatabaseCatalog::instance().tryGetTable(target.table_id, getContext()); + if (!table) + continue; + + insertAtEnd(data_paths, table->getDataPaths()); + } + return data_paths; +} + + +bool StorageTimeSeries::optimize( + const ASTPtr & query, + const StorageMetadataPtr &, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Names & deduplicate_by_columns, + bool cleanup, + ContextPtr local_context) +{ + if (!has_inner_tables) + { + throw Exception(ErrorCodes::INCORRECT_QUERY, "TimeSeries table {} targets only existing tables. Execute the statement directly on it.", + getStorageID().getNameForLogs()); + } + + bool optimized = false; + for (const auto & target : targets) + { + if (target.is_inner_table) + { + auto inner_table = DatabaseCatalog::instance().getTable(target.table_id, local_context); + optimized |= inner_table->optimize(query, inner_table->getInMemoryMetadataPtr(), partition, final, deduplicate, deduplicate_by_columns, cleanup, local_context); + } + } + + return optimized; +} + + +void StorageTimeSeries::checkAlterIsPossible(const AlterCommands & commands, ContextPtr) const +{ + for (const auto & command : commands) + { + if (!command.isCommentAlter() && command.type != AlterCommand::MODIFY_SQL_SECURITY) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}", command.type, getName()); + } +} + +void StorageTimeSeries::alter(const AlterCommands & params, ContextPtr local_context, AlterLockHolder & table_lock_holder) +{ + IStorage::alter(params, local_context, table_lock_holder); +} + + +void StorageTimeSeries::renameInMemory(const StorageID & new_table_id) +{ + UNUSED(new_table_id); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Renaming is not supported by storage {} yet", getName()); +} + + +void StorageTimeSeries::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional &) +{ + for (const auto & target : targets) + { + /// We backup the target table's data only if it's inner. + if (target.is_inner_table) + { + auto table = DatabaseCatalog::instance().getTable(target.table_id, getContext()); + table->backupData(backup_entries_collector, fs::path{data_path_in_backup} / toString(target.kind), {}); + } + } +} + +void StorageTimeSeries::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional &) +{ + for (const auto & target : targets) + { + /// We backup the target table's data only if it's inner. + if (target.is_inner_table) + { + auto table = DatabaseCatalog::instance().getTable(target.table_id, getContext()); + table->restoreDataFromBackup(restorer, fs::path{data_path_in_backup} / toString(target.kind), {}); + } + } +} + + +void StorageTimeSeries::read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) +{ + UNUSED(query_plan); + UNUSED(column_names); + UNUSED(storage_snapshot); + UNUSED(query_info); + UNUSED(local_context); + UNUSED(processed_stage); + UNUSED(max_block_size); + UNUSED(num_streams); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "SELECT is not supported by storage {} yet", getName()); +} + + +SinkToStoragePtr StorageTimeSeries::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool async_insert) +{ + UNUSED(query); + UNUSED(metadata_snapshot); + UNUSED(local_context); + UNUSED(async_insert); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "INSERT is not supported by storage {} yet", getName()); +} + + +void registerStorageTimeSeries(StorageFactory & factory) +{ + factory.registerStorage("TimeSeries", [](const StorageFactory::Arguments & args) + { + /// Pass local_context here to convey setting to inner tables. + return std::make_shared( + args.table_id, args.getLocalContext(), args.mode, args.query, args.columns, args.comment); + } + , + { + .supports_settings = true, + .supports_schema_inference = true, + }); +} + +} diff --git a/src/Storages/StorageTimeSeries.h b/src/Storages/StorageTimeSeries.h new file mode 100644 index 00000000000..9ee09108803 --- /dev/null +++ b/src/Storages/StorageTimeSeries.h @@ -0,0 +1,107 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ +struct TimeSeriesSettings; +using TimeSeriesSettingsPtr = std::shared_ptr; + +/// Represents a table engine to keep time series received by Prometheus protocols. +/// Examples of using this table engine: +/// +/// CREATE TABLE ts ENGINE = TimeSeries() +/// -OR- +/// CREATE TABLE ts ENGINE = TimeSeries() DATA [db].table1 TAGS [db].table2 METRICS [db].table3 +/// -OR- +/// CREATE TABLE ts ENGINE = TimeSeries() DATA ENGINE = MergeTree TAGS ENGINE = ReplacingMergeTree METRICS ENGINE = ReplacingMergeTree +/// -OR- +/// CREATE TABLE ts ( +/// id UUID DEFAULT reinterpretAsUUID(sipHash128(metric_name, all_tags)) CODEC(ZSTD(3)), +/// instance LowCardinality(String), +/// job String +/// ) ENGINE = TimeSeries() +/// SETTINGS tags_to_columns = {'instance': 'instance', 'job': 'job'} +/// DATA ENGINE = ReplicatedMergeTree('zkpath', 'replica'), ... +/// +class StorageTimeSeries final : public IStorage, WithContext +{ +public: + /// Adds missing columns and reorder columns, and also adds inner table engines if they aren't specified. + static void normalizeTableDefinition(ASTCreateQuery & create_query, const ContextPtr & local_context); + + StorageTimeSeries(const StorageID & table_id, const ContextPtr & local_context, LoadingStrictnessLevel mode, + const ASTCreateQuery & query, const ColumnsDescription & columns, const String & comment); + + ~StorageTimeSeries() override; + + std::string getName() const override { return "TimeSeries"; } + + TimeSeriesSettings getStorageSettings() const; + TimeSeriesSettingsPtr getStorageSettingsPtr() const { return storage_settings; } + + StorageID getTargetTableId(ViewTarget::Kind target_kind) const; + StoragePtr getTargetTable(ViewTarget::Kind target_kind, const ContextPtr & local_context) const; + StoragePtr tryGetTargetTable(ViewTarget::Kind target_kind, const ContextPtr & local_context) const; + + void startup() override; + void shutdown(bool is_drop) override; + + void read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override; + + bool optimize( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Names & deduplicate_by_columns, + bool cleanup, + ContextPtr local_context) override; + + void drop() override; + void dropInnerTableIfAny(bool sync, ContextPtr local_context) override; + + void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; + + void renameInMemory(const StorageID & new_table_id) override; + + void checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const override; + void alter(const AlterCommands & params, ContextPtr local_context, AlterLockHolder & table_lock_holder) override; + + void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; + void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; + + std::optional totalRows(const Settings & settings) const override; + std::optional totalBytes(const Settings & settings) const override; + std::optional totalBytesUncompressed(const Settings & settings) const override; + Strings getDataPaths() const override; + +private: + TimeSeriesSettingsPtr storage_settings; + + struct Target + { + ViewTarget::Kind kind; + StorageID table_id = StorageID::createEmpty(); + bool is_inner_table; + }; + + std::vector targets; + bool has_inner_tables; +}; + +} diff --git a/src/Storages/TimeSeries/TimeSeriesColumnNames.h b/src/Storages/TimeSeries/TimeSeriesColumnNames.h new file mode 100644 index 00000000000..9176ec5384a --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesColumnNames.h @@ -0,0 +1,34 @@ +#pragma once + + +namespace DB +{ + +struct TimeSeriesColumnNames +{ + /// The "data" table contains time series: + static constexpr const char * ID = "id"; + static constexpr const char * Timestamp = "timestamp"; + static constexpr const char * Value = "value"; + + /// The "tags" table contains identifiers for each combination of a metric name with corresponding tags (labels): + + /// The default expression specified for the "id" column contains an expression for calculating an identifier of a time series by a metric name and tags. + //static constexpr const char * kID = "id"; + static constexpr const char * MetricName = "metric_name"; + + /// Contains tags which have no corresponding columns specified in the "tags_to_columns" setting. + static constexpr const char * Tags = "tags"; + + /// Contains all tags, including those ones which have corresponding columns specified in the "tags_to_columns" setting. + /// This is a generated column, it's not stored anywhere, it's generated on the fly. + static constexpr const char * AllTags = "all_tags"; + + /// The "metrics" table contains general information (metadata) about metrics: + static constexpr const char * MetricFamilyName = "metric_family_name"; + static constexpr const char * Type = "type"; + static constexpr const char * Unit = "unit"; + static constexpr const char * Help = "help"; +}; + +} diff --git a/src/Storages/TimeSeries/TimeSeriesColumnsValidator.cpp b/src/Storages/TimeSeries/TimeSeriesColumnsValidator.cpp new file mode 100644 index 00000000000..0ce5528939a --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesColumnsValidator.cpp @@ -0,0 +1,246 @@ +#include + +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCOMPATIBLE_COLUMNS; + extern const int THERE_IS_NO_COLUMN; +} + + +TimeSeriesColumnsValidator::TimeSeriesColumnsValidator(StorageID time_series_storage_id_, + std::reference_wrapper time_series_settings_) + : time_series_storage_id(std::move(time_series_storage_id_)) + , time_series_settings(time_series_settings_) +{ +} + + +void TimeSeriesColumnsValidator::validateColumns(const ColumnsDescription & columns) const +{ + try + { + validateColumnsImpl(columns); + } + catch (Exception & e) + { + e.addMessage("While checking columns of TimeSeries table {}", time_series_storage_id.getNameForLogs()); + throw; + } +} + + +void TimeSeriesColumnsValidator::validateColumnsImpl(const ColumnsDescription & columns) const +{ + + auto get_column_description = [&](const String & column_name) -> const ColumnDescription & + { + const auto * column = columns.tryGet(column_name); + if (!column) + { + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Column {} is required for the TimeSeries table engine", column_name); + } + return *column; + }; + + /// Validate columns for the "data" table. + validateColumnForID(get_column_description(TimeSeriesColumnNames::ID)); + validateColumnForTimestamp(get_column_description(TimeSeriesColumnNames::Timestamp)); + validateColumnForValue(get_column_description(TimeSeriesColumnNames::Value)); + + /// Validate columns for the "tags" table. + validateColumnForMetricName(get_column_description(TimeSeriesColumnNames::MetricName)); + + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + validateColumnForTagValue(get_column_description(column_name)); + } + + validateColumnForTagsMap(get_column_description(TimeSeriesColumnNames::Tags)); + validateColumnForTagsMap(get_column_description(TimeSeriesColumnNames::AllTags)); + + /// Validate columns for the "metrics" table. + validateColumnForMetricFamilyName(get_column_description(TimeSeriesColumnNames::MetricFamilyName)); + validateColumnForType(get_column_description(TimeSeriesColumnNames::Type)); + validateColumnForUnit(get_column_description(TimeSeriesColumnNames::Unit)); + validateColumnForHelp(get_column_description(TimeSeriesColumnNames::Help)); +} + + +void TimeSeriesColumnsValidator::validateTargetColumns(ViewTarget::Kind target_kind, const StorageID & target_table_id, const ColumnsDescription & target_columns) const +{ + try + { + validateTargetColumnsImpl(target_kind, target_columns); + } + catch (Exception & e) + { + e.addMessage("While checking columns of table {} which is the {} target of TimeSeries table {}", target_table_id.getNameForLogs(), + toString(target_kind), time_series_storage_id.getNameForLogs()); + throw; + } +} + + +void TimeSeriesColumnsValidator::validateTargetColumnsImpl(ViewTarget::Kind target_kind, const ColumnsDescription & target_columns) const +{ + auto get_column_description = [&](const String & column_name) -> const ColumnDescription & + { + const auto * column = target_columns.tryGet(column_name); + if (!column) + { + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Column {} is required for the TimeSeries table engine", column_name); + } + return *column; + }; + + switch (target_kind) + { + case ViewTarget::Data: + { + /// Here "check_default = false" because it's ok for the "id" column in the target table not to contain + /// an expression for calculating the identifier of a time series. + validateColumnForID(get_column_description(TimeSeriesColumnNames::ID), /* check_default= */ false); + + validateColumnForTimestamp(get_column_description(TimeSeriesColumnNames::Timestamp)); + validateColumnForValue(get_column_description(TimeSeriesColumnNames::Value)); + + break; + } + + case ViewTarget::Tags: + { + validateColumnForMetricName(get_column_description(TimeSeriesColumnNames::MetricName)); + + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + validateColumnForTagValue(get_column_description(column_name)); + } + + validateColumnForTagsMap(get_column_description(TimeSeriesColumnNames::Tags)); + + break; + } + + case ViewTarget::Metrics: + { + validateColumnForMetricFamilyName(get_column_description(TimeSeriesColumnNames::MetricFamilyName)); + validateColumnForType(get_column_description(TimeSeriesColumnNames::Type)); + validateColumnForUnit(get_column_description(TimeSeriesColumnNames::Unit)); + validateColumnForHelp(get_column_description(TimeSeriesColumnNames::Help)); + break; + } + + default: + UNREACHABLE(); + } +} + + +void TimeSeriesColumnsValidator::validateColumnForID(const ColumnDescription & column, bool check_default) const +{ + if (check_default && !column.default_desc.expression) + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "The DEFAULT expression for column {} must contain an expression " + "which will be used to calculate the identifier of each time series: {} {} DEFAULT ...", + column.name, column.name, column.type->getName()); + } +} + +void TimeSeriesColumnsValidator::validateColumnForTimestamp(const ColumnDescription & column) const +{ + if (!isDateTime64(removeNullable(column.type))) + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Column {} has illegal data type {}, expected DateTime64", + column.name, column.type->getName()); + } +} + +void TimeSeriesColumnsValidator::validateColumnForTimestamp(const ColumnDescription & column, UInt32 & out_scale) const +{ + auto maybe_datetime64_type = removeNullable(column.type); + if (!isDateTime64(maybe_datetime64_type)) + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Column {} has illegal data type {}, expected DateTime64", + column.name, column.type->getName()); + } + const auto & datetime64_type = typeid_cast(*maybe_datetime64_type); + out_scale = datetime64_type.getScale(); +} + +void TimeSeriesColumnsValidator::validateColumnForValue(const ColumnDescription & column) const +{ + if (!isFloat(removeNullable(column.type))) + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Column {} has illegal data type {}, expected Float32 or Float64", + column.name, column.type->getName()); + } +} + +void TimeSeriesColumnsValidator::validateColumnForMetricName(const ColumnDescription & column) const +{ + validateColumnForTagValue(column); +} + +void TimeSeriesColumnsValidator::validateColumnForTagValue(const ColumnDescription & column) const +{ + if (!isString(removeLowCardinalityAndNullable(column.type))) + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Column {} has illegal data type {}, expected String or LowCardinality(String)", + column.name, column.type->getName()); + } +} + +void TimeSeriesColumnsValidator::validateColumnForTagsMap(const ColumnDescription & column) const +{ + if (!isMap(column.type) + || !isString(removeLowCardinality(typeid_cast(*column.type).getKeyType())) + || !isString(removeLowCardinality(typeid_cast(*column.type).getValueType()))) + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Column {} has illegal data type {}, expected Map(String, String) or Map(LowCardinality(String), String)", + column.name, column.type->getName()); + } +} + +void TimeSeriesColumnsValidator::validateColumnForMetricFamilyName(const ColumnDescription & column) const +{ + if (!isString(removeLowCardinalityAndNullable(column.type))) + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Column {} has illegal data type {}, expected String or LowCardinality(String)", + column.name, column.type->getName()); + } +} + +void TimeSeriesColumnsValidator::validateColumnForType(const ColumnDescription & column) const +{ + validateColumnForMetricFamilyName(column); +} + +void TimeSeriesColumnsValidator::validateColumnForUnit(const ColumnDescription & column) const +{ + validateColumnForMetricFamilyName(column); +} + +void TimeSeriesColumnsValidator::validateColumnForHelp(const ColumnDescription & column) const +{ + validateColumnForMetricFamilyName(column); +} + +} diff --git a/src/Storages/TimeSeries/TimeSeriesColumnsValidator.h b/src/Storages/TimeSeries/TimeSeriesColumnsValidator.h new file mode 100644 index 00000000000..cafee9da03c --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesColumnsValidator.h @@ -0,0 +1,51 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ColumnsDescription; +struct ColumnDescription; +struct TimeSeriesSettings; + +/// Checks the types of columns of a TimeSeries table. +class TimeSeriesColumnsValidator +{ +public: + /// Constructor stores a reference to argument `time_series_settings_` (it's unnecessary to copy it). + TimeSeriesColumnsValidator(StorageID time_series_storage_id_, + std::reference_wrapper time_series_settings_); + + /// Checks the columns of a TimeSeries table and throws an exception if some of the required columns don't exist or have illegal types. + void validateColumns(const ColumnsDescription & columns) const; + + /// Checks columns of a target table that a TimeSeries table is going to use. + /// Throws an exception if some of the required columns don't exist or have illegal types. + void validateTargetColumns(ViewTarget::Kind target_kind, const StorageID & target_table_id, const ColumnsDescription & target_columns) const; + + /// Each of the following functions validates a specific column type. + void validateColumnForID(const ColumnDescription & column, bool check_default = true) const; + void validateColumnForTimestamp(const ColumnDescription & column) const; + void validateColumnForTimestamp(const ColumnDescription & column, UInt32 & out_scale) const; + void validateColumnForValue(const ColumnDescription & column) const; + + void validateColumnForMetricName(const ColumnDescription & column) const; + void validateColumnForTagValue(const ColumnDescription & column) const; + void validateColumnForTagsMap(const ColumnDescription & column) const; + + void validateColumnForMetricFamilyName(const ColumnDescription & column) const; + void validateColumnForType(const ColumnDescription & column) const; + void validateColumnForUnit(const ColumnDescription & column) const; + void validateColumnForHelp(const ColumnDescription & column) const; + +private: + void validateColumnsImpl(const ColumnsDescription & columns) const; + void validateTargetColumnsImpl(ViewTarget::Kind target_kind, const ColumnsDescription & target_columns) const; + + const StorageID time_series_storage_id; + const TimeSeriesSettings & time_series_settings; +}; + +} diff --git a/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp new file mode 100644 index 00000000000..78f8afe2528 --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp @@ -0,0 +1,416 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCOMPATIBLE_COLUMNS; + extern const int INCORRECT_QUERY; +} + + +TimeSeriesDefinitionNormalizer::TimeSeriesDefinitionNormalizer(StorageID time_series_storage_id_, + std::reference_wrapper time_series_settings_, + const ASTCreateQuery * as_create_query_) + : time_series_storage_id(std::move(time_series_storage_id_)) + , time_series_settings(time_series_settings_) + , as_create_query(as_create_query_) +{ +} + + +void TimeSeriesDefinitionNormalizer::normalize(ASTCreateQuery & create_query) const +{ + reorderColumns(create_query); + addMissingColumns(create_query); + addMissingDefaultForIDColumn(create_query); + + if (as_create_query) + addMissingInnerEnginesFromAsTable(create_query); + + addMissingInnerEngines(create_query); +} + + +void TimeSeriesDefinitionNormalizer::reorderColumns(ASTCreateQuery & create) const +{ + if (!create.columns_list || !create.columns_list->columns) + return; + + auto & columns = create.columns_list->columns->children; + + /// Build a map "column_name -> column_declaration". + std::unordered_map> columns_by_name; + for (const auto & column : columns) + { + auto column_declaration = typeid_cast>(column); + columns_by_name[column_declaration->name] = column_declaration; + } + + /// Remove all columns and then add them again in the canonical order. + columns.clear(); + + auto add_column_in_correct_order = [&](std::string_view column_name) + { + auto it = columns_by_name.find(column_name); + if (it != columns_by_name.end()) + { + /// Add the column back to the list. + columns.push_back(it->second); + + /// Remove the column from the map to allow the check at the end of this function + /// that all columns from the original list are added back to the list. + columns_by_name.erase(it); + } + }; + + /// Reorder columns for the "data" table. + add_column_in_correct_order(TimeSeriesColumnNames::ID); + add_column_in_correct_order(TimeSeriesColumnNames::Timestamp); + add_column_in_correct_order(TimeSeriesColumnNames::Value); + + /// Reorder columns for the "tags" table. + add_column_in_correct_order(TimeSeriesColumnNames::MetricName); + + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + add_column_in_correct_order(column_name); + } + + add_column_in_correct_order(TimeSeriesColumnNames::Tags); + add_column_in_correct_order(TimeSeriesColumnNames::AllTags); + + /// Reorder columns for the "metrics" table. + add_column_in_correct_order(TimeSeriesColumnNames::MetricFamilyName); + add_column_in_correct_order(TimeSeriesColumnNames::Type); + add_column_in_correct_order(TimeSeriesColumnNames::Unit); + add_column_in_correct_order(TimeSeriesColumnNames::Help); + + /// All columns from the original list must be added back to the list. + if (!columns_by_name.empty()) + { + throw Exception( + ErrorCodes::INCOMPATIBLE_COLUMNS, + "{}: Column {} can't be used in this table. " + "The TimeSeries table engine supports only a limited set of columns (id, timestamp, value, metric_name, tags, metric_family_name, type, unit, help). " + "Extra columns representing tags must be specified in the 'tags_to_columns' setting.", + time_series_storage_id.getNameForLogs(), columns_by_name.begin()->first); + } +} + + +void TimeSeriesDefinitionNormalizer::addMissingColumns(ASTCreateQuery & create) const +{ + if (!create.as_table.empty()) + { + /// If the create query has the "AS other_table" clause ("CREATE TABLE table AS other_table") + /// then all columns must be extracted from that "other_table". + /// Function InterpreterCreateQuery::getTablePropertiesAndNormalizeCreateQuery() will do that for us, + /// we don't need to fill missing columns by default in that case. + return; + } + + if (!create.columns_list) + create.set(create.columns_list, std::make_shared()); + + if (!create.columns_list->columns) + create.columns_list->set(create.columns_list->columns, std::make_shared()); + auto & columns = create.columns_list->columns->children; + + /// Here in this function we rely on that the columns are already sorted in the canonical order (see the reorderColumns() function). + /// NOTE: The order in which this function processes columns MUST be exactly the same as the order in reorderColumns(). + size_t position = 0; + + auto is_next_column_named = [&](std::string_view column_name) + { + if (position < columns.size() && (typeid_cast(*columns[position]).name == column_name)) + { + ++position; + return true; + } + return false; + }; + + auto make_new_column = [&](const String & column_name, ASTPtr type) + { + auto new_column = std::make_shared(); + new_column->name = column_name; + new_column->type = type; + columns.insert(columns.begin() + position, new_column); + ++position; + }; + + auto get_uuid_type = [] { return makeASTDataType("UUID"); }; + auto get_datetime_type = [] { return makeASTDataType("DateTime64", std::make_shared(3ul)); }; + auto get_float_type = [] { return makeASTDataType("Float64"); }; + auto get_string_type = [] { return makeASTDataType("String"); }; + auto get_lc_string_type = [&] { return makeASTDataType("LowCardinality", get_string_type()); }; + auto get_string_to_string_map_type = [&] { return makeASTDataType("Map", get_string_type(), get_string_type()); }; + auto get_lc_string_to_string_map_type = [&] { return makeASTDataType("Map", get_lc_string_type(), get_string_type()); }; + + /// Add missing columns for the "data" table. + if (!is_next_column_named(TimeSeriesColumnNames::ID)) + make_new_column(TimeSeriesColumnNames::ID, get_uuid_type()); + + if (!is_next_column_named(TimeSeriesColumnNames::Timestamp)) + make_new_column(TimeSeriesColumnNames::Timestamp, get_datetime_type()); + + if (!is_next_column_named(TimeSeriesColumnNames::Value)) + make_new_column(TimeSeriesColumnNames::Value, get_float_type()); + + /// Add missing columns for the "tags" table. + if (!is_next_column_named(TimeSeriesColumnNames::MetricName)) + { + /// We use 'LowCardinality(String)' as the default type of the `metric_name` column: + /// it looks like a correct optimization because there are shouldn't be too many different metrics. + make_new_column(TimeSeriesColumnNames::MetricName, get_lc_string_type()); + } + + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + if (!is_next_column_named(column_name)) + make_new_column(column_name, get_string_type()); + } + + if (!is_next_column_named(TimeSeriesColumnNames::Tags)) + { + /// We use 'Map(LowCardinality(String), String)' as the default type of the `tags` column: + /// it looks like a correct optimization because there are shouldn't be too many different tag names. + make_new_column(TimeSeriesColumnNames::Tags, get_lc_string_to_string_map_type()); + } + + if (!is_next_column_named(TimeSeriesColumnNames::AllTags)) + { + /// The `all_tags` column is virtual (it's calculated on the fly and never stored anywhere) + /// so here we don't need to use the LowCardinality optimization as for the `tags` column. + make_new_column(TimeSeriesColumnNames::AllTags, get_string_to_string_map_type()); + } + + /// Add missing columns for the "metrics" table. + if (!is_next_column_named(TimeSeriesColumnNames::MetricFamilyName)) + make_new_column(TimeSeriesColumnNames::MetricFamilyName, get_string_type()); + + if (!is_next_column_named(TimeSeriesColumnNames::Type)) + make_new_column(TimeSeriesColumnNames::Type, get_string_type()); + + if (!is_next_column_named(TimeSeriesColumnNames::Unit)) + make_new_column(TimeSeriesColumnNames::Unit, get_string_type()); + + if (!is_next_column_named(TimeSeriesColumnNames::Help)) + make_new_column(TimeSeriesColumnNames::Help, get_string_type()); + + /// If the following fails that means the order in which columns are processed in this function doesn't match the order of columns in reorderColumns(). + chassert(position == columns.size()); +} + + +void TimeSeriesDefinitionNormalizer::addMissingDefaultForIDColumn(ASTCreateQuery & create) const +{ + /// Find the 'id' column and make a default expression for it. + if (!create.columns_list || !create.columns_list->columns) + return; + + auto & columns = create.columns_list->columns->children; + auto * it = std::find_if(columns.begin(), columns.end(), [](const ASTPtr & column) + { + return typeid_cast(*column).name == TimeSeriesColumnNames::ID; + }); + + if (it == columns.end()) + return; + + auto & column_declaration = typeid_cast(**it); + + /// We add a DEFAULT for the 'id' column only if it's not specified yet. + if (column_declaration.default_specifier.empty() && !column_declaration.default_expression) + { + column_declaration.default_specifier = "DEFAULT"; + column_declaration.default_expression = chooseIDAlgorithm(column_declaration); + } +} + + +ASTPtr TimeSeriesDefinitionNormalizer::chooseIDAlgorithm(const ASTColumnDeclaration & id_column) const +{ + /// Build a list of arguments for a hash function. + /// All hash functions below allow multiple arguments, so we use two arguments: metric_name, all_tags. + ASTs arguments_for_hash_function; + arguments_for_hash_function.push_back(std::make_shared(TimeSeriesColumnNames::MetricName)); + arguments_for_hash_function.push_back(std::make_shared(TimeSeriesColumnNames::AllTags)); + + auto make_hash_function = [&](const String & function_name) + { + auto function = std::make_shared(); + function->name = function_name; + auto arguments_list = std::make_shared(); + arguments_list->children = std::move(arguments_for_hash_function); + function->arguments = arguments_list; + return function; + }; + + /// The type of a hash function depends on the type of the 'id' column. + auto id_type = DataTypeFactory::instance().get(id_column.type); + WhichDataType id_type_which(*id_type); + + if (id_type_which.isUInt64()) + { + return make_hash_function("sipHash64"); + } + else if (id_type_which.isFixedString() && typeid_cast(*id_type).getN() == 16) + { + return make_hash_function("sipHash128"); + } + else if (id_type_which.isUUID()) + { + return makeASTFunction("reinterpretAsUUID", make_hash_function("sipHash128")); + } + else if (id_type_which.isUInt128()) + { + return makeASTFunction("reinterpretAsUInt128", make_hash_function("sipHash128")); + } + else + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "{}: The DEFAULT expression for column {} must contain an expression " + "which will be used to calculate the identifier of each time series: {} {} DEFAULT ... " + "If the DEFAULT expression is not specified then it can be chosen implicitly but only if the column type is one of these: UInt64, UInt128, UUID. " + "For type {} the DEFAULT expression can't be chosen automatically, so please specify it explicitly", + time_series_storage_id.getNameForLogs(), id_column.name, id_column.name, id_type->getName(), id_type->getName()); + } +} + + +void TimeSeriesDefinitionNormalizer::addMissingInnerEnginesFromAsTable(ASTCreateQuery & create) const +{ + if (!as_create_query) + return; + + for (auto target_kind : {ViewTarget::Data, ViewTarget::Tags, ViewTarget::Metrics}) + { + if (as_create_query->hasTargetTableID(target_kind)) + { + /// It's unlikely correct to use "CREATE table AS other_table" when "other_table" has external tables like this: + /// CREATE TABLE other_table ENGINE=TimeSeries data mydata + /// (because `table` would use the same table "mydata"). + /// Thus we just prohibit that. + QualifiedTableName as_table{as_create_query->getDatabase(), as_create_query->getTable()}; + throw Exception( + ErrorCodes::INCORRECT_QUERY, + "Cannot CREATE a table AS {}.{} because it has external tables", + backQuoteIfNeed(as_table.database), backQuoteIfNeed(as_table.table)); + } + + auto inner_table_engine = create.getTargetInnerEngine(target_kind); + if (!inner_table_engine) + { + /// Copy an inner engine's definition from the other table. + inner_table_engine = as_create_query->getTargetInnerEngine(target_kind); + if (inner_table_engine) + create.setTargetInnerEngine(target_kind, typeid_cast>(inner_table_engine->clone())); + } + } +} + + +void TimeSeriesDefinitionNormalizer::addMissingInnerEngines(ASTCreateQuery & create) const +{ + for (auto target_kind : {ViewTarget::Data, ViewTarget::Tags, ViewTarget::Metrics}) + { + if (create.hasTargetTableID(target_kind)) + continue; /// External target is set, inner engine is not needed. + + auto inner_table_engine = create.getTargetInnerEngine(target_kind); + if (inner_table_engine && inner_table_engine->engine) + continue; /// Engine is set already, skip it. + + if (!inner_table_engine) + { + /// Some part of storage definition (such as PARTITION BY) is specified, but the inner ENGINE is not: just set default one. + inner_table_engine = std::make_shared(); + create.setTargetInnerEngine(target_kind, inner_table_engine); + } + + /// Set engine by default. + setInnerEngineByDefault(target_kind, *inner_table_engine); + } +} + + +void TimeSeriesDefinitionNormalizer::setInnerEngineByDefault(ViewTarget::Kind inner_table_kind, ASTStorage & inner_storage_def) const +{ + switch (inner_table_kind) + { + case ViewTarget::Data: + { + inner_storage_def.set(inner_storage_def.engine, makeASTFunction("MergeTree")); + inner_storage_def.engine->no_empty_args = false; + + if (!inner_storage_def.order_by && !inner_storage_def.primary_key && inner_storage_def.engine->name.ends_with("MergeTree")) + { + inner_storage_def.set(inner_storage_def.order_by, + makeASTFunction("tuple", + std::make_shared(TimeSeriesColumnNames::ID), + std::make_shared(TimeSeriesColumnNames::Timestamp))); + } + break; + } + + case ViewTarget::Tags: + { + inner_storage_def.set(inner_storage_def.engine, makeASTFunction("ReplacingMergeTree")); + inner_storage_def.engine->no_empty_args = false; + + if (!inner_storage_def.order_by && !inner_storage_def.primary_key && inner_storage_def.engine->name.ends_with("MergeTree")) + { + inner_storage_def.set(inner_storage_def.primary_key, + std::make_shared(TimeSeriesColumnNames::MetricName)); + + ASTs order_by_list; + order_by_list.push_back(std::make_shared(TimeSeriesColumnNames::MetricName)); + order_by_list.push_back(std::make_shared(TimeSeriesColumnNames::ID)); + + auto order_by_tuple = std::make_shared(); + order_by_tuple->name = "tuple"; + auto arguments_list = std::make_shared(); + arguments_list->children = std::move(order_by_list); + order_by_tuple->arguments = arguments_list; + inner_storage_def.set(inner_storage_def.order_by, order_by_tuple); + } + break; + } + + case ViewTarget::Metrics: + { + inner_storage_def.set(inner_storage_def.engine, makeASTFunction("ReplacingMergeTree")); + inner_storage_def.engine->no_empty_args = false; + + if (!inner_storage_def.order_by && !inner_storage_def.primary_key && inner_storage_def.engine->name.ends_with("MergeTree")) + { + inner_storage_def.set(inner_storage_def.order_by, std::make_shared(TimeSeriesColumnNames::MetricFamilyName)); + } + break; + } + + default: + UNREACHABLE(); /// This function must not be called with any other `kind`. + } +} + +} diff --git a/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.h b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.h new file mode 100644 index 00000000000..1f959eb3ce0 --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ASTColumnDeclaration; +class ASTCreateQuery; +struct ColumnDescription; +struct TimeSeriesSettings; + +/// Normalizes a TimeSeries table definition. +class TimeSeriesDefinitionNormalizer +{ +public: + /// Constructor stores a reference to argument `time_series_settings_` (it's unnecessary to copy it). + TimeSeriesDefinitionNormalizer(StorageID time_series_storage_id_, + std::reference_wrapper time_series_settings_, + const ASTCreateQuery * as_create_query_); + + /// Adds missing columns to the definition and reorders all the columns in the canonical way. + /// Also adds engines of inner tables to the definition if they aren't specified yet. + /// The `as_table_create_query` parameter must be nullptr if it isn't a "CREATE AS query". + void normalize(ASTCreateQuery & create_query) const; + +private: + /// Reorders existing columns in the canonical way. + void reorderColumns(ASTCreateQuery & create) const; + + /// Adds missing columns with data types set by default.. + void addMissingColumns(ASTCreateQuery & create) const; + + /// Adds the DEFAULT expression for the 'id' column if it isn't specified yet. + void addMissingDefaultForIDColumn(ASTCreateQuery & create) const; + + /// Generates a formulae for calculating the identifier of a time series from the metric name and all the tags. + ASTPtr chooseIDAlgorithm(const ASTColumnDeclaration & id_column) const; + + /// Copies the definitions of inner engines from "CREATE AS
" if this is that kind of query. + void addMissingInnerEnginesFromAsTable(ASTCreateQuery & create) const; + + /// Adds engines of inner tables to the definition if they aren't specified yet. + void addMissingInnerEngines(ASTCreateQuery & create) const; + + /// Sets the engine of an inner table by default. + void setInnerEngineByDefault(ViewTarget::Kind inner_table_kind, ASTStorage & inner_storage_def) const; + + const StorageID time_series_storage_id; + const TimeSeriesSettings & time_series_settings; + const ASTCreateQuery * as_create_query = nullptr; +}; + +} diff --git a/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp new file mode 100644 index 00000000000..e43cba01cdb --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp @@ -0,0 +1,163 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +TimeSeriesInnerTablesCreator::TimeSeriesInnerTablesCreator(ContextPtr context_, + StorageID time_series_storage_id_, + std::reference_wrapper time_series_columns_, + std::reference_wrapper time_series_settings_) + : WithContext(context_) + , time_series_storage_id(std::move(time_series_storage_id_)) + , time_series_columns(time_series_columns_) + , time_series_settings(time_series_settings_) +{ +} + +TimeSeriesInnerTablesCreator::~TimeSeriesInnerTablesCreator() = default; + + +ColumnsDescription TimeSeriesInnerTablesCreator::getInnerTableColumnsDescription(ViewTarget::Kind inner_table_kind) const +{ + ColumnsDescription columns; + + switch (inner_table_kind) + { + case ViewTarget::Data: + { + /// Column "id". + { + auto id_column = time_series_columns.get(TimeSeriesColumnNames::ID); + /// The expression for calculating the identifier of a time series can be transferred only to the "tags" inner table + /// (because it usually depends on columns like "metric_name" or "all_tags"). + id_column.default_desc = {}; + columns.add(std::move(id_column)); + } + + /// Column "timestamp". + columns.add(time_series_columns.get(TimeSeriesColumnNames::Timestamp)); + + /// Column "value". + columns.add(time_series_columns.get(TimeSeriesColumnNames::Value)); + break; + } + + case ViewTarget::Tags: + { + /// Column "id". + columns.add(time_series_columns.get(TimeSeriesColumnNames::ID)); + + /// Column "metric_name". + columns.add(time_series_columns.get(TimeSeriesColumnNames::MetricName)); + + /// Columns corresponding to specific tags specified in the "tags_to_columns" setting. + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + columns.add(time_series_columns.get(column_name)); + } + + /// Column "tags". + columns.add(time_series_columns.get(TimeSeriesColumnNames::Tags)); + + /// Column "all_tags". + ColumnDescription all_tags_column = time_series_columns.get(TimeSeriesColumnNames::AllTags); + /// Column "all_tags" is here only to calculate the identifier of a time series for the "id" column, so it can be ephemeral. + all_tags_column.default_desc.kind = ColumnDefaultKind::Ephemeral; + if (!all_tags_column.default_desc.expression) + { + all_tags_column.default_desc.ephemeral_default = true; + all_tags_column.default_desc.expression = makeASTFunction("defaultValueOfTypeName", std::make_shared(all_tags_column.type->getName())); + } + columns.add(std::move(all_tags_column)); + + break; + } + + case ViewTarget::Metrics: + { + columns.add(time_series_columns.get(TimeSeriesColumnNames::MetricFamilyName)); + columns.add(time_series_columns.get(TimeSeriesColumnNames::Type)); + columns.add(time_series_columns.get(TimeSeriesColumnNames::Unit)); + columns.add(time_series_columns.get(TimeSeriesColumnNames::Help)); + break; + } + + default: + UNREACHABLE(); + } + + return columns; +} + + +StorageID TimeSeriesInnerTablesCreator::getInnerTableID(ViewTarget::Kind inner_table_kind, const UUID & inner_table_uuid) const +{ + StorageID res = time_series_storage_id; + if (time_series_storage_id.hasUUID()) + res.table_name = fmt::format(".inner_id.{}.{}", toString(inner_table_kind), time_series_storage_id.uuid); + else + res.table_name = fmt::format(".inner.{}.{}", toString(inner_table_kind), time_series_storage_id.table_name); + res.uuid = inner_table_uuid; + return res; +} + + +std::shared_ptr TimeSeriesInnerTablesCreator::getInnerTableCreateQuery( + ViewTarget::Kind inner_table_kind, + const UUID & inner_table_uuid, + const std::shared_ptr inner_storage_def) const +{ + auto manual_create_query = std::make_shared(); + + auto inner_table_id = getInnerTableID(inner_table_kind, inner_table_uuid); + manual_create_query->setDatabase(inner_table_id.database_name); + manual_create_query->setTable(inner_table_id.table_name); + manual_create_query->uuid = inner_table_id.uuid; + manual_create_query->has_uuid = inner_table_id.uuid != UUIDHelpers::Nil; + + auto new_columns_list = std::make_shared(); + new_columns_list->set(new_columns_list->columns, InterpreterCreateQuery::formatColumns(getInnerTableColumnsDescription(inner_table_kind))); + manual_create_query->set(manual_create_query->columns_list, new_columns_list); + + if (inner_storage_def) + manual_create_query->set(manual_create_query->storage, inner_storage_def->clone()); + + return manual_create_query; +} + +StorageID TimeSeriesInnerTablesCreator::createInnerTable( + ViewTarget::Kind inner_table_kind, + const UUID & inner_table_uuid, + const std::shared_ptr inner_storage_def) const +{ + /// We will make a query to create the inner target table. + auto create_context = Context::createCopy(getContext()); + + auto manual_create_query = getInnerTableCreateQuery(inner_table_kind, inner_table_uuid, inner_storage_def); + + /// Create the inner target table. + InterpreterCreateQuery create_interpreter(manual_create_query, create_context); + create_interpreter.setInternal(true); + create_interpreter.execute(); + + return DatabaseCatalog::instance().getTable({manual_create_query->getDatabase(), manual_create_query->getTable()}, getContext())->getStorageID(); +} + +} diff --git a/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.h b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.h new file mode 100644 index 00000000000..a59bd2107bb --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.h @@ -0,0 +1,47 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ASTCreateQuery; +class ColumnsDescription; +struct TimeSeriesSettings; + +/// Generates inner tables for the TimeSeries table engine. +class TimeSeriesInnerTablesCreator : public WithContext +{ +public: + /// Constructor stores references to arguments `time_series_columns_` and `time_series_settings_` (it's unnecessary to copy them). + TimeSeriesInnerTablesCreator(ContextPtr context_, + StorageID time_series_storage_id_, + std::reference_wrapper time_series_columns_, + std::reference_wrapper time_series_settings_); + + ~TimeSeriesInnerTablesCreator(); + + /// Returns a column description of an inner table. + ColumnsDescription getInnerTableColumnsDescription(ViewTarget::Kind inner_table_kind) const; + + /// Returns a StorageID of an inner table. + StorageID getInnerTableID(ViewTarget::Kind inner_table_kind, const UUID & inner_table_uuid) const; + + /// Generates a CREATE TABLE query for an inner table. + std::shared_ptr getInnerTableCreateQuery(ViewTarget::Kind inner_table_kind, + const UUID & inner_table_uuid, + const std::shared_ptr inner_storage_def) const; + + /// Creates an inner table. + StorageID createInnerTable(ViewTarget::Kind inner_table_kind, + const UUID & inner_table_uuid, + const std::shared_ptr inner_storage_def) const; + +private: + const StorageID time_series_storage_id; + const ColumnsDescription & time_series_columns; + const TimeSeriesSettings & time_series_settings; +}; + +} diff --git a/src/Storages/TimeSeries/TimeSeriesSettings.cpp b/src/Storages/TimeSeries/TimeSeriesSettings.cpp new file mode 100644 index 00000000000..3a15be59191 --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesSettings.cpp @@ -0,0 +1,34 @@ +#include + +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + +IMPLEMENT_SETTINGS_TRAITS(TimeSeriesSettingsTraits, LIST_OF_TIME_SERIES_SETTINGS) + +void TimeSeriesSettings::loadFromQuery(ASTStorage & storage_def) +{ + if (storage_def.settings) + { + try + { + applyChanges(storage_def.settings->changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + e.addMessage("for storage " + storage_def.engine->name); + throw; + } + } +} + +} diff --git a/src/Storages/TimeSeries/TimeSeriesSettings.h b/src/Storages/TimeSeries/TimeSeriesSettings.h new file mode 100644 index 00000000000..ea31aa8dac1 --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesSettings.h @@ -0,0 +1,25 @@ +#pragma once + +#include + + +namespace DB +{ +class ASTStorage; + +#define LIST_OF_TIME_SERIES_SETTINGS(M, ALIAS) \ + M(Map, tags_to_columns, Map{}, "Map specifying which tags should be put to separate columns of the 'tags' table. Syntax: {'tag1': 'column1', 'tag2' : column2, ...}", 0) \ + +DECLARE_SETTINGS_TRAITS(TimeSeriesSettingsTraits, LIST_OF_TIME_SERIES_SETTINGS) + +/// Settings for the TimeSeries table engine. +/// Could be loaded from a CREATE TABLE query (SETTINGS clause). For example: +/// CREATE TABLE mytable ENGINE = TimeSeries() SETTINGS tags_to_columns = {'job':'job', 'instance':'instance'} DATA ENGINE = ReplicatedMergeTree('zkpath', 'replica'), ... +struct TimeSeriesSettings : public BaseSettings +{ + void loadFromQuery(ASTStorage & storage_def); +}; + +using TimeSeriesSettingsPtr = std::shared_ptr; + +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 8f33314397c..14474bf001d 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -26,6 +26,8 @@ void registerStorageGenerateRandom(StorageFactory & factory); void registerStorageExecutable(StorageFactory & factory); void registerStorageWindowView(StorageFactory & factory); void registerStorageLoop(StorageFactory & factory); +void registerStorageTimeSeries(StorageFactory & factory); + #if USE_RAPIDJSON || USE_SIMDJSON void registerStorageFuzzJSON(StorageFactory & factory); #endif @@ -126,6 +128,8 @@ void registerStorages() registerStorageExecutable(factory); registerStorageWindowView(factory); registerStorageLoop(factory); + registerStorageTimeSeries(factory); + #if USE_RAPIDJSON || USE_SIMDJSON registerStorageFuzzJSON(factory); #endif From 03ebab2ca52f8e7c5e8a96a692b9dc863bba2271 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 14 May 2024 17:08:14 +0200 Subject: [PATCH 268/644] Refactor factory of PrometheusRequestHandler to make extending it easier. --- programs/keeper/Keeper.cpp | 5 +- src/Server/HTTPHandlerFactory.cpp | 28 +-- src/Server/HTTPHandlerFactory.h | 18 +- src/Server/PrometheusMetricsWriter.cpp | 152 ++++++++-------- src/Server/PrometheusMetricsWriter.h | 37 ++-- src/Server/PrometheusRequestHandler.cpp | 171 +++++++++++++----- src/Server/PrometheusRequestHandler.h | 45 +++-- src/Server/PrometheusRequestHandlerConfig.h | 27 +++ .../PrometheusRequestHandlerFactory.cpp | 151 ++++++++++++++++ src/Server/PrometheusRequestHandlerFactory.h | 97 ++++++++++ 10 files changed, 534 insertions(+), 197 deletions(-) create mode 100644 src/Server/PrometheusRequestHandlerConfig.h create mode 100644 src/Server/PrometheusRequestHandlerFactory.cpp create mode 100644 src/Server/PrometheusRequestHandlerFactory.h diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 783f60cb8ff..0cf3f5c86d6 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -38,7 +38,7 @@ #include #include #include -#include +#include #include #include "Core/Defines.h" @@ -509,14 +509,13 @@ try auto address = socketBindListen(socket, listen_host, port); socket.setReceiveTimeout(my_http_context->getReceiveTimeout()); socket.setSendTimeout(my_http_context->getSendTimeout()); - auto metrics_writer = std::make_shared(config, "prometheus", async_metrics); servers->emplace_back( listen_host, port_name, "Prometheus: http://" + address.toString(), std::make_unique( std::move(my_http_context), - createPrometheusMainHandlerFactory(*this, config_getter(), metrics_writer, "PrometheusHandler-factory"), + createKeeperPrometheusHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 5344b2d024b..2e7f16f59d7 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -1,18 +1,16 @@ -#include #include #include +#include +#include #include -#include #include #include "HTTPHandler.h" -#include "Server/PrometheusMetricsWriter.h" #include "StaticRequestHandler.h" #include "ReplicasStatusHandler.h" #include "InterserverIOHTTPHandler.h" -#include "PrometheusRequestHandler.h" #include "WebUIRequestHandler.h" @@ -124,7 +122,7 @@ static inline auto createHandlersFactoryFromConfig( } else if (handler_type == "prometheus") { - main_handler_factory->addHandler(createPrometheusHandlerFactory(server, config, async_metrics, prefix + "." + key)); + main_handler_factory->addHandler(createPrometheusHandlerFactoryForHTTPRule(server, config, prefix + "." + key, async_metrics)); } else if (handler_type == "replicas_status") { @@ -201,10 +199,7 @@ HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco:: else if (name == "InterserverIOHTTPHandler-factory" || name == "InterserverIOHTTPSHandler-factory") return createInterserverHTTPHandlerFactory(server, name); else if (name == "PrometheusHandler-factory") - { - auto metrics_writer = std::make_shared(config, "prometheus", async_metrics); - return createPrometheusMainHandlerFactory(server, config, metrics_writer, name); - } + return createPrometheusHandlerFactory(server, config, async_metrics, name); throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown HTTP handler factory name."); } @@ -291,20 +286,9 @@ void addDefaultHandlersFactory( ); factory.addHandler(query_handler); - /// We check that prometheus handler will be served on current (default) port. - /// Otherwise it will be created separately, see createHandlerFactory(...). - if (config.has("prometheus") && config.getInt("prometheus.port", 0) == 0) - { - auto writer = std::make_shared(config, "prometheus", async_metrics); - auto creator = [&server, writer] () -> std::unique_ptr - { - return std::make_unique(server, writer); - }; - auto prometheus_handler = std::make_shared>(std::move(creator)); - prometheus_handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); - prometheus_handler->allowGetAndHeadRequest(); + /// createPrometheusHandlerFactoryForHTTPRuleDefaults() can return nullptr if prometheus protocols must not be served on http port. + if (auto prometheus_handler = createPrometheusHandlerFactoryForHTTPRuleDefaults(server, config, async_metrics)) factory.addHandler(prometheus_handler); - } } } diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index b4c32366463..db4bb73cbc4 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -1,15 +1,12 @@ #pragma once -#include -#include #include #include #include #include -#include - #include + namespace DB { @@ -19,6 +16,7 @@ namespace ErrorCodes } class IServer; +class AsynchronousMetrics; template class HandlingRuleHTTPHandlerFactory : public HTTPRequestHandlerFactory @@ -126,18 +124,6 @@ HTTPRequestHandlerFactoryPtr createReplicasStatusHandlerFactory(IServer & server const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); -HTTPRequestHandlerFactoryPtr -createPrometheusHandlerFactory(IServer & server, - const Poco::Util::AbstractConfiguration & config, - AsynchronousMetrics & async_metrics, - const std::string & config_prefix); - -HTTPRequestHandlerFactoryPtr createPrometheusMainHandlerFactory( - IServer & server, - const Poco::Util::AbstractConfiguration & config, - PrometheusMetricsWriterPtr metrics_writer, - const std::string & name); - /// @param server - used in handlers to check IServer::isCancelled() /// @param config - not the same as server.config(), since it can be newer /// @param async_metrics - used for prometheus (in case of prometheus.asynchronous_metrics=true) diff --git a/src/Server/PrometheusMetricsWriter.cpp b/src/Server/PrometheusMetricsWriter.cpp index 85eafbe4808..43370116015 100644 --- a/src/Server/PrometheusMetricsWriter.cpp +++ b/src/Server/PrometheusMetricsWriter.cpp @@ -1,13 +1,27 @@ #include "PrometheusMetricsWriter.h" -#include +#include +#include #include #include - -#include +#include #include "config.h" + +#if USE_NURAFT +namespace ProfileEvents +{ + extern const std::vector keeper_profile_events; +} + +namespace CurrentMetrics +{ + extern const std::vector keeper_metrics; +} +#endif + + namespace { @@ -107,100 +121,84 @@ void writeAsyncMetrics(DB::WriteBuffer & wb, const DB::AsynchronousMetricValues } -#if USE_NURAFT -namespace ProfileEvents -{ - extern const std::vector keeper_profile_events; -} - -namespace CurrentMetrics -{ - extern const std::vector keeper_metrics; -} -#endif - namespace DB { -PrometheusMetricsWriter::PrometheusMetricsWriter( - const Poco::Util::AbstractConfiguration & config, const std::string & config_name, - const AsynchronousMetrics & async_metrics_) - : async_metrics(async_metrics_) - , send_events(config.getBool(config_name + ".events", true)) - , send_metrics(config.getBool(config_name + ".metrics", true)) - , send_asynchronous_metrics(config.getBool(config_name + ".asynchronous_metrics", true)) - , send_errors(config.getBool(config_name + ".errors", true)) +void PrometheusMetricsWriter::writeEvents(WriteBuffer & wb) const { + for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) + writeEvent(wb, i); } -void PrometheusMetricsWriter::write(WriteBuffer & wb) const +void PrometheusMetricsWriter::writeMetrics(WriteBuffer & wb) const { - if (send_events) + for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) + writeMetric(wb, i); +} + +void PrometheusMetricsWriter::writeAsynchronousMetrics(WriteBuffer & wb, const AsynchronousMetrics & async_metrics) const +{ + writeAsyncMetrics(wb, async_metrics.getValues()); +} + +void PrometheusMetricsWriter::writeErrors(WriteBuffer & wb) const +{ + size_t total_count = 0; + + for (size_t i = 0, end = ErrorCodes::end(); i < end; ++i) { - for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) - writeEvent(wb, i); - } + const auto & error = ErrorCodes::values[i].get(); + std::string_view name = ErrorCodes::getName(static_cast(i)); - if (send_metrics) - { - for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) - writeMetric(wb, i); - } + if (name.empty()) + continue; - if (send_asynchronous_metrics) - writeAsyncMetrics(wb, async_metrics.getValues()); + std::string key{error_metrics_prefix + toString(name)}; + std::string help = fmt::format("The number of {} errors since last server restart", name); - if (send_errors) - { - size_t total_count = 0; - - for (size_t i = 0, end = ErrorCodes::end(); i < end; ++i) - { - const auto & error = ErrorCodes::values[i].get(); - std::string_view name = ErrorCodes::getName(static_cast(i)); - - if (name.empty()) - continue; - - std::string key{error_metrics_prefix + toString(name)}; - std::string help = fmt::format("The number of {} errors since last server restart", name); - - writeOutLine(wb, "# HELP", key, help); - writeOutLine(wb, "# TYPE", key, "counter"); - /// We are interested in errors which are happened only on this server. - writeOutLine(wb, key, error.local.count); - - total_count += error.local.count; - } - - /// Write the total number of errors as a separate metric - std::string key{error_metrics_prefix + toString("ALL")}; - writeOutLine(wb, "# HELP", key, "The total number of errors since last server restart"); + writeOutLine(wb, "# HELP", key, help); writeOutLine(wb, "# TYPE", key, "counter"); - writeOutLine(wb, key, total_count); + /// We are interested in errors which are happened only on this server. + writeOutLine(wb, key, error.local.count); + + total_count += error.local.count; } + /// Write the total number of errors as a separate metric + std::string key{error_metrics_prefix + toString("ALL")}; + writeOutLine(wb, "# HELP", key, "The total number of errors since last server restart"); + writeOutLine(wb, "# TYPE", key, "counter"); + writeOutLine(wb, key, total_count); } -void KeeperPrometheusMetricsWriter::write([[maybe_unused]] WriteBuffer & wb) const + +void KeeperPrometheusMetricsWriter::writeEvents([[maybe_unused]] WriteBuffer & wb) const { #if USE_NURAFT - if (send_events) - { - for (auto event : ProfileEvents::keeper_profile_events) - writeEvent(wb, event); - } - - if (send_metrics) - { - for (auto metric : CurrentMetrics::keeper_metrics) - writeMetric(wb, metric); - } - - if (send_asynchronous_metrics) - writeAsyncMetrics(wb, async_metrics.getValues()); + for (auto event : ProfileEvents::keeper_profile_events) + writeEvent(wb, event); #endif } +void KeeperPrometheusMetricsWriter::writeMetrics([[maybe_unused]] WriteBuffer & wb) const +{ +#if USE_NURAFT + for (auto metric : CurrentMetrics::keeper_metrics) + writeMetric(wb, metric); +#endif +} + +void KeeperPrometheusMetricsWriter::writeAsynchronousMetrics([[maybe_unused]] WriteBuffer & wb, + [[maybe_unused]] const AsynchronousMetrics & async_metrics) const +{ +#if USE_NURAFT + writeAsyncMetrics(wb, async_metrics.getValues()); +#endif +} + +void KeeperPrometheusMetricsWriter::writeErrors(WriteBuffer &) const +{ +} + } diff --git a/src/Server/PrometheusMetricsWriter.h b/src/Server/PrometheusMetricsWriter.h index 933ad909ee0..cf2587d80b8 100644 --- a/src/Server/PrometheusMetricsWriter.h +++ b/src/Server/PrometheusMetricsWriter.h @@ -1,44 +1,33 @@ #pragma once -#include - -#include -#include -#include - -#include +#include namespace DB { +class AsynchronousMetrics; +class WriteBuffer; /// Write metrics in Prometheus format class PrometheusMetricsWriter { public: - PrometheusMetricsWriter( - const Poco::Util::AbstractConfiguration & config, const std::string & config_name, - const AsynchronousMetrics & async_metrics_); - - virtual void write(WriteBuffer & wb) const; - virtual ~PrometheusMetricsWriter() = default; -protected: - const AsynchronousMetrics & async_metrics; - const bool send_events; - const bool send_metrics; - const bool send_asynchronous_metrics; - const bool send_errors; + virtual void writeMetrics(WriteBuffer & wb) const; + virtual void writeAsynchronousMetrics(WriteBuffer & wb, const AsynchronousMetrics & async_metrics) const; + virtual void writeEvents(WriteBuffer & wb) const; + virtual void writeErrors(WriteBuffer & wb) const; }; + class KeeperPrometheusMetricsWriter : public PrometheusMetricsWriter { - using PrometheusMetricsWriter::PrometheusMetricsWriter; - - void write(WriteBuffer & wb) const override; +public: + void writeMetrics(WriteBuffer & wb) const override; + void writeAsynchronousMetrics(WriteBuffer & wb, const AsynchronousMetrics & async_metrics) const override; + void writeEvents(WriteBuffer & wb) const override; + void writeErrors(WriteBuffer & wb) const override; }; -using PrometheusMetricsWriterPtr = std::shared_ptr; - } diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index 87c106c3fc0..7aabe07753d 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -1,74 +1,159 @@ #include +#include #include #include -#include #include -#include -#include -#include -#include "Server/PrometheusMetricsWriter.h" - -#include +#include +#include "config.h" namespace DB { -void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int SUPPORT_IS_DISABLED; +} + +/// Base implementation of a prometheus protocol. +class PrometheusRequestHandler::Impl +{ +public: + explicit Impl(PrometheusRequestHandler & parent) : parent_ref(parent) {} + virtual ~Impl() = default; + virtual void beforeHandlingRequest(HTTPServerRequest & /* request */) {} + virtual void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) = 0; + virtual void onException() {} + +protected: + PrometheusRequestHandler & parent() { return parent_ref; } + IServer & server() { return parent().server; } + const PrometheusRequestHandlerConfig & config() { return parent().config; } + PrometheusMetricsWriter & metrics_writer() { return *parent().metrics_writer; } + LoggerPtr log() { return parent().log; } + WriteBuffer & getOutputStream(HTTPServerResponse & response) { return parent().getOutputStream(response); } + +private: + PrometheusRequestHandler & parent_ref; +}; + + +/// Implementation of the exposing metrics protocol. +class PrometheusRequestHandler::ExposeMetricsImpl : public Impl +{ +public: + explicit ExposeMetricsImpl(PrometheusRequestHandler & parent) : Impl(parent) {} + + void beforeHandlingRequest(HTTPServerRequest & request) override + { + LOG_INFO(log(), "Handling metrics request from {}", request.get("User-Agent")); + chassert(config().type == PrometheusRequestHandlerConfig::Type::ExposeMetrics); + } + + void handleRequest(HTTPServerRequest & /* request */, HTTPServerResponse & response) override + { + response.setContentType("text/plain; version=0.0.4; charset=UTF-8"); + auto & out = getOutputStream(response); + + if (config().expose_events) + metrics_writer().writeEvents(out); + + if (config().expose_metrics) + metrics_writer().writeMetrics(out); + + if (config().expose_asynchronous_metrics) + metrics_writer().writeAsynchronousMetrics(out, parent().async_metrics); + + if (config().expose_errors) + metrics_writer().writeErrors(out); + } +}; + + +PrometheusRequestHandler::PrometheusRequestHandler( + IServer & server_, + const PrometheusRequestHandlerConfig & config_, + const AsynchronousMetrics & async_metrics_, + std::shared_ptr metrics_writer_) + : server(server_) + , config(config_) + , async_metrics(async_metrics_) + , metrics_writer(metrics_writer_) + , log(getLogger("PrometheusRequestHandler")) +{ + createImpl(); +} + +PrometheusRequestHandler::~PrometheusRequestHandler() = default; + +void PrometheusRequestHandler::createImpl() +{ + switch (config.type) + { + case PrometheusRequestHandlerConfig::Type::ExposeMetrics: + { + impl = std::make_unique(*this); + return; + } + } + UNREACHABLE(); +} + +void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event_) { try { - const auto & config = server.config(); - unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT); + write_event = write_event_; + http_method = request.getMethod(); + chassert(!write_buffer_from_response); - /// In order to make keep-alive works. + /// Make keep-alive works. if (request.getVersion() == HTTPServerRequest::HTTP_1_1) response.setChunkedTransferEncoding(true); - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response, config.keep_alive_timeout); - response.setContentType("text/plain; version=0.0.4; charset=UTF-8"); + impl->beforeHandlingRequest(request); + impl->handleRequest(request, response); - WriteBufferFromHTTPServerResponse wb(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout, write_event); - metrics_writer->write(wb); - wb.finalize(); + if (write_buffer_from_response) + { + write_buffer_from_response->finalize(); + write_buffer_from_response = nullptr; + } } catch (...) { - tryLogCurrentException("PrometheusRequestHandler"); + tryLogCurrentException(log); + tryCallOnException(); + + /// `write_buffer_from_response` must be finalized already or at least tried to finalize. + write_buffer_from_response = nullptr; } } -HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactory( - IServer & server, - const Poco::Util::AbstractConfiguration & config, - AsynchronousMetrics & async_metrics, - const std::string & config_prefix) +WriteBuffer & PrometheusRequestHandler::getOutputStream(HTTPServerResponse & response) { - auto writer = std::make_shared(config, config_prefix + ".handler", async_metrics); - auto creator = [&server, writer]() -> std::unique_ptr - { - return std::make_unique(server, writer); - }; - - auto factory = std::make_shared>(std::move(creator)); - factory->addFiltersFromConfig(config, config_prefix); - return factory; + if (write_buffer_from_response) + return *write_buffer_from_response; + write_buffer_from_response = std::make_unique( + response, http_method == HTTPRequest::HTTP_HEAD, config.keep_alive_timeout, write_event); + return *write_buffer_from_response; } -HTTPRequestHandlerFactoryPtr createPrometheusMainHandlerFactory( - IServer & server, const Poco::Util::AbstractConfiguration & config, PrometheusMetricsWriterPtr metrics_writer, const std::string & name) +void PrometheusRequestHandler::tryCallOnException() { - auto factory = std::make_shared(name); - auto creator = [&server, metrics_writer] + try { - return std::make_unique(server, metrics_writer); - }; + if (impl) + impl->onException(); + } + catch (...) + { + tryLogCurrentException(log, "onException"); + } +} - auto handler = std::make_shared>(std::move(creator)); - handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); - handler->allowGetAndHeadRequest(); - factory->addHandler(handler); - return factory; -} } diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index d120752c8c5..c850cc67277 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -1,28 +1,49 @@ #pragma once #include +#include -#include "PrometheusMetricsWriter.h" namespace DB { - +class AsynchronousMetrics; class IServer; +class PrometheusMetricsWriter; +class WriteBufferFromHTTPServerResponse; +/// Handles requests for prometheus protocols (expose_metrics). class PrometheusRequestHandler : public HTTPRequestHandler { -private: - IServer & server; - PrometheusMetricsWriterPtr metrics_writer; - public: - PrometheusRequestHandler(IServer & server_, PrometheusMetricsWriterPtr metrics_writer_) - : server(server_) - , metrics_writer(std::move(metrics_writer_)) - { - } + PrometheusRequestHandler(IServer & server_, const PrometheusRequestHandlerConfig & config_, + const AsynchronousMetrics & async_metrics_, std::shared_ptr metrics_writer_); + ~PrometheusRequestHandler() override; - void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; + void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event_) override; + +private: + /// Creates an internal implementation based on which PrometheusRequestHandlerConfig::Type is used. + void createImpl(); + + /// Returns the write buffer used for the current HTTP response. + WriteBuffer & getOutputStream(HTTPServerResponse & response); + + /// Calls onException() in a try-catch block. + void tryCallOnException(); + + IServer & server; + const PrometheusRequestHandlerConfig config; + const AsynchronousMetrics & async_metrics; + const std::shared_ptr metrics_writer; + const LoggerPtr log; + + class Impl; + class ExposeMetricsImpl; + std::unique_ptr impl; + + String http_method; + std::unique_ptr write_buffer_from_response; + ProfileEvents::Event write_event; }; } diff --git a/src/Server/PrometheusRequestHandlerConfig.h b/src/Server/PrometheusRequestHandlerConfig.h new file mode 100644 index 00000000000..d584e501985 --- /dev/null +++ b/src/Server/PrometheusRequestHandlerConfig.h @@ -0,0 +1,27 @@ +#pragma once + + +namespace DB +{ + +/// Configuration of a Prometheus protocol handler after it's parsed from a configuration file. +struct PrometheusRequestHandlerConfig +{ + enum class Type + { + /// Exposes ClickHouse metrics for scraping by Prometheus. + ExposeMetrics, + }; + + Type type = Type::ExposeMetrics; + + /// Settings for type ExposeMetrics: + bool expose_metrics = false; + bool expose_asynchronous_metrics = false; + bool expose_events = false; + bool expose_errors = false; + + size_t keep_alive_timeout = 0; +}; + +} diff --git a/src/Server/PrometheusRequestHandlerFactory.cpp b/src/Server/PrometheusRequestHandlerFactory.cpp new file mode 100644 index 00000000000..d06aa5fa569 --- /dev/null +++ b/src/Server/PrometheusRequestHandlerFactory.cpp @@ -0,0 +1,151 @@ +#include + +#include +#include +#include +#include + + +namespace DB +{ + +namespace +{ + /// Parses common configuration which is attached to any other configuration. The common configuration looks like this: + /// 30 + void parseCommonConfig(const Poco::Util::AbstractConfiguration & config, PrometheusRequestHandlerConfig & res) + { + res.keep_alive_timeout = config.getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT); + } + + /// Parses a configuration like this: + /// + /// true + /// true + /// true + /// true + PrometheusRequestHandlerConfig parseExposeMetricsConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) + { + PrometheusRequestHandlerConfig res; + res.type = PrometheusRequestHandlerConfig::Type::ExposeMetrics; + res.expose_metrics = config.getBool(config_prefix + ".metrics", true); + res.expose_asynchronous_metrics = config.getBool(config_prefix + ".asynchronous_metrics", true); + res.expose_events = config.getBool(config_prefix + ".events", true); + res.expose_errors = config.getBool(config_prefix + ".errors", true); + parseCommonConfig(config, res); + return res; + } + + /// Returns true if the protocol represented by a passed config can be handled. + bool canBeHandled(const PrometheusRequestHandlerConfig & config, bool for_keeper) + { + /// The standalone ClickHouse Keeper can only expose its metrics. + /// It can't handle other Prometheus protocols. + return !for_keeper || (config.type == PrometheusRequestHandlerConfig::Type::ExposeMetrics); + } + + /// Creates a writer which serializes exposing metrics. + std::shared_ptr createPrometheusMetricWriter(bool for_keeper) + { + if (for_keeper) + return std::make_unique(); + else + return std::make_unique(); + } + + /// Base function for making a factory for PrometheusRequestHandler. This function can return nullptr. + std::shared_ptr> createPrometheusHandlerFactoryFromConfig( + IServer & server, + const AsynchronousMetrics & async_metrics, + const PrometheusRequestHandlerConfig & config, + bool for_keeper) + { + if (!canBeHandled(config, for_keeper)) + return nullptr; + auto metric_writer = createPrometheusMetricWriter(for_keeper); + auto creator = [&server, &async_metrics, config, metric_writer]() -> std::unique_ptr + { + return std::make_unique(server, config, async_metrics, metric_writer); + }; + return std::make_shared>(std::move(creator)); + } + + /// Generic function for createPrometheusHandlerFactory() and createKeeperPrometheusHandlerFactory(). + HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactoryImpl( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const AsynchronousMetrics & asynchronous_metrics, + const String & name, + bool for_keeper) + { + auto factory = std::make_shared(name); + + auto parsed_config = parseExposeMetricsConfig(config, "prometheus"); + if (auto handler = createPrometheusHandlerFactoryFromConfig(server, asynchronous_metrics, parsed_config, for_keeper)) + { + String endpoint = config.getString("prometheus.endpoint", "/metrics"); + handler->attachStrictPath(endpoint); + handler->allowGetAndHeadRequest(); + factory->addHandler(handler); + } + + return factory; + } + +} + + +HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactory( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const AsynchronousMetrics & asynchronous_metrics, + const String & name) +{ + return createPrometheusHandlerFactoryImpl(server, config, asynchronous_metrics, name, /* for_keeper= */ false); +} + + +HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactoryForHTTPRule( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + const AsynchronousMetrics & asynchronous_metrics) +{ + auto parsed_config = parseExposeMetricsConfig(config, config_prefix + ".handler"); + auto handler = createPrometheusHandlerFactoryFromConfig(server, asynchronous_metrics, parsed_config, /* for_keeper= */ false); + chassert(handler); /// `handler` can't be nullptr here because `for_keeper` is false. + handler->addFiltersFromConfig(config, config_prefix); + return handler; +} + + +HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactoryForHTTPRuleDefaults( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const AsynchronousMetrics & asynchronous_metrics) +{ + /// The "defaults" HTTP handler should serve the prometheus exposing metrics protocol on the http port + /// only if it isn't already served on its own port and if there is no section. + if (!config.has("prometheus") || config.getInt("prometheus.port", 0) || config.has("prometheus.handlers")) + return nullptr; + + auto parsed_config = parseExposeMetricsConfig(config, "prometheus"); + String endpoint = config.getString("prometheus.endpoint", "/metrics"); + auto handler = createPrometheusHandlerFactoryFromConfig(server, asynchronous_metrics, parsed_config, /* for_keeper= */ false); + chassert(handler); /// `handler` can't be nullptr here because `for_keeper` is false. + handler->attachStrictPath(endpoint); + handler->allowGetAndHeadRequest(); + return handler; +} + + +HTTPRequestHandlerFactoryPtr createKeeperPrometheusHandlerFactory( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const AsynchronousMetrics & asynchronous_metrics, + const String & name) +{ + return createPrometheusHandlerFactoryImpl(server, config, asynchronous_metrics, name, /* for_keeper= */ true); +} + +} diff --git a/src/Server/PrometheusRequestHandlerFactory.h b/src/Server/PrometheusRequestHandlerFactory.h new file mode 100644 index 00000000000..fb19e9d4070 --- /dev/null +++ b/src/Server/PrometheusRequestHandlerFactory.h @@ -0,0 +1,97 @@ +#pragma once + +#include +#include + + +namespace Poco::Util { class AbstractConfiguration; } + +namespace DB +{ + +class IServer; +class HTTPRequestHandlerFactory; +using HTTPRequestHandlerFactoryPtr = std::shared_ptr; +class AsynchronousMetrics; + +/// Makes a handler factory to handle prometheus protocols. +/// Expects a configuration like this: +/// +/// +/// 1234 +/// /metric +/// true +/// true +/// true +/// true +/// +/// +/// An alternative port to serve prometheus protocols can be specified in the section: +/// +/// +/// +/// 4321 +/// prometheus +/// +/// +HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactory( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const AsynchronousMetrics & asynchronous_metrics, + const String & name); + +/// Makes a HTTP handler factory to handle requests for prometheus metrics for a HTTP rule in the section. +/// Expects a configuration like this: +/// +/// 8123 +/// +/// +/// /metrics +/// +/// prometheus +/// true +/// true +/// true +/// true +/// +/// +/// +HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactoryForHTTPRule( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, /// path to "http_handlers.my_handler_1" + const AsynchronousMetrics & asynchronous_metrics); + +/// Makes a HTTP Handler factory to handle requests for prometheus metrics as a part of the default HTTP rule in the section. +/// Expects a configuration like this: +/// +/// 8123 +/// +/// +/// +/// +/// /metric +/// true +/// true +/// true +/// true +/// +/// +/// The "defaults" HTTP handler should serve the prometheus exposing metrics protocol on the http port +/// only if it isn't already served on its own port , +/// and also if there is no section in the configuration +/// (because if that section exists then it must be in charge of how prometheus protocols are handled). +HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactoryForHTTPRuleDefaults( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const AsynchronousMetrics & asynchronous_metrics); + +/// Makes a handler factory to handle prometheus protocols. +/// Supports the "expose_metrics" protocol only. +HTTPRequestHandlerFactoryPtr createKeeperPrometheusHandlerFactory( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const AsynchronousMetrics & asynchronous_metrics, + const String & name); + +} From 03b93059566690e4f509d7705abe1e670c4f73dd Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 11 Jul 2024 22:45:02 +0200 Subject: [PATCH 269/644] Send exception from PrometheusRequestHandler back to client. Set thread name. --- src/Server/PrometheusRequestHandler.cpp | 32 +++++++++++++++++++ src/Server/PrometheusRequestHandler.h | 4 +++ src/Server/PrometheusRequestHandlerConfig.h | 1 + .../PrometheusRequestHandlerFactory.cpp | 4 +++ 4 files changed, 41 insertions(+) diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index 7aabe07753d..219e6ae55da 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -1,8 +1,10 @@ #include #include +#include #include #include +#include #include #include #include "config.h" @@ -103,6 +105,8 @@ void PrometheusRequestHandler::createImpl() void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event_) { + setThreadName("PrometheusHndlr"); + try { write_event = write_event_; @@ -129,6 +133,10 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe tryLogCurrentException(log); tryCallOnException(); + ExecutionStatus status = ExecutionStatus::fromCurrentException("", send_stacktrace); + trySendExceptionToClient(status.message, status.code, request, response); + tryCallOnException(); + /// `write_buffer_from_response` must be finalized already or at least tried to finalize. write_buffer_from_response = nullptr; } @@ -143,6 +151,30 @@ WriteBuffer & PrometheusRequestHandler::getOutputStream(HTTPServerResponse & res return *write_buffer_from_response; } +void PrometheusRequestHandler::trySendExceptionToClient(const String & exception_message, int exception_code, HTTPServerRequest & request, HTTPServerResponse & response) +{ + try + { + sendExceptionToHTTPClient(exception_message, exception_code, request, response, write_buffer_from_response.get(), log); + } + catch (...) + { + tryLogCurrentException(log, "Couldn't send exception to client"); + + if (write_buffer_from_response) + { + try + { + write_buffer_from_response->finalize(); + } + catch (...) + { + tryLogCurrentException(log, "Cannot flush data to client (after sending exception)"); + } + } + } +} + void PrometheusRequestHandler::tryCallOnException() { try diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index c850cc67277..1d985a23fbe 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -28,6 +28,9 @@ private: /// Returns the write buffer used for the current HTTP response. WriteBuffer & getOutputStream(HTTPServerResponse & response); + /// Writes the current exception to the response. + void trySendExceptionToClient(const String & exception_message, int exception_code, HTTPServerRequest & request, HTTPServerResponse & response); + /// Calls onException() in a try-catch block. void tryCallOnException(); @@ -42,6 +45,7 @@ private: std::unique_ptr impl; String http_method; + bool send_stacktrace = false; std::unique_ptr write_buffer_from_response; ProfileEvents::Event write_event; }; diff --git a/src/Server/PrometheusRequestHandlerConfig.h b/src/Server/PrometheusRequestHandlerConfig.h index d584e501985..bae0a8e1199 100644 --- a/src/Server/PrometheusRequestHandlerConfig.h +++ b/src/Server/PrometheusRequestHandlerConfig.h @@ -22,6 +22,7 @@ struct PrometheusRequestHandlerConfig bool expose_errors = false; size_t keep_alive_timeout = 0; + bool is_stacktrace_enabled = true; }; } diff --git a/src/Server/PrometheusRequestHandlerFactory.cpp b/src/Server/PrometheusRequestHandlerFactory.cpp index d06aa5fa569..35d736f6d3a 100644 --- a/src/Server/PrometheusRequestHandlerFactory.cpp +++ b/src/Server/PrometheusRequestHandlerFactory.cpp @@ -12,9 +12,13 @@ namespace DB namespace { /// Parses common configuration which is attached to any other configuration. The common configuration looks like this: + /// + /// true + /// /// 30 void parseCommonConfig(const Poco::Util::AbstractConfiguration & config, PrometheusRequestHandlerConfig & res) { + res.is_stacktrace_enabled = config.getBool("prometheus.enable_stacktrace", true); res.keep_alive_timeout = config.getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT); } From 7d88995f42bd1a12dd691ac0c67f8609ba6f9d2b Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 11 Jul 2024 22:11:16 +0200 Subject: [PATCH 270/644] Provide alternative way to setup configuration for exposing metrics using new section in configuration. --- .../PrometheusRequestHandlerFactory.cpp | 51 ++++++++++++++++--- src/Server/PrometheusRequestHandlerFactory.h | 19 +++++++ 2 files changed, 64 insertions(+), 6 deletions(-) diff --git a/src/Server/PrometheusRequestHandlerFactory.cpp b/src/Server/PrometheusRequestHandlerFactory.cpp index 35d736f6d3a..00b87515fc8 100644 --- a/src/Server/PrometheusRequestHandlerFactory.cpp +++ b/src/Server/PrometheusRequestHandlerFactory.cpp @@ -9,6 +9,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_ELEMENT_IN_CONFIG; +} + namespace { /// Parses common configuration which is attached to any other configuration. The common configuration looks like this: @@ -40,6 +45,22 @@ namespace return res; } + /// Parses a configuration like this: + /// expose_metrics + /// true + /// true + /// true + /// true + PrometheusRequestHandlerConfig parseHandlerConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) + { + String type = config.getString(config_prefix + ".type"); + + if (type == "expose_metrics") + return parseExposeMetricsConfig(config, config_prefix); + else + throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Unknown type {} is specified in the configuration for a prometheus protocol", type); + } + /// Returns true if the protocol represented by a passed config can be handled. bool canBeHandled(const PrometheusRequestHandlerConfig & config, bool for_keeper) { @@ -84,13 +105,31 @@ namespace { auto factory = std::make_shared(name); - auto parsed_config = parseExposeMetricsConfig(config, "prometheus"); - if (auto handler = createPrometheusHandlerFactoryFromConfig(server, asynchronous_metrics, parsed_config, for_keeper)) + if (config.has("prometheus.handlers")) { - String endpoint = config.getString("prometheus.endpoint", "/metrics"); - handler->attachStrictPath(endpoint); - handler->allowGetAndHeadRequest(); - factory->addHandler(handler); + Strings keys; + config.keys("prometheus.handlers", keys); + for (const String & key : keys) + { + String prefix = "prometheus.handlers." + key; + auto parsed_config = parseHandlerConfig(config, prefix + ".handler"); + if (auto handler = createPrometheusHandlerFactoryFromConfig(server, asynchronous_metrics, parsed_config, for_keeper)) + { + handler->addFiltersFromConfig(config, prefix); + factory->addHandler(handler); + } + } + } + else + { + auto parsed_config = parseExposeMetricsConfig(config, "prometheus"); + if (auto handler = createPrometheusHandlerFactoryFromConfig(server, asynchronous_metrics, parsed_config, for_keeper)) + { + String endpoint = config.getString("prometheus.endpoint", "/metrics"); + handler->attachStrictPath(endpoint); + handler->allowGetAndHeadRequest(); + factory->addHandler(handler); + } } return factory; diff --git a/src/Server/PrometheusRequestHandlerFactory.h b/src/Server/PrometheusRequestHandlerFactory.h index fb19e9d4070..50961ed0bc0 100644 --- a/src/Server/PrometheusRequestHandlerFactory.h +++ b/src/Server/PrometheusRequestHandlerFactory.h @@ -26,6 +26,25 @@ class AsynchronousMetrics; /// true /// /// +/// More prometheus protocols can be supported with using a different configuration +/// (which is similar to the section): +/// +/// +/// 1234 +/// +/// +/// /metrics +/// +/// expose_metrics +/// true +/// true +/// true +/// true +/// +/// +/// +/// +/// /// An alternative port to serve prometheus protocols can be specified in the section: /// /// From 24103c733d868c2ebb64f6096bcf047d8c00d6c2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 27 May 2024 17:19:36 +0200 Subject: [PATCH 271/644] Support prometheus remote write protocol. --- src/CMakeLists.txt | 2 + src/Common/ErrorCodes.cpp | 4 + ...tobufZeroCopyInputStreamFromReadBuffer.cpp | 56 ++ ...rotobufZeroCopyInputStreamFromReadBuffer.h | 38 ++ src/Interpreters/ClientInfo.cpp | 2 + src/Interpreters/ClientInfo.h | 1 + src/Interpreters/SessionLog.cpp | 5 +- src/Server/HTTP/checkHTTPHeader.cpp | 22 + src/Server/HTTP/checkHTTPHeader.h | 13 + src/Server/PrometheusRequestHandler.cpp | 166 ++++++ src/Server/PrometheusRequestHandler.h | 4 +- src/Server/PrometheusRequestHandlerConfig.h | 8 + .../PrometheusRequestHandlerFactory.cpp | 34 ++ src/Server/PrometheusRequestHandlerFactory.h | 7 + src/Storages/StorageTimeSeries.cpp | 24 + src/Storages/StorageTimeSeries.h | 4 + .../PrometheusRemoteWriteProtocol.cpp | 538 ++++++++++++++++++ .../PrometheusRemoteWriteProtocol.h | 35 ++ src/Storages/TimeSeries/TimeSeriesTagNames.h | 13 + 19 files changed, 973 insertions(+), 3 deletions(-) create mode 100644 src/IO/Protobuf/ProtobufZeroCopyInputStreamFromReadBuffer.cpp create mode 100644 src/IO/Protobuf/ProtobufZeroCopyInputStreamFromReadBuffer.h create mode 100644 src/Server/HTTP/checkHTTPHeader.cpp create mode 100644 src/Server/HTTP/checkHTTPHeader.h create mode 100644 src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp create mode 100644 src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.h create mode 100644 src/Storages/TimeSeries/TimeSeriesTagNames.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index cc10fdf9646..98dd0601a1b 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -78,6 +78,7 @@ add_headers_and_sources(clickhouse_common_io Common/Scheduler) add_headers_and_sources(clickhouse_common_io Common/Scheduler/Nodes) add_headers_and_sources(clickhouse_common_io IO) add_headers_and_sources(clickhouse_common_io IO/Archives) +add_headers_and_sources(clickhouse_common_io IO/Protobuf) add_headers_and_sources(clickhouse_common_io IO/S3) add_headers_and_sources(clickhouse_common_io IO/AzureBlobStorage) list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp) @@ -470,6 +471,7 @@ dbms_target_link_libraries (PUBLIC ch_contrib::sparsehash) if (TARGET ch_contrib::protobuf) dbms_target_link_libraries (PRIVATE ch_contrib::protobuf) + target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::protobuf) endif () if (TARGET clickhouse_grpc_protos) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index b1b8e2367a4..44a1cd071cb 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -604,6 +604,10 @@ M(723, PARQUET_EXCEPTION) \ M(724, TOO_MANY_TABLES) \ M(725, TOO_MANY_DATABASES) \ + M(726, UNEXPECTED_HTTP_HEADERS) \ + M(727, UNEXPECTED_TABLE_ENGINE) \ + M(728, UNEXPECTED_DATA_TYPE) \ + M(729, ILLEGAL_TIME_SERIES_TAGS) \ \ M(900, DISTRIBUTED_CACHE_ERROR) \ M(901, CANNOT_USE_DISTRIBUTED_CACHE) \ diff --git a/src/IO/Protobuf/ProtobufZeroCopyInputStreamFromReadBuffer.cpp b/src/IO/Protobuf/ProtobufZeroCopyInputStreamFromReadBuffer.cpp new file mode 100644 index 00000000000..86b7eb4d7f7 --- /dev/null +++ b/src/IO/Protobuf/ProtobufZeroCopyInputStreamFromReadBuffer.cpp @@ -0,0 +1,56 @@ +#include "config.h" + +#if USE_PROTOBUF +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +ProtobufZeroCopyInputStreamFromReadBuffer::ProtobufZeroCopyInputStreamFromReadBuffer(std::unique_ptr in_) : in(std::move(in_)) +{ +} + +ProtobufZeroCopyInputStreamFromReadBuffer::~ProtobufZeroCopyInputStreamFromReadBuffer() = default; + +bool ProtobufZeroCopyInputStreamFromReadBuffer::Next(const void ** data, int * size) +{ + if (in->eof()) + return false; + *data = in->position(); + *size = static_cast(in->available()); + in->position() += *size; + return true; +} + +void ProtobufZeroCopyInputStreamFromReadBuffer::BackUp(int count) +{ + if (static_cast(in->offset()) < count) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "ProtobufZeroCopyInputStreamFromReadBuffer::BackUp() cannot back up {} bytes (max = {} bytes)", + count, + in->offset()); + + in->position() -= count; +} + +bool ProtobufZeroCopyInputStreamFromReadBuffer::Skip(int count) +{ + return static_cast(in->tryIgnore(count)) == count; +} + +int64_t ProtobufZeroCopyInputStreamFromReadBuffer::ByteCount() const +{ + return in->count(); +} + +} + +#endif diff --git a/src/IO/Protobuf/ProtobufZeroCopyInputStreamFromReadBuffer.h b/src/IO/Protobuf/ProtobufZeroCopyInputStreamFromReadBuffer.h new file mode 100644 index 00000000000..3f86815ef3f --- /dev/null +++ b/src/IO/Protobuf/ProtobufZeroCopyInputStreamFromReadBuffer.h @@ -0,0 +1,38 @@ +#pragma once + +#include "config.h" +#if USE_PROTOBUF + +#include + + +namespace DB +{ +class ReadBuffer; + +class ProtobufZeroCopyInputStreamFromReadBuffer : public google::protobuf::io::ZeroCopyInputStream +{ +public: + explicit ProtobufZeroCopyInputStreamFromReadBuffer(std::unique_ptr in_); + ~ProtobufZeroCopyInputStreamFromReadBuffer() override; + + // Obtains a chunk of data from the stream. + bool Next(const void ** data, int * size) override; + + // Backs up a number of bytes, so that the next call to Next() returns + // data again that was already returned by the last call to Next(). + void BackUp(int count) override; + + // Skips a number of bytes. + bool Skip(int count) override; + + // Returns the total number of bytes read since this object was created. + int64_t ByteCount() const override; + +private: + std::unique_ptr in; +}; + +} + +#endif diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index 39fdef23baa..daf1e300046 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -254,6 +254,8 @@ String toString(ClientInfo::Interface interface) return "LOCAL"; case ClientInfo::Interface::TCP_INTERSERVER: return "TCP_INTERSERVER"; + case ClientInfo::Interface::PROMETHEUS: + return "PROMETHEUS"; } return std::format("Unknown server interface ({}).", static_cast(interface)); diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index ca32b4c5cfa..48dea3cc3ea 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -38,6 +38,7 @@ public: POSTGRESQL = 5, LOCAL = 6, TCP_INTERSERVER = 7, + PROMETHEUS = 8, }; enum class HTTPMethod : uint8_t diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index 0615a2a1d62..866f5ba8c0a 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -105,9 +105,10 @@ ColumnsDescription SessionLogElement::getColumnsDescription() {"MySQL", static_cast(Interface::MYSQL)}, {"PostgreSQL", static_cast(Interface::POSTGRESQL)}, {"Local", static_cast(Interface::LOCAL)}, - {"TCP_Interserver", static_cast(Interface::TCP_INTERSERVER)} + {"TCP_Interserver", static_cast(Interface::TCP_INTERSERVER)}, + {"Prometheus", static_cast(Interface::PROMETHEUS)}, }); - static_assert(magic_enum::enum_count() == 7); + static_assert(magic_enum::enum_count() == 8); auto lc_string_datatype = std::make_shared(std::make_shared()); diff --git a/src/Server/HTTP/checkHTTPHeader.cpp b/src/Server/HTTP/checkHTTPHeader.cpp new file mode 100644 index 00000000000..812adde022a --- /dev/null +++ b/src/Server/HTTP/checkHTTPHeader.cpp @@ -0,0 +1,22 @@ +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNEXPECTED_HTTP_HEADERS; +} + +void checkHTTPHeader(const HTTPRequest & request, const String & header_name, const String & expected_value) +{ + if (!request.has(header_name)) + throw Exception(ErrorCodes::UNEXPECTED_HTTP_HEADERS, "No HTTP header {}", header_name); + if (request.get(header_name) != expected_value) + throw Exception(ErrorCodes::UNEXPECTED_HTTP_HEADERS, "HTTP header {} has unexpected value '{}' (instead of '{}')", header_name, request.get(header_name), expected_value); +} + +} diff --git a/src/Server/HTTP/checkHTTPHeader.h b/src/Server/HTTP/checkHTTPHeader.h new file mode 100644 index 00000000000..956599ae66b --- /dev/null +++ b/src/Server/HTTP/checkHTTPHeader.h @@ -0,0 +1,13 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/// Checks that the HTTP request has a specified header with a specified value. +void checkHTTPHeader(const HTTPRequest & request, const String & header_name, const String & expected_value); + +} diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index 219e6ae55da..98d652540d3 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -9,6 +9,19 @@ #include #include "config.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + namespace DB { @@ -74,6 +87,154 @@ public: }; +/// Base implementation of a protocol with Context and authentication. +class PrometheusRequestHandler::ImplWithContext : public Impl +{ +public: + explicit ImplWithContext(PrometheusRequestHandler & parent) : Impl(parent), default_settings(parent.server.context()->getSettingsRef()) { } + + virtual void handlingRequestWithContext(HTTPServerRequest & request, HTTPServerResponse & response) = 0; + +protected: + void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override + { + SCOPE_EXIT({ + request_credentials.reset(); + context.reset(); + session.reset(); + params.reset(); + }); + + params = std::make_unique(default_settings, request); + parent().send_stacktrace = config().is_stacktrace_enabled && params->getParsed("stacktrace", false); + + if (!authenticateUserAndMakeContext(request, response)) + return; /// The user is not authenticated yet, and the HTTP_UNAUTHORIZED response is sent with the "WWW-Authenticate" header, + /// and `request_credentials` must be preserved until the next request or until any exception. + + /// Initialize query scope. + std::optional query_scope; + if (context) + query_scope.emplace(context); + + handlingRequestWithContext(request, response); + } + + bool authenticateUserAndMakeContext(HTTPServerRequest & request, HTTPServerResponse & response) + { + session = std::make_unique(server().context(), ClientInfo::Interface::PROMETHEUS, request.isSecure()); + + if (!authenticateUser(request, response)) + return false; + + makeContext(request); + return true; + } + + bool authenticateUser(HTTPServerRequest & request, HTTPServerResponse & response) + { + return authenticateUserByHTTP(request, *params, response, *session, request_credentials, server().context(), log()); + } + + void makeContext(HTTPServerRequest & request) + { + context = session->makeQueryContext(); + + /// Anything else beside HTTP POST should be readonly queries. + setReadOnlyIfHTTPMethodIdempotent(context, request.getMethod()); + + auto roles = params->getAll("role"); + if (!roles.empty()) + context->setCurrentRoles(roles); + + auto param_could_be_skipped = [&] (const String & name) + { + /// Empty parameter appears when URL like ?&a=b or a=b&&c=d. Just skip them for user's convenience. + if (name.empty()) + return true; + + /// Some parameters (database, default_format, everything used in the code above) do not + /// belong to the Settings class. + static const NameSet reserved_param_names{"user", "password", "quota_key", "stacktrace", "role", "query_id"}; + return reserved_param_names.contains(name); + }; + + /// Settings can be overridden in the query. + SettingsChanges settings_changes; + for (const auto & [key, value] : *params) + { + if (!param_could_be_skipped(key)) + { + /// Other than query parameters are treated as settings. + settings_changes.push_back({key, value}); + } + } + + context->checkSettingsConstraints(settings_changes, SettingSource::QUERY); + context->applySettingsChanges(settings_changes); + + /// Set the query id supplied by the user, if any, and also update the OpenTelemetry fields. + context->setCurrentQueryId(params->get("query_id", request.get("X-ClickHouse-Query-Id", ""))); + } + + void onException() override + { + // So that the next requests on the connection have to always start afresh in case of exceptions. + request_credentials.reset(); + } + + const Settings & default_settings; + std::unique_ptr params; + std::unique_ptr session; + std::unique_ptr request_credentials; + ContextMutablePtr context; +}; + + +/// Implementation of the remote-write protocol. +class PrometheusRequestHandler::RemoteWriteImpl : public ImplWithContext +{ +public: + using ImplWithContext::ImplWithContext; + + void beforeHandlingRequest(HTTPServerRequest & request) override + { + LOG_INFO(log(), "Handling remote write request from {}", request.get("User-Agent", "")); + chassert(config().type == PrometheusRequestHandlerConfig::Type::RemoteWrite); + } + + void handlingRequestWithContext([[maybe_unused]] HTTPServerRequest & request, [[maybe_unused]] HTTPServerResponse & response) override + { +#if USE_PROMETHEUS_PROTOBUFS + checkHTTPHeader(request, "Content-Type", "application/x-protobuf"); + checkHTTPHeader(request, "Content-Encoding", "snappy"); + + ProtobufZeroCopyInputStreamFromReadBuffer zero_copy_input_stream{ + std::make_unique(wrapReadBufferReference(request.getStream()))}; + + prometheus::WriteRequest write_request; + if (!write_request.ParsePartialFromZeroCopyStream(&zero_copy_input_stream)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse WriteRequest"); + + auto table = DatabaseCatalog::instance().getTable(StorageID{config().time_series_table_name}, context); + PrometheusRemoteWriteProtocol protocol{table, context}; + + if (write_request.timeseries_size()) + protocol.writeTimeSeries(write_request.timeseries()); + + if (write_request.metadata_size()) + protocol.writeMetricsMetadata(write_request.metadata()); + + response.setContentType("text/plain; charset=UTF-8"); + response.send(); + +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Prometheus remote write protocol is disabled"); +#endif + } +}; + + PrometheusRequestHandler::PrometheusRequestHandler( IServer & server_, const PrometheusRequestHandlerConfig & config_, @@ -99,6 +260,11 @@ void PrometheusRequestHandler::createImpl() impl = std::make_unique(*this); return; } + case PrometheusRequestHandlerConfig::Type::RemoteWrite: + { + impl = std::make_unique(*this); + return; + } } UNREACHABLE(); } diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index 1d985a23fbe..b4d1e849bdd 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -11,7 +11,7 @@ class IServer; class PrometheusMetricsWriter; class WriteBufferFromHTTPServerResponse; -/// Handles requests for prometheus protocols (expose_metrics). +/// Handles requests for prometheus protocols (expose_metrics, remote_write). class PrometheusRequestHandler : public HTTPRequestHandler { public: @@ -41,7 +41,9 @@ private: const LoggerPtr log; class Impl; + class ImplWithContext; class ExposeMetricsImpl; + class RemoteWriteImpl; std::unique_ptr impl; String http_method; diff --git a/src/Server/PrometheusRequestHandlerConfig.h b/src/Server/PrometheusRequestHandlerConfig.h index bae0a8e1199..d8fd03f19b2 100644 --- a/src/Server/PrometheusRequestHandlerConfig.h +++ b/src/Server/PrometheusRequestHandlerConfig.h @@ -1,5 +1,7 @@ #pragma once +#include + namespace DB { @@ -11,6 +13,9 @@ struct PrometheusRequestHandlerConfig { /// Exposes ClickHouse metrics for scraping by Prometheus. ExposeMetrics, + + /// Handles Prometheus remote-write protocol. + RemoteWrite, }; Type type = Type::ExposeMetrics; @@ -21,6 +26,9 @@ struct PrometheusRequestHandlerConfig bool expose_events = false; bool expose_errors = false; + /// Settings for types RemoteWrite, RemoteRead: + QualifiedTableName time_series_table_name; + size_t keep_alive_timeout = 0; bool is_stacktrace_enabled = true; }; diff --git a/src/Server/PrometheusRequestHandlerFactory.cpp b/src/Server/PrometheusRequestHandlerFactory.cpp index 00b87515fc8..d4b1ab6cd93 100644 --- a/src/Server/PrometheusRequestHandlerFactory.cpp +++ b/src/Server/PrometheusRequestHandlerFactory.cpp @@ -45,18 +45,52 @@ namespace return res; } + /// Extracts a qualified table name from the config. It can be set either as + ///
mydb.prometheus
+ /// or + /// mydb + /// prometheus
+ QualifiedTableName parseTableNameFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) + { + QualifiedTableName res; + res.table = config.getString(config_prefix + ".table", "prometheus"); + res.database = config.getString(config_prefix + ".database", ""); + if (res.database.empty()) + res = QualifiedTableName::parseFromString(res.table); + if (res.database.empty()) + res.database = "default"; + return res; + } + + /// Parses a configuration like this: + /// + /// db.time_series_table_name
+ PrometheusRequestHandlerConfig parseRemoteWriteConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) + { + PrometheusRequestHandlerConfig res; + res.type = PrometheusRequestHandlerConfig::Type::RemoteWrite; + res.time_series_table_name = parseTableNameFromConfig(config, config_prefix); + parseCommonConfig(config, res); + return res; + } + /// Parses a configuration like this: /// expose_metrics /// true /// true /// true /// true + /// -OR- + /// remote_write + /// db.time_series_table_name
PrometheusRequestHandlerConfig parseHandlerConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { String type = config.getString(config_prefix + ".type"); if (type == "expose_metrics") return parseExposeMetricsConfig(config, config_prefix); + else if (type == "remote_write") + return parseRemoteWriteConfig(config, config_prefix); else throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Unknown type {} is specified in the configuration for a prometheus protocol", type); } diff --git a/src/Server/PrometheusRequestHandlerFactory.h b/src/Server/PrometheusRequestHandlerFactory.h index 50961ed0bc0..a7227b12f7e 100644 --- a/src/Server/PrometheusRequestHandlerFactory.h +++ b/src/Server/PrometheusRequestHandlerFactory.h @@ -74,6 +74,13 @@ HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactory( /// true /// /// +/// +/// /write +/// +/// remote_write +/// db.time_series_table_name
+///
+///
/// HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactoryForHTTPRule( IServer & server, diff --git a/src/Storages/StorageTimeSeries.cpp b/src/Storages/StorageTimeSeries.cpp index d85db53d78d..e2ef586c3ca 100644 --- a/src/Storages/StorageTimeSeries.cpp +++ b/src/Storages/StorageTimeSeries.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes extern const int INCORRECT_QUERY; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; + extern const int UNEXPECTED_TABLE_ENGINE; } @@ -427,6 +428,29 @@ SinkToStoragePtr StorageTimeSeries::write(const ASTPtr & query, const StorageMet } +std::shared_ptr storagePtrToTimeSeries(StoragePtr storage) +{ + if (auto res = typeid_cast>(storage)) + return res; + + throw Exception( + ErrorCodes::UNEXPECTED_TABLE_ENGINE, + "This operation can be executed on a TimeSeries table only, the engine of table {} is not TimeSeries", + storage->getStorageID().getNameForLogs()); +} + +std::shared_ptr storagePtrToTimeSeries(ConstStoragePtr storage) +{ + if (auto res = typeid_cast>(storage)) + return res; + + throw Exception( + ErrorCodes::UNEXPECTED_TABLE_ENGINE, + "This operation can be executed on a TimeSeries table only, the engine of table {} is not TimeSeries", + storage->getStorageID().getNameForLogs()); +} + + void registerStorageTimeSeries(StorageFactory & factory) { factory.registerStorage("TimeSeries", [](const StorageFactory::Arguments & args) diff --git a/src/Storages/StorageTimeSeries.h b/src/Storages/StorageTimeSeries.h index 9ee09108803..35db3131a0b 100644 --- a/src/Storages/StorageTimeSeries.h +++ b/src/Storages/StorageTimeSeries.h @@ -2,6 +2,7 @@ #include #include +#include #include @@ -104,4 +105,7 @@ private: bool has_inner_tables; }; +std::shared_ptr storagePtrToTimeSeries(StoragePtr storage); +std::shared_ptr storagePtrToTimeSeries(ConstStoragePtr storage); + } diff --git a/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp new file mode 100644 index 00000000000..b3845e88406 --- /dev/null +++ b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp @@ -0,0 +1,538 @@ +#include + +#include "config.h" +#if USE_PROMETHEUS_PROTOBUFS + +#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 ILLEGAL_TIME_SERIES_TAGS; + extern const int ILLEGAL_COLUMN; +} + + +namespace +{ + /// Checks that a specified set of labels is sorted and has no duplications, and there is one label named "__name__". + void checkLabels(const ::google::protobuf::RepeatedPtrField<::prometheus::Label> & labels) + { + bool metric_name_found = false; + for (size_t i = 0; i != static_cast(labels.size()); ++i) + { + const auto & label = labels[static_cast(i)]; + const auto & label_name = label.name(); + const auto & label_value = label.value(); + + if (label_name.empty()) + throw Exception(ErrorCodes::ILLEGAL_TIME_SERIES_TAGS, "Label name should not be empty"); + if (label_value.empty()) + continue; /// Empty label value is treated like the label doesn't exist. + + if (label_name == TimeSeriesTagNames::MetricName) + metric_name_found = true; + + if (i) + { + /// Check that labels are sorted. + const auto & previous_label_name = labels[static_cast(i - 1)].name(); + if (label_name <= previous_label_name) + { + if (label_name == previous_label_name) + throw Exception(ErrorCodes::ILLEGAL_TIME_SERIES_TAGS, "Found duplicate label {}", label_name); + else + throw Exception(ErrorCodes::ILLEGAL_TIME_SERIES_TAGS, "Label names are not sorted in lexicographical order ({} > {})", + previous_label_name, label_name); + } + } + } + + if (!metric_name_found) + throw Exception(ErrorCodes::ILLEGAL_TIME_SERIES_TAGS, "Metric name (label {}) not found", TimeSeriesTagNames::MetricName); + } + + /// Finds the description of an insertable column in the list. + const ColumnDescription & getInsertableColumnDescription(const ColumnsDescription & columns, const String & column_name, const StorageID & time_series_storage_id) + { + const ColumnDescription * column = columns.tryGet(column_name); + if (!column || ((column->default_desc.kind != ColumnDefaultKind::Default) && (column->default_desc.kind != ColumnDefaultKind::Ephemeral))) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "{}: Column {} {}", + time_series_storage_id.getNameForLogs(), column_name, column ? "non-insertable" : "doesn't exist"); + } + return *column; + } + + /// Calculates the identifier of each time series in "tags_block" using the default expression for the "id" column, + /// and adds column "id" with the results to "tags_block". + IColumn & calculateId(const ContextPtr & context, const ColumnDescription & id_column_description, Block & tags_block) + { + auto blocks = std::make_shared(); + blocks->push_back(tags_block); + + auto header = tags_block.cloneEmpty(); + auto pipe = Pipe(std::make_shared(blocks, header)); + + Block header_with_id; + const auto & id_name = id_column_description.name; + auto id_type = id_column_description.type; + header_with_id.insert(ColumnWithTypeAndName{id_type, id_name}); + + auto adding_missing_defaults_dag = addMissingDefaults( + pipe.getHeader(), + header_with_id.getNamesAndTypesList(), + ColumnsDescription{id_column_description}, + context); + + auto adding_missing_defaults_actions = std::make_shared(adding_missing_defaults_dag); + pipe.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(stream_header, adding_missing_defaults_actions); + }); + + auto convert_actions_dag = ActionsDAG::makeConvertingActions( + pipe.getHeader().getColumnsWithTypeAndName(), + header_with_id.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + auto actions = std::make_shared( + convert_actions_dag, + ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)); + pipe.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(stream_header, actions); + }); + + QueryPipeline pipeline{std::move(pipe)}; + PullingPipelineExecutor executor{pipeline}; + + MutableColumnPtr id_column; + + Block block_from_executor; + while (executor.pull(block_from_executor)) + { + if (block_from_executor) + { + MutableColumnPtr id_column_part = block_from_executor.getByName(id_name).column->assumeMutable(); + if (id_column) + id_column->insertRangeFrom(*id_column_part, 0, id_column_part->size()); + else + id_column = std::move(id_column_part); + } + } + + if (!id_column) + id_column = id_type->createColumn(); + + IColumn & id_column_ref = *id_column; + tags_block.insert(0, ColumnWithTypeAndName{std::move(id_column), id_type, id_name}); + return id_column_ref; + } + + /// Converts a timestamp in milliseconds to a DateTime64 with a specified scale. + DateTime64 scaleTimestamp(Int64 timestamp_ms, UInt32 scale) + { + if (scale == 3) + return timestamp_ms; + else if (scale > 3) + return timestamp_ms * DecimalUtils::scaleMultiplier(scale - 3); + else + return timestamp_ms / DecimalUtils::scaleMultiplier(3 - scale); + } + + struct BlocksToInsert + { + std::vector> blocks; + }; + + /// Converts time series from the protobuf format to prepared blocks for inserting into target tables. + BlocksToInsert toBlocks(const google::protobuf::RepeatedPtrField & time_series, + const ContextPtr & context, + const StorageID & time_series_storage_id, + const StorageInMemoryMetadata & time_series_storage_metadata, + const TimeSeriesSettings & time_series_settings) + { + size_t num_tags_rows = time_series.size(); + + size_t num_data_rows = 0; + for (const auto & element : time_series) + num_data_rows += element.samples_size(); + + if (!num_data_rows) + return {}; /// Nothing to insert into target tables. + + /// Column types must be extracted from the target tables' metadata. + const auto & columns_description = time_series_storage_metadata.columns; + + auto get_column_description = [&](const String & column_name) -> const ColumnDescription & + { + return getInsertableColumnDescription(columns_description, column_name, time_series_storage_id); + }; + + /// We're going to prepare two blocks - one for the "data" table, and one for the "tags" table. + Block data_block, tags_block; + + auto make_column_for_data_block = [&](const ColumnDescription & column_description) -> IColumn & + { + auto column = column_description.type->createColumn(); + column->reserve(num_data_rows); + auto * column_ptr = column.get(); + data_block.insert(ColumnWithTypeAndName{std::move(column), column_description.type, column_description.name}); + return *column_ptr; + }; + + auto make_column_for_tags_block = [&](const ColumnDescription & column_description) -> IColumn & + { + auto column = column_description.type->createColumn(); + column->reserve(num_tags_rows); + auto * column_ptr = column.get(); + tags_block.insert(ColumnWithTypeAndName{std::move(column), column_description.type, column_description.name}); + return *column_ptr; + }; + + /// Create columns. + + /// Column "id". + const auto & id_description = get_column_description(TimeSeriesColumnNames::ID); + TimeSeriesColumnsValidator validator{time_series_storage_id, time_series_settings}; + validator.validateColumnForID(id_description); + auto & id_column_in_data_table = make_column_for_data_block(id_description); + + /// Column "timestamp". + const auto & timestamp_description = get_column_description(TimeSeriesColumnNames::Timestamp); + UInt32 timestamp_scale; + validator.validateColumnForTimestamp(timestamp_description, timestamp_scale); + auto & timestamp_column = make_column_for_data_block(timestamp_description); + + /// Column "value". + const auto & value_description = get_column_description(TimeSeriesColumnNames::Value); + validator.validateColumnForValue(value_description); + auto & value_column = make_column_for_data_block(value_description); + + /// Column "metric_name". + const auto & metric_name_description = get_column_description(TimeSeriesColumnNames::MetricName); + validator.validateColumnForMetricName(metric_name_description); + auto & metric_name_column = make_column_for_tags_block(metric_name_description); + + /// Columns we should check explicitly that they're filled after filling each row. + std::vector columns_to_fill_in_tags_table; + + /// Columns corresponding to specific tags specified in the "tags_to_columns" setting. + std::unordered_map columns_by_tag_name; + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & tag_name = tuple.at(0).safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + const auto & column_description = get_column_description(column_name); + validator.validateColumnForTagValue(column_description); + auto & column = make_column_for_tags_block(column_description); + columns_by_tag_name[tag_name] = &column; + columns_to_fill_in_tags_table.emplace_back(&column); + } + + /// Column "tags". + const auto & tags_description = get_column_description(TimeSeriesColumnNames::Tags); + validator.validateColumnForTagsMap(tags_description); + auto & tags_column = typeid_cast(make_column_for_tags_block(tags_description)); + IColumn & tags_names = tags_column.getNestedData().getColumn(0); + IColumn & tags_values = tags_column.getNestedData().getColumn(1); + auto & tags_offsets = tags_column.getNestedColumn().getOffsets(); + + /// Column "all_tags". + const auto & all_tags_description = get_column_description(TimeSeriesColumnNames::AllTags); + validator.validateColumnForTagsMap(all_tags_description); + auto & all_tags_column = typeid_cast(make_column_for_tags_block(all_tags_description)); + IColumn & all_tags_names = all_tags_column.getNestedData().getColumn(0); + IColumn & all_tags_values = all_tags_column.getNestedData().getColumn(1); + auto & all_tags_offsets = all_tags_column.getNestedColumn().getOffsets(); + + /// Prepare a block for inserting into the "tags" table. + size_t current_row_in_tags = 0; + for (size_t i = 0; i != static_cast(time_series.size()); ++i) + { + const auto & element = time_series[static_cast(i)]; + if (!element.samples_size()) + continue; + + const auto & labels = element.labels(); + checkLabels(labels); + + for (size_t j = 0; j != static_cast(labels.size()); ++j) + { + const auto & label = labels[static_cast(j)]; + const auto & tag_name = label.name(); + const auto & tag_value = label.value(); + + if (tag_name == TimeSeriesTagNames::MetricName) + { + metric_name_column.insertData(tag_value.data(), tag_value.length()); + } + else + { + all_tags_names.insertData(tag_name.data(), tag_name.length()); + all_tags_values.insertData(tag_value.data(), tag_value.length()); + + auto it = columns_by_tag_name.find(tag_name); + bool has_column_for_tag_value = (it != columns_by_tag_name.end()); + if (has_column_for_tag_value) + { + auto * column = it->second; + column->insertData(tag_value.data(), tag_value.length()); + } + else + { + tags_names.insertData(tag_name.data(), tag_name.length()); + tags_values.insertData(tag_value.data(), tag_value.length()); + } + } + } + + all_tags_offsets.push_back(all_tags_names.size()); + tags_offsets.push_back(tags_names.size()); + + for (auto * column : columns_to_fill_in_tags_table) + { + if (column->size() == current_row_in_tags) + column->insertDefault(); + } + + ++current_row_in_tags; + } + + /// Calculate an identifier for each time series, make a new column from those identifiers, and add it to "tags_block". + auto & id_column_in_tags_table = calculateId(context, columns_description.get(TimeSeriesColumnNames::ID), tags_block); + + /// Prepare a block for inserting to the "data" table. + current_row_in_tags = 0; + for (size_t i = 0; i != static_cast(time_series.size()); ++i) + { + const auto & element = time_series[static_cast(i)]; + if (!element.samples_size()) + continue; + + id_column_in_data_table.insertManyFrom(id_column_in_tags_table, current_row_in_tags, element.samples_size()); + for (const auto & sample : element.samples()) + { + timestamp_column.insert(scaleTimestamp(sample.timestamp(), timestamp_scale)); + value_column.insert(sample.value()); + } + + ++current_row_in_tags; + } + + /// The "all_tags" column in the "tags" table is either ephemeral or doesn't exists. + /// We've used the "all_tags" column to calculate the "id" column already, + /// and now we don't need it to insert to the "tags" table. + tags_block.erase(TimeSeriesColumnNames::AllTags); + + BlocksToInsert res; + + /// A block to the "tags" table should be inserted first. + /// (Because any INSERT can fail and we don't want to have rows in the data table with no corresponding "id" written to the "tags" table.) + res.blocks.emplace_back(ViewTarget::Tags, std::move(tags_block)); + res.blocks.emplace_back(ViewTarget::Data, std::move(data_block)); + + return res; + } + + std::string_view metricTypeToString(prometheus::MetricMetadata::MetricType metric_type) + { + using namespace std::literals; + switch (metric_type) + { + case prometheus::MetricMetadata::UNKNOWN: return "unknown"sv; + case prometheus::MetricMetadata::COUNTER: return "counter"sv; + case prometheus::MetricMetadata::GAUGE: return "gauge"sv; + case prometheus::MetricMetadata::HISTOGRAM: return "histogram"sv; + case prometheus::MetricMetadata::GAUGEHISTOGRAM: return "gaugehistogram"sv; + case prometheus::MetricMetadata::SUMMARY: return "summary"sv; + case prometheus::MetricMetadata::INFO: return "info"sv; + case prometheus::MetricMetadata::STATESET: return "stateset"sv; + default: break; + } + return ""; + } + + /// Converts metrics metadata from the protobuf format to prepared blocks for inserting into target tables. + BlocksToInsert toBlocks(const google::protobuf::RepeatedPtrField & metrics_metadata, + const StorageID & time_series_storage_id, + const StorageInMemoryMetadata & time_series_storage_metadata, + const TimeSeriesSettings & time_series_settings) + { + size_t num_rows = metrics_metadata.size(); + + if (!num_rows) + return {}; /// Nothing to insert into target tables. + + /// Column types must be extracted from the target tables' metadata. + const auto & columns_description = time_series_storage_metadata.columns; + + auto get_column_description = [&](const String & column_name) -> const ColumnDescription & + { + return getInsertableColumnDescription(columns_description, column_name, time_series_storage_id); + }; + + /// We're going to prepare one blocks for the "metrics" table. + Block block; + + auto make_column = [&](const ColumnDescription & column_description) -> IColumn & + { + auto column = column_description.type->createColumn(); + column->reserve(num_rows); + auto * column_ptr = column.get(); + block.insert(ColumnWithTypeAndName{std::move(column), column_description.type, column_description.name}); + return *column_ptr; + }; + + /// Create columns. + + /// Column "metric_family_name". + const auto & metric_family_name_description = get_column_description(TimeSeriesColumnNames::MetricFamilyName); + TimeSeriesColumnsValidator validator{time_series_storage_id, time_series_settings}; + validator.validateColumnForMetricFamilyName(metric_family_name_description); + auto & metric_family_name_column = make_column(metric_family_name_description); + + /// Column "type". + const auto & type_description = get_column_description(TimeSeriesColumnNames::Type); + validator.validateColumnForType(type_description); + auto & type_column = make_column(type_description); + + /// Column "unit". + const auto & unit_description = get_column_description(TimeSeriesColumnNames::Unit); + validator.validateColumnForUnit(unit_description); + auto & unit_column = make_column(unit_description); + + /// Column "help". + const auto & help_description = get_column_description(TimeSeriesColumnNames::Help); + validator.validateColumnForHelp(help_description); + auto & help_column = make_column(help_description); + + /// Fill those columns. + for (const auto & element : metrics_metadata) + { + const auto & metric_family_name = element.metric_family_name(); + const auto & type_str = metricTypeToString(element.type()); + const auto & help = element.help(); + const auto & unit = element.unit(); + + metric_family_name_column.insertData(metric_family_name.data(), metric_family_name.length()); + type_column.insertData(type_str.data(), type_str.length()); + unit_column.insertData(unit.data(), unit.length()); + help_column.insertData(help.data(), help.length()); + } + + /// Prepare a result. + BlocksToInsert res; + res.blocks.emplace_back(ViewTarget::Metrics, std::move(block)); + return res; + } + + /// Inserts blocks to target tables. + void insertToTargetTables(BlocksToInsert && blocks, StorageTimeSeries & time_series_storage, ContextPtr context, Poco::Logger * log) + { + auto time_series_storage_id = time_series_storage.getStorageID(); + + for (auto & [table_kind, block] : blocks.blocks) + { + if (block) + { + const auto & target_table_id = time_series_storage.getTargetTableId(table_kind); + + LOG_INFO(log, "{}: Inserting {} rows to the {} table", + time_series_storage_id.getNameForLogs(), block.rows(), toString(table_kind)); + + auto insert_query = std::make_shared(); + insert_query->table_id = target_table_id; + + auto columns_ast = std::make_shared(); + for (const auto & name : block.getNames()) + columns_ast->children.emplace_back(std::make_shared(name)); + insert_query->columns = columns_ast; + + ContextMutablePtr insert_context = Context::createCopy(context); + insert_context->setCurrentQueryId(context->getCurrentQueryId() + ":" + String{toString(table_kind)}); + + InterpreterInsertQuery interpreter(insert_query, insert_context); + BlockIO io = interpreter.execute(); + PushingPipelineExecutor executor(io.pipeline); + + executor.start(); + executor.push(std::move(block)); + executor.finish(); + } + } + } +} + + +PrometheusRemoteWriteProtocol::PrometheusRemoteWriteProtocol(StoragePtr time_series_storage_, const ContextPtr & context_) + : WithContext(context_) + , time_series_storage(storagePtrToTimeSeries(time_series_storage_)) + , log(getLogger("PrometheusRemoteWriteProtocol")) +{ +} + +PrometheusRemoteWriteProtocol::~PrometheusRemoteWriteProtocol() = default; + + +void PrometheusRemoteWriteProtocol::writeTimeSeries(const google::protobuf::RepeatedPtrField & time_series) +{ + auto time_series_storage_id = time_series_storage->getStorageID(); + + LOG_TRACE(log, "{}: Writing {} time series", + time_series_storage_id.getNameForLogs(), time_series.size()); + + auto time_series_storage_metadata = time_series_storage->getInMemoryMetadataPtr(); + auto time_series_settings = time_series_storage->getStorageSettingsPtr(); + + auto blocks = toBlocks(time_series, getContext(), time_series_storage_id, *time_series_storage_metadata, *time_series_settings); + insertToTargetTables(std::move(blocks), *time_series_storage, getContext(), log.get()); + + LOG_TRACE(log, "{}: {} time series written", + time_series_storage_id.getNameForLogs(), time_series.size()); +} + +void PrometheusRemoteWriteProtocol::writeMetricsMetadata(const google::protobuf::RepeatedPtrField & metrics_metadata) +{ + auto time_series_storage_id = time_series_storage->getStorageID(); + + LOG_TRACE(log, "{}: Writing {} metrics metadata", + time_series_storage_id.getNameForLogs(), metrics_metadata.size()); + + auto time_series_storage_metadata = time_series_storage->getInMemoryMetadataPtr(); + auto time_series_settings = time_series_storage->getStorageSettingsPtr(); + + auto blocks = toBlocks(metrics_metadata, time_series_storage_id, *time_series_storage_metadata, *time_series_settings); + insertToTargetTables(std::move(blocks), *time_series_storage, getContext(), log.get()); + + LOG_TRACE(log, "{}: {} metrics metadata written", + time_series_storage_id.getNameForLogs(), metrics_metadata.size()); +} + +} + +#endif diff --git a/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.h b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.h new file mode 100644 index 00000000000..24c65e96cbe --- /dev/null +++ b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.h @@ -0,0 +1,35 @@ +#pragma once + +#include "config.h" +#if USE_PROMETHEUS_PROTOBUFS + +#include +#include +#include + + +namespace DB +{ +class StorageTimeSeries; + +/// Helper class to support the prometheus remote write protocol. +class PrometheusRemoteWriteProtocol : WithContext +{ +public: + PrometheusRemoteWriteProtocol(StoragePtr time_series_storage_, const ContextPtr & context_); + ~PrometheusRemoteWriteProtocol(); + + /// Insert time series received by remote write protocol to our table. + void writeTimeSeries(const google::protobuf::RepeatedPtrField & time_series); + + /// Insert metrics metadata received by remote write protocol to our table. + void writeMetricsMetadata(const google::protobuf::RepeatedPtrField & metrics_metadata); + +private: + std::shared_ptr time_series_storage; + Poco::LoggerPtr log; +}; + +} + +#endif diff --git a/src/Storages/TimeSeries/TimeSeriesTagNames.h b/src/Storages/TimeSeries/TimeSeriesTagNames.h new file mode 100644 index 00000000000..23b005ed414 --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesTagNames.h @@ -0,0 +1,13 @@ +#pragma once + + +namespace DB +{ + +/// Label names with special meaning. +struct TimeSeriesTagNames +{ + static constexpr const char * MetricName = "__name__"; +}; + +} From eacbadf560997c22a178e365f61d60868a08c036 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 30 May 2024 17:23:38 +0200 Subject: [PATCH 272/644] Support prometheus remote read protocol. --- ...bufZeroCopyOutputStreamFromWriteBuffer.cpp | 60 +++ ...tobufZeroCopyOutputStreamFromWriteBuffer.h | 40 ++ src/Server/PrometheusRequestHandler.cpp | 68 +++ src/Server/PrometheusRequestHandler.h | 3 +- src/Server/PrometheusRequestHandlerConfig.h | 3 + .../PrometheusRequestHandlerFactory.cpp | 14 + src/Server/PrometheusRequestHandlerFactory.h | 7 + .../PrometheusRemoteReadProtocol.cpp | 444 ++++++++++++++++++ .../TimeSeries/PrometheusRemoteReadProtocol.h | 36 ++ .../TimeSeries/TimeSeriesColumnsValidator.cpp | 26 + .../TimeSeries/TimeSeriesColumnsValidator.h | 4 + 11 files changed, 704 insertions(+), 1 deletion(-) create mode 100644 src/IO/Protobuf/ProtobufZeroCopyOutputStreamFromWriteBuffer.cpp create mode 100644 src/IO/Protobuf/ProtobufZeroCopyOutputStreamFromWriteBuffer.h create mode 100644 src/Storages/TimeSeries/PrometheusRemoteReadProtocol.cpp create mode 100644 src/Storages/TimeSeries/PrometheusRemoteReadProtocol.h diff --git a/src/IO/Protobuf/ProtobufZeroCopyOutputStreamFromWriteBuffer.cpp b/src/IO/Protobuf/ProtobufZeroCopyOutputStreamFromWriteBuffer.cpp new file mode 100644 index 00000000000..d1e02b436f3 --- /dev/null +++ b/src/IO/Protobuf/ProtobufZeroCopyOutputStreamFromWriteBuffer.cpp @@ -0,0 +1,60 @@ +#include "config.h" + +#if USE_PROTOBUF +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +ProtobufZeroCopyOutputStreamFromWriteBuffer::ProtobufZeroCopyOutputStreamFromWriteBuffer(WriteBuffer & out_) : out(&out_) +{ +} + +ProtobufZeroCopyOutputStreamFromWriteBuffer::ProtobufZeroCopyOutputStreamFromWriteBuffer(std::unique_ptr out_) + : ProtobufZeroCopyOutputStreamFromWriteBuffer(*out_) +{ + out_holder = std::move(out_); +} + +ProtobufZeroCopyOutputStreamFromWriteBuffer::~ProtobufZeroCopyOutputStreamFromWriteBuffer() = default; + +bool ProtobufZeroCopyOutputStreamFromWriteBuffer::Next(void ** data, int * size) +{ + *data = out->position(); + *size = static_cast(out->available()); + out->position() += *size; + return true; +} + +void ProtobufZeroCopyOutputStreamFromWriteBuffer::BackUp(int count) +{ + if (static_cast(out->offset()) < count) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "ProtobufZeroCopyOutputStreamFromWriteBuffer::BackUp() cannot back up {} bytes (max = {} bytes)", + count, + out->offset()); + + out->position() -= count; +} + +int64_t ProtobufZeroCopyOutputStreamFromWriteBuffer::ByteCount() const +{ + return out->count(); +} + +void ProtobufZeroCopyOutputStreamFromWriteBuffer::finalize() +{ + out->finalize(); +} + +} + +#endif diff --git a/src/IO/Protobuf/ProtobufZeroCopyOutputStreamFromWriteBuffer.h b/src/IO/Protobuf/ProtobufZeroCopyOutputStreamFromWriteBuffer.h new file mode 100644 index 00000000000..c47cef9ff4d --- /dev/null +++ b/src/IO/Protobuf/ProtobufZeroCopyOutputStreamFromWriteBuffer.h @@ -0,0 +1,40 @@ +#pragma once + +#include "config.h" +#if USE_PROTOBUF + +#include + + +namespace DB +{ +class WriteBuffer; + +class ProtobufZeroCopyOutputStreamFromWriteBuffer : public google::protobuf::io::ZeroCopyOutputStream +{ +public: + explicit ProtobufZeroCopyOutputStreamFromWriteBuffer(WriteBuffer & out_); + explicit ProtobufZeroCopyOutputStreamFromWriteBuffer(std::unique_ptr out_); + + ~ProtobufZeroCopyOutputStreamFromWriteBuffer() override; + + // Obtains a buffer into which data can be written. + bool Next(void ** data, int * size) override; + + // Backs up a number of bytes, so that the end of the last buffer returned + // by Next() is not actually written. + void BackUp(int count) override; + + // Returns the total number of bytes written since this object was created. + int64_t ByteCount() const override; + + void finalize(); + +private: + WriteBuffer * out; + std::unique_ptr out_holder; +}; + +} + +#endif diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index 98d652540d3..275f6f3c04b 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -12,7 +12,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -20,6 +22,7 @@ #include #include #include +#include #include @@ -234,6 +237,66 @@ public: } }; +/// Implementation of the remote-read protocol. +class PrometheusRequestHandler::RemoteReadImpl : public ImplWithContext +{ +public: + using ImplWithContext::ImplWithContext; + + void beforeHandlingRequest(HTTPServerRequest & request) override + { + LOG_INFO(log(), "Handling remote read request from {}", request.get("User-Agent", "")); + chassert(config().type == PrometheusRequestHandlerConfig::Type::RemoteRead); + } + + void handlingRequestWithContext([[maybe_unused]] HTTPServerRequest & request, [[maybe_unused]] HTTPServerResponse & response) override + { +#if USE_PROMETHEUS_PROTOBUFS + checkHTTPHeader(request, "Content-Type", "application/x-protobuf"); + checkHTTPHeader(request, "Content-Encoding", "snappy"); + + auto table = DatabaseCatalog::instance().getTable(StorageID{config().time_series_table_name}, context); + PrometheusRemoteReadProtocol protocol{table, context}; + + ProtobufZeroCopyInputStreamFromReadBuffer zero_copy_input_stream{ + std::make_unique(wrapReadBufferReference(request.getStream()))}; + + prometheus::ReadRequest read_request; + if (!read_request.ParseFromZeroCopyStream(&zero_copy_input_stream)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse ReadRequest"); + + prometheus::ReadResponse read_response; + + size_t num_queries = read_request.queries_size(); + for (size_t i = 0; i != num_queries; ++i) + { + const auto & query = read_request.queries(static_cast(i)); + auto & new_query_result = *read_response.add_results(); + protocol.readTimeSeries( + *new_query_result.mutable_timeseries(), + query.start_timestamp_ms(), + query.end_timestamp_ms(), + query.matchers(), + query.hints()); + } + +# if 0 + LOG_DEBUG(log, "ReadResponse = {}", read_response.DebugString()); +# endif + + response.setContentType("application/x-protobuf"); + response.set("Content-Encoding", "snappy"); + + ProtobufZeroCopyOutputStreamFromWriteBuffer zero_copy_output_stream{std::make_unique(getOutputStream(response))}; + read_response.SerializeToZeroCopyStream(&zero_copy_output_stream); + zero_copy_output_stream.finalize(); + +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Prometheus remote read protocol is disabled"); +#endif + } +}; + PrometheusRequestHandler::PrometheusRequestHandler( IServer & server_, @@ -265,6 +328,11 @@ void PrometheusRequestHandler::createImpl() impl = std::make_unique(*this); return; } + case PrometheusRequestHandlerConfig::Type::RemoteRead: + { + impl = std::make_unique(*this); + return; + } } UNREACHABLE(); } diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index b4d1e849bdd..6df718f2a05 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -11,7 +11,7 @@ class IServer; class PrometheusMetricsWriter; class WriteBufferFromHTTPServerResponse; -/// Handles requests for prometheus protocols (expose_metrics, remote_write). +/// Handles requests for prometheus protocols (expose_metrics, remote_write, remote-read). class PrometheusRequestHandler : public HTTPRequestHandler { public: @@ -44,6 +44,7 @@ private: class ImplWithContext; class ExposeMetricsImpl; class RemoteWriteImpl; + class RemoteReadImpl; std::unique_ptr impl; String http_method; diff --git a/src/Server/PrometheusRequestHandlerConfig.h b/src/Server/PrometheusRequestHandlerConfig.h index d8fd03f19b2..d01d28f702c 100644 --- a/src/Server/PrometheusRequestHandlerConfig.h +++ b/src/Server/PrometheusRequestHandlerConfig.h @@ -16,6 +16,9 @@ struct PrometheusRequestHandlerConfig /// Handles Prometheus remote-write protocol. RemoteWrite, + + /// Handles Prometheus remote-read protocol. + RemoteRead, }; Type type = Type::ExposeMetrics; diff --git a/src/Server/PrometheusRequestHandlerFactory.cpp b/src/Server/PrometheusRequestHandlerFactory.cpp index d4b1ab6cd93..52f1d3b64c1 100644 --- a/src/Server/PrometheusRequestHandlerFactory.cpp +++ b/src/Server/PrometheusRequestHandlerFactory.cpp @@ -74,6 +74,18 @@ namespace return res; } + /// Parses a configuration like this: + /// + /// db.time_series_table_name
+ PrometheusRequestHandlerConfig parseRemoteReadConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) + { + PrometheusRequestHandlerConfig res; + res.type = PrometheusRequestHandlerConfig::Type::RemoteRead; + res.time_series_table_name = parseTableNameFromConfig(config, config_prefix); + parseCommonConfig(config, res); + return res; + } + /// Parses a configuration like this: /// expose_metrics /// true @@ -91,6 +103,8 @@ namespace return parseExposeMetricsConfig(config, config_prefix); else if (type == "remote_write") return parseRemoteWriteConfig(config, config_prefix); + else if (type == "remote_read") + return parseRemoteReadConfig(config, config_prefix); else throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Unknown type {} is specified in the configuration for a prometheus protocol", type); } diff --git a/src/Server/PrometheusRequestHandlerFactory.h b/src/Server/PrometheusRequestHandlerFactory.h index a7227b12f7e..c52395ca93f 100644 --- a/src/Server/PrometheusRequestHandlerFactory.h +++ b/src/Server/PrometheusRequestHandlerFactory.h @@ -81,6 +81,13 @@ HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactory( /// db.time_series_table_name
/// /// +/// +/// /read +/// +/// remote_read +/// db.time_series_table_name
+///
+///
/// HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactoryForHTTPRule( IServer & server, diff --git a/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.cpp b/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.cpp new file mode 100644 index 00000000000..bb7adb2b0df --- /dev/null +++ b/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.cpp @@ -0,0 +1,444 @@ +#include + +#include "config.h" +#if USE_PROMETHEUS_PROTOBUFS + +#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 +{ + +namespace ErrorCodes +{ + extern const int BAD_REQUEST_PARAMETER; +} + + +namespace +{ + /// Makes an ASTIdentifier for a column of the specified table. + ASTPtr makeASTColumn(const StorageID & table_id, const String & column_name) + { + return std::make_shared(Strings{table_id.database_name, table_id.table_name, column_name}); + } + + /// Makes an AST for condition `data_table.timestamp >= min_timestamp_ms` + ASTPtr makeASTTimestampGreaterOrEquals(Int64 min_timestamp_ms, const StorageID & data_table_id) + { + return makeASTFunction("greaterOrEquals", + makeASTColumn(data_table_id, TimeSeriesColumnNames::Timestamp), + std::make_shared(Field{DecimalField{DateTime64{min_timestamp_ms}, 3}})); + } + + /// Makes an AST for condition `data_table.timestamp <= max_timestamp_ms` + ASTPtr makeASTTimestampLessOrEquals(Int64 max_timestamp_ms, const StorageID & data_table_id) + { + return makeASTFunction("lessOrEquals", + makeASTColumn(data_table_id, TimeSeriesColumnNames::Timestamp), + std::make_shared(Field{DecimalField{DateTime64{max_timestamp_ms}, 3}})); + } + + /// Makes an AST for the expression referencing a tag value. + ASTPtr makeASTLabelName(const String & label_name, const StorageID & tags_table_id, const std::unordered_map & column_name_by_tag_name) + { + if (label_name == TimeSeriesTagNames::MetricName) + return makeASTColumn(tags_table_id, TimeSeriesColumnNames::MetricName); + + auto it = column_name_by_tag_name.find(label_name); + if (it != column_name_by_tag_name.end()) + return makeASTColumn(tags_table_id, it->second); + + /// arrayElement() can be used to extract a value from a Map too. + return makeASTFunction("arrayElement", makeASTColumn(tags_table_id, TimeSeriesColumnNames::Tags), std::make_shared(label_name)); + } + + /// Makes an AST for a label matcher, for example `metric_name == 'value'` or `NOT match(labels['label_name'], 'regexp')`. + ASTPtr makeASTLabelMatcher( + const prometheus::LabelMatcher & label_matcher, + const StorageID & tags_table_id, + const std::unordered_map & column_name_by_tag_name) + { + const auto & label_name = label_matcher.name(); + const auto & label_value = label_matcher.value(); + auto type = label_matcher.type(); + + if (type == prometheus::LabelMatcher::EQ) + return makeASTFunction("equals", makeASTLabelName(label_name, tags_table_id, column_name_by_tag_name), std::make_shared(label_value)); + else if (type == prometheus::LabelMatcher::NEQ) + return makeASTFunction("notEquals", makeASTLabelName(label_name, tags_table_id, column_name_by_tag_name), std::make_shared(label_value)); + else if (type == prometheus::LabelMatcher::RE) + return makeASTFunction("match", makeASTLabelName(label_name, tags_table_id, column_name_by_tag_name), std::make_shared(label_value)); + else if (type == prometheus::LabelMatcher::NRE) + return makeASTFunction("not", makeASTFunction("match", makeASTLabelName(label_name, tags_table_id, column_name_by_tag_name), std::make_shared(label_value))); + else + throw Exception(ErrorCodes::BAD_REQUEST_PARAMETER, "Unexpected type of label matcher: {}", type); + } + + /// Makes an AST checking that tags match a specified label matcher and that timestamp is in range [min_timestamp_ms, max_timestamp_ms]. + ASTPtr makeASTFilterForReadingTimeSeries( + const google::protobuf::RepeatedPtrField & label_matcher, + Int64 min_timestamp_ms, + Int64 max_timestamp_ms, + const StorageID & data_table_id, + const StorageID & tags_table_id, + const std::unordered_map & column_name_by_tag_name) + { + ASTs filters; + + if (min_timestamp_ms) + filters.push_back(makeASTTimestampGreaterOrEquals(min_timestamp_ms, data_table_id)); + + if (max_timestamp_ms) + filters.push_back(makeASTTimestampLessOrEquals(max_timestamp_ms, data_table_id)); + + for (const auto & label_matcher_element : label_matcher) + filters.push_back(makeASTLabelMatcher(label_matcher_element, tags_table_id, column_name_by_tag_name)); + + if (filters.empty()) + return nullptr; + + return makeASTForLogicalAnd(std::move(filters)); + } + + /// Makes a mapping from a tag name to a column name. + std::unordered_map makeColumnNameByTagNameMap(const TimeSeriesSettings & storage_settings) + { + std::unordered_map res; + const Map & tags_to_columns = storage_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & tag_name = tuple.at(0).safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + res[tag_name] = column_name; + } + return res; + } + + /// The function builds a SELECT query for reading time series: + /// SELECT tags_table.metric_name, tags_table.tag_column1, ... tags_table.tag_columnN, tags_table.tags, + /// groupArray(CAST(data_table.timestamp, 'DateTime64(3)'), CAST(data_table.value, 'Float64')) + /// FROM data_table + /// SEMI LEFT JOIN tag_table ON data_table.id = tags_table.id + /// WHERE filter + /// GROUP BY tags_table.tag_column1, ..., tags_table.tag_columnN, tags_table.tags + ASTPtr buildSelectQueryForReadingTimeSeries( + Int64 min_timestamp_ms, + Int64 max_timestamp_ms, + const google::protobuf::RepeatedPtrField & label_matcher, + const TimeSeriesSettings & time_series_settings, + const StorageID & data_table_id, + const StorageID & tags_table_id) + { + auto select_query = std::make_shared(); + + /// SELECT tags_table.metric_name, any(tags_table.tag_column1), ... any(tags_table.tag_columnN), any(tags_table.tags), + /// groupArray(data_table.timestamp, data_table.value) + { + auto exp_list = std::make_shared(); + + exp_list->children.push_back( + makeASTColumn(tags_table_id, TimeSeriesColumnNames::MetricName)); + + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + exp_list->children.push_back( + makeASTColumn(tags_table_id, column_name)); + } + + exp_list->children.push_back( + makeASTColumn(tags_table_id, TimeSeriesColumnNames::Tags)); + + exp_list->children.push_back( + makeASTFunction("groupArray", + makeASTFunction("tuple", + makeASTFunction("CAST", makeASTColumn(data_table_id, TimeSeriesColumnNames::Timestamp), std::make_shared("DateTime64(3)")), + makeASTFunction("CAST", makeASTColumn(data_table_id, TimeSeriesColumnNames::Value), std::make_shared("Float64"))))); + + select_query->setExpression(ASTSelectQuery::Expression::SELECT, exp_list); + } + + /// FROM data_table + auto tables = std::make_shared(); + + { + auto table = std::make_shared(); + auto table_exp = std::make_shared(); + table_exp->database_and_table_name = std::make_shared(data_table_id); + table_exp->children.emplace_back(table_exp->database_and_table_name); + + table->table_expression = table_exp; + tables->children.push_back(table); + } + + /// SEMI LEFT JOIN tags_table ON data_table.id = tags_table.id + { + auto table = std::make_shared(); + + auto table_join = std::make_shared(); + table_join->kind = JoinKind::Left; + table_join->strictness = JoinStrictness::Semi; + + table_join->on_expression = makeASTFunction("equals", makeASTColumn(data_table_id, TimeSeriesColumnNames::ID), makeASTColumn(tags_table_id, TimeSeriesColumnNames::ID)); + table->table_join = table_join; + + auto table_exp = std::make_shared(); + table_exp->database_and_table_name = std::make_shared(tags_table_id); + table_exp->children.emplace_back(table_exp->database_and_table_name); + + table->table_expression = table_exp; + tables->children.push_back(table); + + select_query->setExpression(ASTSelectQuery::Expression::TABLES, tables); + } + + auto column_name_by_tag_name = makeColumnNameByTagNameMap(time_series_settings); + + /// WHERE + if (auto where = makeASTFilterForReadingTimeSeries(label_matcher, min_timestamp_ms, max_timestamp_ms, data_table_id, tags_table_id, column_name_by_tag_name)) + select_query->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where)); + + /// GROUP BY tags_table.metric_name, tags_table.tag_column1, ..., tags_table.tag_columnN, tags_table.tags + { + auto exp_list = std::make_shared(); + + exp_list->children.push_back( + makeASTColumn(tags_table_id, TimeSeriesColumnNames::MetricName)); + + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + exp_list->children.push_back( + makeASTColumn(tags_table_id, column_name)); + } + + exp_list->children.push_back(makeASTColumn(tags_table_id, TimeSeriesColumnNames::Tags)); + + select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, exp_list); + } + + return select_query; + } + + /// Sorts a list of pairs {tag_name, tag_value} by tag name. + void sortLabelsByName(std::vector> & labels) + { + auto less_by_label_name = [](const std::pair & left, const std::pair & right) + { + return left.first < right.first; + }; + std::sort(labels.begin(), labels.end(), less_by_label_name); + } + + /// Sorts a list of pairs {timestamp, value} by timestamp. + void sortTimeSeriesByTimestamp(std::vector> & time_series) + { + auto less_by_timestamp = [](const std::pair & left, const std::pair & right) + { + return left.first < right.first; + }; + std::sort(time_series.begin(), time_series.end(), less_by_timestamp); + } + + /// Converts a block generated by the SELECT query for converting time series to the protobuf format. + void convertBlockToProtobuf( + Block && block, + google::protobuf::RepeatedPtrField & out_time_series, + const StorageID & time_series_storage_id, + const TimeSeriesSettings & time_series_settings) + { + size_t num_rows = block.rows(); + if (!num_rows) + return; + + size_t column_index = 0; + + /// We analyze columns sequentially. + auto get_next_column_with_type = [&] -> const ColumnWithTypeAndName & { return block.getByPosition(column_index++); }; + auto get_next_column = [&] -> const IColumn & { return *(get_next_column_with_type().column); }; + + /// Column "metric_name". + const auto & metric_name_column_with_type = get_next_column_with_type(); + TimeSeriesColumnsValidator validator{time_series_storage_id, time_series_settings}; + validator.validateColumnForMetricName(metric_name_column_with_type); + const auto & metric_name_column = *metric_name_column_with_type.column; + + /// Columns corresponding to specific tags specified in the "tags_to_columns" setting. + std::unordered_map column_by_tag_name; + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & tag_name = tuple.at(0).safeGet(); + const auto & column_with_type = get_next_column_with_type(); + validator.validateColumnForTagValue(column_with_type); + const auto & column = *column_with_type.column; + column_by_tag_name[tag_name] = &column; + } + + /// Column "tags". + const auto & tags_column_with_type = get_next_column_with_type(); + validator.validateColumnForTagsMap(tags_column_with_type); + const auto & tags_column = checkAndGetColumn(*tags_column_with_type.column); + const auto & tags_names = tags_column.getNestedData().getColumn(0); + const auto & tags_values = tags_column.getNestedData().getColumn(1); + const auto & tags_offsets = tags_column.getNestedColumn().getOffsets(); + + /// Column containing time series: groupArray(CAST(data_table.timestamp, 'DateTime64(3)'), CAST(data_table.value, 'Float64')) + const auto & time_series_column = checkAndGetColumn(get_next_column()); + const auto & time_series_timestamps = checkAndGetColumn>(checkAndGetColumn(time_series_column.getData()).getColumn(0)); + const auto & time_series_values = checkAndGetColumn(checkAndGetColumn(time_series_column.getData()).getColumn(1)); + const auto & time_series_offsets = time_series_column.getOffsets(); + + /// We will sort labels lexicographically and time series by timestamp before sending them to a client. + std::vector> labels; + std::vector> time_series; + + for (size_t i = 0; i != num_rows; ++i) + { + /// Collect labels. + size_t num_labels = 1; /* 1 for a metric name */ + + for (const auto & [_, column] : column_by_tag_name) + { + if (!column->isNullAt(i) && !column->getDataAt(i).empty()) + ++num_labels; + } + + size_t tags_start_offset = tags_offsets[i - 1]; + size_t tags_end_offset = tags_offsets[i]; + num_labels += tags_end_offset - tags_start_offset; + + labels.clear(); + labels.reserve(num_labels); + + labels.emplace_back(TimeSeriesTagNames::MetricName, metric_name_column.getDataAt(i)); + + for (const auto & [tag_name, column] : column_by_tag_name) + { + if (!column->isNullAt(i) && !column->getDataAt(i).empty()) + labels.emplace_back(tag_name, column->getDataAt(i)); + } + + for (size_t j = tags_start_offset; j != tags_end_offset; ++j) + { + std::string_view tag_name{tags_names.getDataAt(j)}; + std::string_view tag_value{tags_values.getDataAt(j)}; + labels.emplace_back(tag_name, tag_value); + } + + /// Sort labels. + sortLabelsByName(labels); + + /// Collect time series. + size_t time_series_start_offset = time_series_offsets[i - 1]; + size_t time_series_end_offset = time_series_offsets[i]; + size_t num_time_series = time_series_end_offset - time_series_start_offset; + + time_series.clear(); + time_series.reserve(num_time_series); + + for (size_t j = time_series_start_offset; j != time_series_end_offset; ++j) + time_series.emplace_back(time_series_timestamps.getElement(j), time_series_values.getElement(j)); + + /// Sort time series. + sortTimeSeriesByTimestamp(time_series); + + /// Prepare a result. + auto & new_time_series = *out_time_series.Add(); + + for (const auto & [label_name, label_value] : labels) + { + auto & new_label = *new_time_series.add_labels(); + new_label.set_name(label_name); + new_label.set_value(label_value); + } + + for (const auto & [timestamp, value] : time_series) + { + auto & new_sample = *new_time_series.add_samples(); + new_sample.set_timestamp(timestamp); + new_sample.set_value(value); + } + } + } +} + + +PrometheusRemoteReadProtocol::PrometheusRemoteReadProtocol(ConstStoragePtr time_series_storage_, const ContextPtr & context_) + : WithContext{context_} + , time_series_storage(storagePtrToTimeSeries(time_series_storage_)) + , log(getLogger("PrometheusRemoteReadProtocol")) +{ +} + +PrometheusRemoteReadProtocol::~PrometheusRemoteReadProtocol() = default; + +void PrometheusRemoteReadProtocol::readTimeSeries(google::protobuf::RepeatedPtrField & out_time_series, + Int64 start_timestamp_ms, + Int64 end_timestamp_ms, + const google::protobuf::RepeatedPtrField & label_matcher, + const prometheus::ReadHints &) +{ + out_time_series.Clear(); + + auto time_series_storage_id = time_series_storage->getStorageID(); + auto time_series_settings = time_series_storage->getStorageSettingsPtr(); + auto data_table_id = time_series_storage->getTargetTableId(ViewTarget::Data); + auto tags_table_id = time_series_storage->getTargetTableId(ViewTarget::Tags); + + ASTPtr select_query = buildSelectQueryForReadingTimeSeries( + start_timestamp_ms, end_timestamp_ms, label_matcher, *time_series_settings, data_table_id, tags_table_id); + + LOG_TRACE(log, "{}: Executing query {}", + time_series_storage_id.getNameForLogs(), select_query); + + InterpreterSelectQuery interpreter(select_query, getContext(), SelectQueryOptions{}); + BlockIO io = interpreter.execute(); + PullingPipelineExecutor executor(io.pipeline); + + Block block; + while (executor.pull(block)) + { + LOG_TRACE(log, "{}: Pulled block with {} columns and {} rows", + time_series_storage_id.getNameForLogs(), block.columns(), block.rows()); + + if (block) + convertBlockToProtobuf(std::move(block), out_time_series, time_series_storage_id, *time_series_settings); + } + + LOG_TRACE(log, "{}: {} time series read", + time_series_storage_id.getNameForLogs(), out_time_series.size()); +} + +} + +#endif diff --git a/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.h b/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.h new file mode 100644 index 00000000000..e10e1f8c8cf --- /dev/null +++ b/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.h @@ -0,0 +1,36 @@ +#pragma once + +#include "config.h" +#if USE_PROMETHEUS_PROTOBUFS + +#include +#include +#include + + +namespace DB +{ +class StorageTimeSeries; + +/// Helper class to support the prometheus remote read protocol. +class PrometheusRemoteReadProtocol : public WithContext +{ +public: + PrometheusRemoteReadProtocol(ConstStoragePtr time_series_storage_, const ContextPtr & context_); + ~PrometheusRemoteReadProtocol(); + + /// Reads time series to send to client by remote read protocol. + void readTimeSeries(google::protobuf::RepeatedPtrField & out_time_series, + Int64 start_timestamp_ms, + Int64 end_timestamp_ms, + const google::protobuf::RepeatedPtrField & label_matcher, + const prometheus::ReadHints & read_hints); + +private: + std::shared_ptr time_series_storage; + Poco::LoggerPtr log; +}; + +} + +#endif diff --git a/src/Storages/TimeSeries/TimeSeriesColumnsValidator.cpp b/src/Storages/TimeSeries/TimeSeriesColumnsValidator.cpp index 0ce5528939a..a2308857e2e 100644 --- a/src/Storages/TimeSeries/TimeSeriesColumnsValidator.cpp +++ b/src/Storages/TimeSeries/TimeSeriesColumnsValidator.cpp @@ -14,6 +14,7 @@ namespace DB namespace ErrorCodes { + extern const int ILLEGAL_COLUMN; extern const int INCOMPATIBLE_COLUMNS; extern const int THERE_IS_NO_COLUMN; } @@ -199,6 +200,11 @@ void TimeSeriesColumnsValidator::validateColumnForMetricName(const ColumnDescrip validateColumnForTagValue(column); } +void TimeSeriesColumnsValidator::validateColumnForMetricName(const ColumnWithTypeAndName & column) const +{ + validateColumnForTagValue(column); +} + void TimeSeriesColumnsValidator::validateColumnForTagValue(const ColumnDescription & column) const { if (!isString(removeLowCardinalityAndNullable(column.type))) @@ -208,6 +214,15 @@ void TimeSeriesColumnsValidator::validateColumnForTagValue(const ColumnDescripti } } +void TimeSeriesColumnsValidator::validateColumnForTagValue(const ColumnWithTypeAndName & column) const +{ + if (!isString(removeLowCardinalityAndNullable(column.type))) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column {} has illegal data type {}, expected String or LowCardinality(String)", + column.name, column.type->getName()); + } +} + void TimeSeriesColumnsValidator::validateColumnForTagsMap(const ColumnDescription & column) const { if (!isMap(column.type) @@ -219,6 +234,17 @@ void TimeSeriesColumnsValidator::validateColumnForTagsMap(const ColumnDescriptio } } +void TimeSeriesColumnsValidator::validateColumnForTagsMap(const ColumnWithTypeAndName & column) const +{ + if (!isMap(column.type) + || !isString(removeLowCardinality(typeid_cast(*column.type).getKeyType())) + || !isString(removeLowCardinality(typeid_cast(*column.type).getValueType()))) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column {} has illegal data type {}, expected Map(String, String) or Map(LowCardinality(String), String)", + column.name, column.type->getName()); + } +} + void TimeSeriesColumnsValidator::validateColumnForMetricFamilyName(const ColumnDescription & column) const { if (!isString(removeLowCardinalityAndNullable(column.type))) diff --git a/src/Storages/TimeSeries/TimeSeriesColumnsValidator.h b/src/Storages/TimeSeries/TimeSeriesColumnsValidator.h index cafee9da03c..43a54bf2ad6 100644 --- a/src/Storages/TimeSeries/TimeSeriesColumnsValidator.h +++ b/src/Storages/TimeSeries/TimeSeriesColumnsValidator.h @@ -8,6 +8,7 @@ namespace DB { class ColumnsDescription; struct ColumnDescription; +struct ColumnWithTypeAndName; struct TimeSeriesSettings; /// Checks the types of columns of a TimeSeries table. @@ -32,8 +33,11 @@ public: void validateColumnForValue(const ColumnDescription & column) const; void validateColumnForMetricName(const ColumnDescription & column) const; + void validateColumnForMetricName(const ColumnWithTypeAndName & column) const; void validateColumnForTagValue(const ColumnDescription & column) const; + void validateColumnForTagValue(const ColumnWithTypeAndName & column) const; void validateColumnForTagsMap(const ColumnDescription & column) const; + void validateColumnForTagsMap(const ColumnWithTypeAndName & column) const; void validateColumnForMetricFamilyName(const ColumnDescription & column) const; void validateColumnForType(const ColumnDescription & column) const; From a5febd4ea0a5609e3acc00751c5a4d8919eb14e7 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 27 May 2024 21:40:50 +0200 Subject: [PATCH 273/644] Add table functions timeSeriesData(), timeSeriesTags(), timeSeriesMetrics() to retrieve the target tables of a TimeSeries table. --- .../TableFunctionTimeSeries.cpp | 128 ++++++++++++++++++ src/TableFunctions/TableFunctionTimeSeries.h | 42 ++++++ src/TableFunctions/registerTableFunctions.cpp | 1 + src/TableFunctions/registerTableFunctions.h | 2 + 4 files changed, 173 insertions(+) create mode 100644 src/TableFunctions/TableFunctionTimeSeries.cpp create mode 100644 src/TableFunctions/TableFunctionTimeSeries.h diff --git a/src/TableFunctions/TableFunctionTimeSeries.cpp b/src/TableFunctions/TableFunctionTimeSeries.cpp new file mode 100644 index 00000000000..62ea088eba0 --- /dev/null +++ b/src/TableFunctions/TableFunctionTimeSeries.cpp @@ -0,0 +1,128 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +template +void TableFunctionTimeSeriesTarget::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + const auto & args_func = ast_function->as(); + + if (!args_func.arguments) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function '{}' must have arguments.", name); + + auto & args = args_func.arguments->children; + + if ((args.size() != 1) && (args.size() != 2)) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Table function '{}' requires one or two arguments: {}([database, ] time_series_table)", name, name); + + if (args.size() == 1) + { + /// timeSeriesMetrics( [my_db.]my_time_series_table ) + if (const auto * id = args[0]->as()) + { + if (auto table_id = id->createTable()) + time_series_storage_id = table_id->getTableId(); + } + } + + if (time_series_storage_id.empty()) + { + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + + if (args.size() == 1) + { + /// timeSeriesMetrics( 'my_time_series_table' ) + time_series_storage_id.table_name = checkAndGetLiteralArgument(args[0], "table_name"); + } + else + { + /// timeSeriesMetrics( 'mydb', 'my_time_series_table' ) + time_series_storage_id.database_name = checkAndGetLiteralArgument(args[0], "database_name"); + time_series_storage_id.table_name = checkAndGetLiteralArgument(args[1], "table_name"); + } + } + + if (time_series_storage_id.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Couldn't get a table name from the arguments of the {} table function", name); + + time_series_storage_id = context->resolveStorageID(time_series_storage_id); + target_table_type_name = getTargetTable(context)->getName(); +} + + +template +StoragePtr TableFunctionTimeSeriesTarget::getTargetTable(const ContextPtr & context) const +{ + auto time_series_storage = storagePtrToTimeSeries(DatabaseCatalog::instance().getTable(time_series_storage_id, context)); + return time_series_storage->getTargetTable(target_kind, context); +} + + +template +StoragePtr TableFunctionTimeSeriesTarget::executeImpl( + const ASTPtr & /* ast_function */, + ContextPtr context, + const String & /* table_name */, + ColumnsDescription /* cached_columns */, + bool /* is_insert_query */) const +{ + return getTargetTable(context); +} + +template +ColumnsDescription TableFunctionTimeSeriesTarget::getActualTableStructure(ContextPtr context, bool /* is_insert_query */) const +{ + return getTargetTable(context)->getInMemoryMetadataPtr()->columns; +} + +template +const char * TableFunctionTimeSeriesTarget::getStorageTypeName() const +{ + return target_table_type_name.c_str(); +} + + +void registerTableFunctionTimeSeries(TableFunctionFactory & factory) +{ + factory.registerFunction>( + {.documentation = { + .description=R"(Provides direct access to the 'data' target table for a specified TimeSeries table.)", + .examples{{"timeSeriesData", "SELECT * from timeSeriesData('mydb', 'time_series_table');", ""}}, + .categories{"Time Series"}} + }); + factory.registerFunction>( + {.documentation = { + .description=R"(Provides direct access to the 'tags' target table for a specified TimeSeries table.)", + .examples{{"timeSeriesTags", "SELECT * from timeSeriesTags('mydb', 'time_series_table');", ""}}, + .categories{"Time Series"}} + }); + factory.registerFunction>( + {.documentation = { + .description=R"(Provides direct access to the 'metrics' target table for a specified TimeSeries table.)", + .examples{{"timeSeriesMetrics", "SELECT * from timeSeriesMetrics('mydb', 'time_series_table');", ""}}, + .categories{"Time Series"}} + }); +} + +} diff --git a/src/TableFunctions/TableFunctionTimeSeries.h b/src/TableFunctions/TableFunctionTimeSeries.h new file mode 100644 index 00000000000..57654413fe4 --- /dev/null +++ b/src/TableFunctions/TableFunctionTimeSeries.h @@ -0,0 +1,42 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +/// Table functions timeSeriesData('mydb', 'my_ts_table'), timeSeriesTags('mydb', 'my_ts_table'), timeSeriesMetrics('mydb', 'my_ts_table') +/// return the data table, the tags table, and the metrics table respectively associated with any TimeSeries table mydb.my_ts_table +template +class TableFunctionTimeSeriesTarget : public ITableFunction +{ +public: + static constexpr auto name = (target_kind == ViewTarget::Data) + ? "timeSeriesData" + : ((target_kind == ViewTarget::Tags) ? "timeSeriesTags" : "timeSeriesMetrics"); + + String getName() const override { return name; } + +private: + void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; + + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const override; + + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; + const char * getStorageTypeName() const override; + + StoragePtr getTargetTable(const ContextPtr & context) const; + + StorageID time_series_storage_id = StorageID::createEmpty(); + String target_table_type_name; +}; + +} diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index ca4913898f9..42987a6a5b9 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -56,6 +56,7 @@ void registerTableFunctions() registerTableFunctionFormat(factory); registerTableFunctionExplain(factory); + registerTableFunctionTimeSeries(factory); registerTableFunctionObjectStorage(factory); registerTableFunctionObjectStorageCluster(factory); diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index efde4d6dcdc..1957888d478 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -67,6 +67,8 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory); void registerTableFunctionObjectStorageCluster(TableFunctionFactory & factory); void registerDataLakeTableFunctions(TableFunctionFactory & factory); +void registerTableFunctionTimeSeries(TableFunctionFactory & factory); + void registerTableFunctions(); } From 30231797c7e48549af7d551be05ec798dde79abe Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 23 Jun 2024 16:10:42 +0200 Subject: [PATCH 274/644] Add columns `min_time` and `max_time`. --- .../PrometheusRemoteReadProtocol.cpp | 32 ++++++++++++++- .../PrometheusRemoteWriteProtocol.cpp | 41 +++++++++++++++++++ .../TimeSeries/TimeSeriesColumnNames.h | 4 ++ .../TimeSeriesDefinitionNormalizer.cpp | 40 +++++++++++++++++- .../TimeSeriesInnerTablesCreator.cpp | 23 +++++++++++ src/Storages/TimeSeries/TimeSeriesSettings.h | 3 ++ 6 files changed, 140 insertions(+), 3 deletions(-) diff --git a/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.cpp b/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.cpp index bb7adb2b0df..d6d258f5ff6 100644 --- a/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.cpp +++ b/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.cpp @@ -61,6 +61,22 @@ namespace std::make_shared(Field{DecimalField{DateTime64{max_timestamp_ms}, 3}})); } + /// Makes an AST for condition `tags_table.max_time >= min_timestamp_ms` + ASTPtr makeASTMaxTimeGreaterOrEquals(Int64 min_timestamp_ms, const StorageID & tags_table_id) + { + return makeASTFunction("greaterOrEquals", + makeASTColumn(tags_table_id, TimeSeriesColumnNames::MaxTime), + std::make_shared(Field{DecimalField{DateTime64{min_timestamp_ms}, 3}})); + } + + /// Makes an AST for condition `tags_table.min_time <= max_timestamp_ms` + ASTPtr makeASTMinTimeLessOrEquals(Int64 max_timestamp_ms, const StorageID & tags_table_id) + { + return makeASTFunction("lessOrEquals", + makeASTColumn(tags_table_id, TimeSeriesColumnNames::MinTime), + std::make_shared(Field{DecimalField{DateTime64{max_timestamp_ms}, 3}})); + } + /// Makes an AST for the expression referencing a tag value. ASTPtr makeASTLabelName(const String & label_name, const StorageID & tags_table_id, const std::unordered_map & column_name_by_tag_name) { @@ -104,15 +120,24 @@ namespace Int64 max_timestamp_ms, const StorageID & data_table_id, const StorageID & tags_table_id, - const std::unordered_map & column_name_by_tag_name) + const std::unordered_map & column_name_by_tag_name, + bool filter_by_min_time_and_max_time) { ASTs filters; if (min_timestamp_ms) + { filters.push_back(makeASTTimestampGreaterOrEquals(min_timestamp_ms, data_table_id)); + if (filter_by_min_time_and_max_time) + filters.push_back(makeASTMaxTimeGreaterOrEquals(min_timestamp_ms, tags_table_id)); + } if (max_timestamp_ms) + { filters.push_back(makeASTTimestampLessOrEquals(max_timestamp_ms, data_table_id)); + if (filter_by_min_time_and_max_time) + filters.push_back(makeASTMinTimeLessOrEquals(max_timestamp_ms, tags_table_id)); + } for (const auto & label_matcher_element : label_matcher) filters.push_back(makeASTLabelMatcher(label_matcher_element, tags_table_id, column_name_by_tag_name)); @@ -221,8 +246,11 @@ namespace auto column_name_by_tag_name = makeColumnNameByTagNameMap(time_series_settings); /// WHERE - if (auto where = makeASTFilterForReadingTimeSeries(label_matcher, min_timestamp_ms, max_timestamp_ms, data_table_id, tags_table_id, column_name_by_tag_name)) + if (auto where = makeASTFilterForReadingTimeSeries(label_matcher, min_timestamp_ms, max_timestamp_ms, data_table_id, tags_table_id, + column_name_by_tag_name, time_series_settings.filter_by_min_time_and_max_time)) + { select_query->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where)); + } /// GROUP BY tags_table.metric_name, tags_table.tag_column1, ..., tags_table.tag_columnN, tags_table.tags { diff --git a/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp index b3845e88406..ca62a2729dc 100644 --- a/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp +++ b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp @@ -162,6 +162,23 @@ namespace return timestamp_ms / DecimalUtils::scaleMultiplier(3 - scale); } + /// Finds min time and max time in a time series. + std::pair findMinTimeAndMaxTime(const google::protobuf::RepeatedPtrField & samples) + { + chassert(!samples.empty()); + Int64 min_time = std::numeric_limits::max(); + Int64 max_time = std::numeric_limits::min(); + for (const auto & sample : samples) + { + Int64 timestamp = sample.timestamp(); + if (timestamp < min_time) + min_time = timestamp; + if (timestamp > max_time) + max_time = timestamp; + } + return {min_time, max_time}; + } + struct BlocksToInsert { std::vector> blocks; @@ -270,6 +287,23 @@ namespace IColumn & all_tags_values = all_tags_column.getNestedData().getColumn(1); auto & all_tags_offsets = all_tags_column.getNestedColumn().getOffsets(); + /// Columns "min_time" and "max_time". + IColumn * min_time_column = nullptr; + IColumn * max_time_column = nullptr; + UInt32 min_time_scale = 0; + UInt32 max_time_scale = 0; + if (time_series_settings.store_min_time_and_max_time) + { + const auto & min_time_description = get_column_description(TimeSeriesColumnNames::MinTime); + const auto & max_time_description = get_column_description(TimeSeriesColumnNames::MaxTime); + validator.validateColumnForTimestamp(min_time_description, min_time_scale); + validator.validateColumnForTimestamp(max_time_description, max_time_scale); + min_time_column = &make_column_for_tags_block(min_time_description); + max_time_column = &make_column_for_tags_block(max_time_description); + columns_to_fill_in_tags_table.emplace_back(min_time_column); + columns_to_fill_in_tags_table.emplace_back(max_time_column); + } + /// Prepare a block for inserting into the "tags" table. size_t current_row_in_tags = 0; for (size_t i = 0; i != static_cast(time_series.size()); ++i) @@ -314,6 +348,13 @@ namespace all_tags_offsets.push_back(all_tags_names.size()); tags_offsets.push_back(tags_names.size()); + if (time_series_settings.store_min_time_and_max_time) + { + auto [min_time, max_time] = findMinTimeAndMaxTime(element.samples()); + min_time_column->insert(scaleTimestamp(min_time, min_time_scale)); + max_time_column->insert(scaleTimestamp(max_time, max_time_scale)); + } + for (auto * column : columns_to_fill_in_tags_table) { if (column->size() == current_row_in_tags) diff --git a/src/Storages/TimeSeries/TimeSeriesColumnNames.h b/src/Storages/TimeSeries/TimeSeriesColumnNames.h index 9176ec5384a..d7b12fdeea8 100644 --- a/src/Storages/TimeSeries/TimeSeriesColumnNames.h +++ b/src/Storages/TimeSeries/TimeSeriesColumnNames.h @@ -24,6 +24,10 @@ struct TimeSeriesColumnNames /// This is a generated column, it's not stored anywhere, it's generated on the fly. static constexpr const char * AllTags = "all_tags"; + /// Contains the time range of a time series. + static constexpr const char * MinTime = "min_time"; + static constexpr const char * MaxTime = "max_time"; + /// The "metrics" table contains general information (metadata) about metrics: static constexpr const char * MetricFamilyName = "metric_family_name"; static constexpr const char * Type = "type"; diff --git a/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp index 78f8afe2528..8c2f56d9858 100644 --- a/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp +++ b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp @@ -95,6 +95,12 @@ void TimeSeriesDefinitionNormalizer::reorderColumns(ASTCreateQuery & create) con add_column_in_correct_order(TimeSeriesColumnNames::Tags); add_column_in_correct_order(TimeSeriesColumnNames::AllTags); + if (time_series_settings.store_min_time_and_max_time) + { + add_column_in_correct_order(TimeSeriesColumnNames::MinTime); + add_column_in_correct_order(TimeSeriesColumnNames::MaxTime); + } + /// Reorder columns for the "metrics" table. add_column_in_correct_order(TimeSeriesColumnNames::MetricFamilyName); add_column_in_correct_order(TimeSeriesColumnNames::Type); @@ -163,6 +169,14 @@ void TimeSeriesDefinitionNormalizer::addMissingColumns(ASTCreateQuery & create) auto get_string_to_string_map_type = [&] { return makeASTDataType("Map", get_string_type(), get_string_type()); }; auto get_lc_string_to_string_map_type = [&] { return makeASTDataType("Map", get_lc_string_type(), get_string_type()); }; + auto make_nullable = [&](std::shared_ptr type) + { + if (type->name == "Nullable") + return type; + else + return makeASTDataType("Nullable", type); + }; + /// Add missing columns for the "data" table. if (!is_next_column_named(TimeSeriesColumnNames::ID)) make_new_column(TimeSeriesColumnNames::ID, get_uuid_type()); @@ -170,6 +184,9 @@ void TimeSeriesDefinitionNormalizer::addMissingColumns(ASTCreateQuery & create) if (!is_next_column_named(TimeSeriesColumnNames::Timestamp)) make_new_column(TimeSeriesColumnNames::Timestamp, get_datetime_type()); + auto timestamp_column = typeid_cast>(columns[position - 1]); + auto timestamp_type = typeid_cast>(timestamp_column->type->ptr()); + if (!is_next_column_named(TimeSeriesColumnNames::Value)) make_new_column(TimeSeriesColumnNames::Value, get_float_type()); @@ -204,6 +221,15 @@ void TimeSeriesDefinitionNormalizer::addMissingColumns(ASTCreateQuery & create) make_new_column(TimeSeriesColumnNames::AllTags, get_string_to_string_map_type()); } + if (time_series_settings.store_min_time_and_max_time) + { + /// We use Nullable(DateTime64(3)) as the default type of the `min_time` and `max_time` columns. + /// It's nullable because it allows the aggregation (see aggregate_min_time_and_max_time) work correctly even + /// for rows in the "tags" table which doesn't have `min_time` and `max_time` (because they have no matching rows in the "data" table). + make_new_column(TimeSeriesColumnNames::MinTime, make_nullable(timestamp_type)); + make_new_column(TimeSeriesColumnNames::MaxTime, make_nullable(timestamp_type)); + } + /// Add missing columns for the "metrics" table. if (!is_next_column_named(TimeSeriesColumnNames::MetricFamilyName)) make_new_column(TimeSeriesColumnNames::MetricFamilyName, get_string_type()); @@ -374,7 +400,13 @@ void TimeSeriesDefinitionNormalizer::setInnerEngineByDefault(ViewTarget::Kind in case ViewTarget::Tags: { - inner_storage_def.set(inner_storage_def.engine, makeASTFunction("ReplacingMergeTree")); + String engine_name; + if (time_series_settings.aggregate_min_time_and_max_time) + engine_name = "AggregatingMergeTree"; + else + engine_name = "ReplacingMergeTree"; + + inner_storage_def.set(inner_storage_def.engine, makeASTFunction(engine_name)); inner_storage_def.engine->no_empty_args = false; if (!inner_storage_def.order_by && !inner_storage_def.primary_key && inner_storage_def.engine->name.ends_with("MergeTree")) @@ -386,6 +418,12 @@ void TimeSeriesDefinitionNormalizer::setInnerEngineByDefault(ViewTarget::Kind in order_by_list.push_back(std::make_shared(TimeSeriesColumnNames::MetricName)); order_by_list.push_back(std::make_shared(TimeSeriesColumnNames::ID)); + if (time_series_settings.store_min_time_and_max_time && !time_series_settings.aggregate_min_time_and_max_time) + { + order_by_list.push_back(std::make_shared(TimeSeriesColumnNames::MinTime)); + order_by_list.push_back(std::make_shared(TimeSeriesColumnNames::MaxTime)); + } + auto order_by_tuple = std::make_shared(); order_by_tuple->name = "tuple"; auto arguments_list = std::make_shared(); diff --git a/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp index e43cba01cdb..5376ec2b124 100644 --- a/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp +++ b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp @@ -1,5 +1,8 @@ #include +#include +#include +#include #include #include #include @@ -87,6 +90,26 @@ ColumnsDescription TimeSeriesInnerTablesCreator::getInnerTableColumnsDescription } columns.add(std::move(all_tags_column)); + /// Columns "min_time" and "max_time". + if (time_series_settings.store_min_time_and_max_time) + { + auto min_time_column = time_series_columns.get(TimeSeriesColumnNames::MinTime); + auto max_time_column = time_series_columns.get(TimeSeriesColumnNames::MaxTime); + if (time_series_settings.aggregate_min_time_and_max_time) + { + AggregateFunctionProperties properties; + auto min_function = AggregateFunctionFactory::instance().get("min", NullsAction::EMPTY, {min_time_column.type}, {}, properties); + auto custom_name = std::make_unique(min_function, DataTypes{min_time_column.type}, Array{}); + min_time_column.type = DataTypeFactory::instance().getCustom(std::make_unique(std::move(custom_name))); + + auto max_function = AggregateFunctionFactory::instance().get("max", NullsAction::EMPTY, {max_time_column.type}, {}, properties); + custom_name = std::make_unique(max_function, DataTypes{max_time_column.type}, Array{}); + max_time_column.type = DataTypeFactory::instance().getCustom(std::make_unique(std::move(custom_name))); + } + columns.add(std::move(min_time_column)); + columns.add(std::move(max_time_column)); + } + break; } diff --git a/src/Storages/TimeSeries/TimeSeriesSettings.h b/src/Storages/TimeSeries/TimeSeriesSettings.h index ea31aa8dac1..acbbc7e7806 100644 --- a/src/Storages/TimeSeries/TimeSeriesSettings.h +++ b/src/Storages/TimeSeries/TimeSeriesSettings.h @@ -9,6 +9,9 @@ class ASTStorage; #define LIST_OF_TIME_SERIES_SETTINGS(M, ALIAS) \ M(Map, tags_to_columns, Map{}, "Map specifying which tags should be put to separate columns of the 'tags' table. Syntax: {'tag1': 'column1', 'tag2' : column2, ...}", 0) \ + M(Bool, store_min_time_and_max_time, true, "If set to true then the table will store 'min_time' and 'max_time' for each time series", 0) \ + M(Bool, aggregate_min_time_and_max_time, true, "When creating an inner target 'tags' table, this flag enables using 'SimpleAggregateFunction(min, Nullable(DateTime64(3)))' instead of just 'Nullable(DateTime64(3))' as the type of the 'min_time' column, and the same for the 'max_time' column", 0) \ + M(Bool, filter_by_min_time_and_max_time, true, "If set to true then the table will use the 'min_time' and 'max_time' columns for filtering time series", 0) \ DECLARE_SETTINGS_TRAITS(TimeSeriesSettingsTraits, LIST_OF_TIME_SERIES_SETTINGS) From fae893e294ee26600146d4f67853fb74bb6ba8d1 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 16 Jun 2024 12:35:48 +0200 Subject: [PATCH 275/644] Add TimeSeries setting `use_all_tags_column_to_generate_id`. --- .../PrometheusRemoteWriteProtocol.cpp | 29 +++++++++++++------ .../TimeSeriesDefinitionNormalizer.cpp | 17 ++++++++++- .../TimeSeriesInnerTablesCreator.cpp | 19 +++++++----- src/Storages/TimeSeries/TimeSeriesSettings.h | 1 + 4 files changed, 49 insertions(+), 17 deletions(-) diff --git a/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp index ca62a2729dc..23d28894acc 100644 --- a/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp +++ b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp @@ -280,12 +280,18 @@ namespace auto & tags_offsets = tags_column.getNestedColumn().getOffsets(); /// Column "all_tags". - const auto & all_tags_description = get_column_description(TimeSeriesColumnNames::AllTags); - validator.validateColumnForTagsMap(all_tags_description); - auto & all_tags_column = typeid_cast(make_column_for_tags_block(all_tags_description)); - IColumn & all_tags_names = all_tags_column.getNestedData().getColumn(0); - IColumn & all_tags_values = all_tags_column.getNestedData().getColumn(1); - auto & all_tags_offsets = all_tags_column.getNestedColumn().getOffsets(); + IColumn * all_tags_names = nullptr; + IColumn * all_tags_values = nullptr; + IColumn::Offsets * all_tags_offsets = nullptr; + if (time_series_settings.use_all_tags_column_to_generate_id) + { + const auto & all_tags_description = get_column_description(TimeSeriesColumnNames::AllTags); + validator.validateColumnForTagsMap(all_tags_description); + auto & all_tags_column = typeid_cast(make_column_for_tags_block(all_tags_description)); + all_tags_names = &all_tags_column.getNestedData().getColumn(0); + all_tags_values = &all_tags_column.getNestedData().getColumn(1); + all_tags_offsets = &all_tags_column.getNestedColumn().getOffsets(); + } /// Columns "min_time" and "max_time". IColumn * min_time_column = nullptr; @@ -327,8 +333,11 @@ namespace } else { - all_tags_names.insertData(tag_name.data(), tag_name.length()); - all_tags_values.insertData(tag_value.data(), tag_value.length()); + if (time_series_settings.use_all_tags_column_to_generate_id) + { + all_tags_names->insertData(tag_name.data(), tag_name.length()); + all_tags_values->insertData(tag_value.data(), tag_value.length()); + } auto it = columns_by_tag_name.find(tag_name); bool has_column_for_tag_value = (it != columns_by_tag_name.end()); @@ -345,9 +354,11 @@ namespace } } - all_tags_offsets.push_back(all_tags_names.size()); tags_offsets.push_back(tags_names.size()); + if (time_series_settings.use_all_tags_column_to_generate_id) + all_tags_offsets->push_back(all_tags_names->size()); + if (time_series_settings.store_min_time_and_max_time) { auto [min_time, max_time] = findMinTimeAndMaxTime(element.samples()); diff --git a/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp index 8c2f56d9858..49dc2f3d1c1 100644 --- a/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp +++ b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp @@ -280,7 +280,22 @@ ASTPtr TimeSeriesDefinitionNormalizer::chooseIDAlgorithm(const ASTColumnDeclarat /// All hash functions below allow multiple arguments, so we use two arguments: metric_name, all_tags. ASTs arguments_for_hash_function; arguments_for_hash_function.push_back(std::make_shared(TimeSeriesColumnNames::MetricName)); - arguments_for_hash_function.push_back(std::make_shared(TimeSeriesColumnNames::AllTags)); + + if (time_series_settings.use_all_tags_column_to_generate_id) + { + arguments_for_hash_function.push_back(std::make_shared(TimeSeriesColumnNames::AllTags)); + } + else + { + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + arguments_for_hash_function.push_back(std::make_shared(column_name)); + } + arguments_for_hash_function.push_back(std::make_shared(TimeSeriesColumnNames::Tags)); + } auto make_hash_function = [&](const String & function_name) { diff --git a/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp index 5376ec2b124..130d49c4c68 100644 --- a/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp +++ b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -80,15 +82,18 @@ ColumnsDescription TimeSeriesInnerTablesCreator::getInnerTableColumnsDescription columns.add(time_series_columns.get(TimeSeriesColumnNames::Tags)); /// Column "all_tags". - ColumnDescription all_tags_column = time_series_columns.get(TimeSeriesColumnNames::AllTags); - /// Column "all_tags" is here only to calculate the identifier of a time series for the "id" column, so it can be ephemeral. - all_tags_column.default_desc.kind = ColumnDefaultKind::Ephemeral; - if (!all_tags_column.default_desc.expression) + if (time_series_settings.use_all_tags_column_to_generate_id) { - all_tags_column.default_desc.ephemeral_default = true; - all_tags_column.default_desc.expression = makeASTFunction("defaultValueOfTypeName", std::make_shared(all_tags_column.type->getName())); + ColumnDescription all_tags_column = time_series_columns.get(TimeSeriesColumnNames::AllTags); + /// Column "all_tags" is here only to calculate the identifier of a time series for the "id" column, so it can be ephemeral. + all_tags_column.default_desc.kind = ColumnDefaultKind::Ephemeral; + if (!all_tags_column.default_desc.expression) + { + all_tags_column.default_desc.ephemeral_default = true; + all_tags_column.default_desc.expression = makeASTFunction("defaultValueOfTypeName", std::make_shared(all_tags_column.type->getName())); + } + columns.add(std::move(all_tags_column)); } - columns.add(std::move(all_tags_column)); /// Columns "min_time" and "max_time". if (time_series_settings.store_min_time_and_max_time) diff --git a/src/Storages/TimeSeries/TimeSeriesSettings.h b/src/Storages/TimeSeries/TimeSeriesSettings.h index acbbc7e7806..4dc6a436cd0 100644 --- a/src/Storages/TimeSeries/TimeSeriesSettings.h +++ b/src/Storages/TimeSeries/TimeSeriesSettings.h @@ -9,6 +9,7 @@ class ASTStorage; #define LIST_OF_TIME_SERIES_SETTINGS(M, ALIAS) \ M(Map, tags_to_columns, Map{}, "Map specifying which tags should be put to separate columns of the 'tags' table. Syntax: {'tag1': 'column1', 'tag2' : column2, ...}", 0) \ + M(Bool, use_all_tags_column_to_generate_id, true, "When generating an expression to calculate an identifier of a time series, this flag enables using the 'all_tags' column in that calculation. The 'all_tags' is a virtual column containing all tags except the metric name", 0) \ M(Bool, store_min_time_and_max_time, true, "If set to true then the table will store 'min_time' and 'max_time' for each time series", 0) \ M(Bool, aggregate_min_time_and_max_time, true, "When creating an inner target 'tags' table, this flag enables using 'SimpleAggregateFunction(min, Nullable(DateTime64(3)))' instead of just 'Nullable(DateTime64(3))' as the type of the 'min_time' column, and the same for the 'max_time' column", 0) \ M(Bool, filter_by_min_time_and_max_time, true, "If set to true then the table will use the 'min_time' and 'max_time' columns for filtering time series", 0) \ From 6fb7ee3c68886af50488289440813ee7e8873d87 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 12 Jul 2024 01:59:54 +0200 Subject: [PATCH 276/644] Add global setting `allow_experimental_time_series_table`. --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/Storages/StorageTimeSeries.cpp | 9 +++++++++ 3 files changed, 11 insertions(+) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6988a66cf1e..115554d44c5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -903,6 +903,7 @@ class IColumn; M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions", 0) \ M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ + M(Bool, allow_experimental_time_series_table, false, "Allows to use the TimeSeries table engine. Disabled by default, because this feature is experimental", 0) \ M(Bool, allow_experimental_variant_type, false, "Allow Variant data type", 0) \ M(Bool, allow_experimental_dynamic_type, false, "Allow Dynamic data type", 0) \ M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b6ef654438e..75a80694d43 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -77,6 +77,7 @@ static std::initializer_list +#include #include #include #include @@ -25,6 +26,7 @@ namespace ErrorCodes extern const int INCORRECT_QUERY; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; + extern const int SUPPORT_IS_DISABLED; extern const int UNEXPECTED_TABLE_ENGINE; } @@ -124,6 +126,13 @@ StorageTimeSeries::StorageTimeSeries( : IStorage(table_id) , WithContext(local_context->getGlobalContext()) { + if (mode <= LoadingStrictnessLevel::CREATE && !local_context->getSettingsRef().allow_experimental_time_series_table) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Experimental TimeSeries table engine " + "is not enabled (the setting 'allow_experimental_time_series_table')"); + } + storage_settings = getTimeSeriesSettingsFromQuery(query); if (mode < LoadingStrictnessLevel::ATTACH) From 083fff6ed6ccff44b678ae3ea6af75501d9359fb Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 17 Jun 2024 10:39:10 +0200 Subject: [PATCH 277/644] Add documentation. --- docs/en/engines/table-engines/index.md | 1 + .../table-engines/integrations/time-series.md | 295 ++++++++++++++++++ docs/en/interfaces/prometheus.md | 160 ++++++++++ .../settings.md | 42 --- docs/en/operations/settings/settings.md | 11 + .../table-functions/timeSeriesData.md | 28 ++ .../table-functions/timeSeriesMetrics.md | 28 ++ .../table-functions/timeSeriesTags.md | 28 ++ .../aspell-ignore/en/aspell-dict.txt | 6 + 9 files changed, 557 insertions(+), 42 deletions(-) create mode 100644 docs/en/engines/table-engines/integrations/time-series.md create mode 100644 docs/en/interfaces/prometheus.md create mode 100644 docs/en/sql-reference/table-functions/timeSeriesData.md create mode 100644 docs/en/sql-reference/table-functions/timeSeriesMetrics.md create mode 100644 docs/en/sql-reference/table-functions/timeSeriesTags.md diff --git a/docs/en/engines/table-engines/index.md b/docs/en/engines/table-engines/index.md index 5e81eacc937..20c7c511aa9 100644 --- a/docs/en/engines/table-engines/index.md +++ b/docs/en/engines/table-engines/index.md @@ -61,6 +61,7 @@ Engines in the family: - [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md) - [PostgreSQL](../../engines/table-engines/integrations/postgresql.md) - [S3Queue](../../engines/table-engines/integrations/s3queue.md) +- [TimeSeries](../../engines/table-engines/integrations/time-series.md) ### Special Engines {#special-engines} diff --git a/docs/en/engines/table-engines/integrations/time-series.md b/docs/en/engines/table-engines/integrations/time-series.md new file mode 100644 index 00000000000..4830fd61d27 --- /dev/null +++ b/docs/en/engines/table-engines/integrations/time-series.md @@ -0,0 +1,295 @@ +--- +slug: /en/engines/table-engines/special/time_series +sidebar_position: 60 +sidebar_label: TimeSeries +--- + +# TimeSeries Engine [Experimental] + +A table engine storing time series, i.e. a set of values associated with timestamps and tags (or labels): + +``` +metric_name1[tag1=value1, tag2=value2, ...] = {timestamp1: value1, timestamp2: value2, ...} +metric_name2[...] = ... +``` + +:::info +This is an experimental feature that may change in backwards-incompatible ways in the future releases. +Enable usage of the TimeSeries table engine +with [allow_experimental_time_series_table](../../../operations/settings/settings.md#allow-experimental-time-series-table) setting. +Input the command `set allow_experimental_time_series_table = 1`. +::: + +## Syntax {#syntax} + +``` sql +CREATE TABLE name [(columns)] ENGINE=TimeSeries +[SETTINGS var1=value1, ...] +[DATA db.data_table_name | DATA ENGINE data_table_engine(arguments)] +[TAGS db.tags_table_name | TAGS ENGINE tags_table_engine(arguments)] +[METRICS db.metrics_table_name | METRICS ENGINE metrics_table_engine(arguments)] +``` + +## Usage {#usage} + +It's easier to start with everything set by default (it's allowed to create a `TimeSeries` table without specifying a list of columns): + +``` sql +CREATE TABLE my_table ENGINE=TimeSeries +``` + +Then this table can be used with the following protocols (a port must be assigned in the server configuration): +- [prometheus remote-write](../../../interfaces/prometheus.md#remote-write) +- [prometheus remote-read](../../../interfaces/prometheus.md#remote-read) + +## Target tables {#target-tables} + +A `TimeSeries` table doesn't have its own data, everything is stored in its target tables. +This is similar to how a [materialized view](../../../sql-reference/statements/create/view#materialized-view) works, +with the difference that a materialized view has one target table +whereas a `TimeSeries` table has three target tables named [data]{#data-table}, [tags]{#tags-table], and [metrics]{#metrics-table}. + +The target tables can be either specified explicitly in the `CREATE TABLE` query +or the `TimeSeries` table engine can generate inner target tables automatically. + +The target tables are the following: +1. The _data_ table {#data-table} contains time series associated with some identifier. +The _data_ table must have columns: + +| Name | Mandatory? | Default type | Possible types | Description | +|---|---|---|---|---| +| `id` | [x] | `UUID` | any | Identifies a combination of a metric names and tags | +| `timestamp` | [x] | `DateTime64(3)` | `DateTime64(X)` | A time point | +| `value` | [x] | `Float64` | `Float32` or `Float64` | A value associated with the `timestamp` | + +2. The _tags_ table {#tags-table} contains identifiers calculated for each combination of a metric name and tags. +The _tags_ table must have columns: + +| Name | Mandatory? | Default type | Possible types | Description | +|---|---|---|---|---| +| `id` | [x] | `UUID` | any (must match the type of `id` in the [data]{#data-table} table) | An `id` identifies a combination of a metric name and tags. The DEFAULT expression specifies how to calculate such an identifier | +| `metric_name` | [x] | `LowCardinality(String)` | `String` or `LowCardinality(String)` | The name of a metric | +| `` | [ ] | `String` | `String` or `LowCardinality(String)` or `LowCardinality(Nullable(String))` | The value of a specific tag, the tag's name and the name of a corresponding column are specified in the [tags_to_columns](#settings) setting | +| `tags` | [x] | `Map(LowCardinality(String), String)` | `Map(String, String)` or `Map(LowCardinality(String), String)` or `Map(LowCardinality(String), LowCardinality(String))` | Map of tags excluding the tag `__name__` containing the name of a metric and excluding tags with names enumerated in the [tags_to_columns](#settings) setting | +| `all_tags` | [ ] | `Map(String, String)` | `Map(String, String)` or `Map(LowCardinality(String), String)` or `Map(LowCardinality(String), LowCardinality(String))` | Ephemeral column, each row is a map of all the tags excluding only the tag `__name__` containing the name of a metric. The only purpose of that column is to be used while calculating `id` | +| `min_time` | [ ] | `Nullable(DateTime64(3))` | `DateTime64(X)` or `Nullable(DateTime64(X))` | Minimum timestamp of time series with that `id`. The column is created if [store_min_time_and_max_time](#settings) is `true` | +| `max_time` | [ ] | `Nullable(DateTime64(3))` | `DateTime64(X)` or `Nullable(DateTime64(X))` | Maximum timestamp of time series with that `id`. The column is created if [store_min_time_and_max_time](#settings) is `true` | + +3. The _metrics_ table {#metrics-table} contains some information about metrics been collected, the types of those metrics and their descriptions. +The _metrics_ table must have columns: + +| Name | Mandatory? | Default type | Possible types | Description | +|---|---|---|---|---| +| `metric_family_name` | [x] | `String` | `String` or `LowCardinality(String)` | The name of a metric family | +| `type` | [x] | `String` | `String` or `LowCardinality(String)` | The type of a metric family, one of "counter", "gauge", "summary", "stateset", "histogram", "gaugehistogram" | +| `unit` | [x] | `String` | `String` or `LowCardinality(String)` | The unit used in a metric | +| `help` | [x] | `String` | `String` or `LowCardinality(String)` | The description of a metric | + +Any row inserted into a `TimeSeries` table will be in fact stored in those three target tables. +A `TimeSeries` table contains all those columns from the [data]{#data-table}, [tags]{#tags-table}, [metrics]{#metrics-table} tables. + +## Creation {#creation} + +There are multiple ways to create a table with the `TimeSeries` table engine. +The simplest statement + +``` sql +CREATE TABLE my_table ENGINE=TimeSeries +``` + +will actually create the following table (you can see that by executing `SHOW CREATE TABLE my_table`): + +``` sql +CREATE TABLE my_table +( + `id` UUID DEFAULT reinterpretAsUUID(sipHash128(metric_name, all_tags)), + `timestamp` DateTime64(3), + `value` Float64, + `metric_name` LowCardinality(String), + `tags` Map(LowCardinality(String), String), + `all_tags` Map(String, String), + `min_time` Nullable(DateTime64(3)), + `max_time` Nullable(DateTime64(3)), + `metric_family_name` String, + `type` String, + `unit` String, + `help` String +) +ENGINE = TimeSeries +DATA ENGINE = MergeTree ORDER BY (id, timestamp) +DATA INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +TAGS ENGINE = AggregatingMergeTree PRIMARY KEY metric_name ORDER BY (metric_name, id) +TAGS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +METRICS ENGINE = ReplacingMergeTree ORDER BY metric_family_name +METRICS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +``` + +So the columns were generated automatically and also there are three inner UUIDs in this statement - +one per each inner target table that was created. +(Inner UUIDs are not shown normally until setting +[show_table_uuid_in_table_create_query_if_not_nil](../../../operations/settings/settings#show_table_uuid_in_table_create_query_if_not_nil) +is set.) + +Inner target tables have names like `.inner_id.data.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`, +`.inner_id.tags.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`, `.inner_id.metrics.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` +and each target table has columns which is a subset of the columns of the main `TimeSeries` table: + +``` sql +CREATE TABLE default.`.inner_id.data.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` +( + `id` UUID, + `timestamp` DateTime64(3), + `value` Float64 +) +ENGINE = MergeTree +ORDER BY (id, timestamp) +``` + +``` sql +CREATE TABLE default.`.inner_id.tags.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` +( + `id` UUID DEFAULT reinterpretAsUUID(sipHash128(metric_name, all_tags)), + `metric_name` LowCardinality(String), + `tags` Map(LowCardinality(String), String), + `all_tags` Map(String, String) EPHEMERAL, + `min_time` SimpleAggregateFunction(min, Nullable(DateTime64(3))), + `max_time` SimpleAggregateFunction(max, Nullable(DateTime64(3))) +) +ENGINE = AggregatingMergeTree +PRIMARY KEY metric_name +ORDER BY (metric_name, id) +``` + +``` sql +CREATE TABLE default.`.inner_id.metrics.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` +( + `metric_family_name` String, + `type` String, + `unit` String, + `help` String +) +ENGINE = ReplacingMergeTree +ORDER BY metric_family_name +``` + +## Adjusting types of columns {#adjusting-column-types} + +You can adjust the types of almost any column of the inner target tables by specifying them explicitly +while defining the main table. For example, + +``` sql +CREATE TABLE my_table +( + timestamp DateTime64(6) +) ENGINE=TimeSeries +``` + +will make the inner [data]{#data-table} table store timestamp in microseconds instead of milliseconds: + +``` sql +CREATE TABLE default.`.inner_id.data.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` +( + `id` UUID, + `timestamp` DateTime64(6), + `value` Float64 +) +ENGINE = MergeTree +ORDER BY (id, timestamp) +``` + +## The `id` column {#id-column} + +The `id` column contains identifiers, every identifier is calculated for a combination of a metric name and tags. +The DEFAULT expression for the `id` column is an expression which will be used to calculate such identifiers. +Both the type of the `id` column and that expression can be adjusted by specifying them explicitly: + +``` sql +CREATE TABLE my_table +( + id UInt64 DEFAULT sipHash64(metric_name, all_tags) +) ENGINE=TimeSeries +``` + +## The `tags` and `all_tags` columns {#tags-and-all-tags} + +There are two columns containing maps of tags - `tags` and `all_tags`. In this example they mean the same, however they can be different +if setting `tags_to_columns` is used. This setting allows to specify that a specific tag should be stored in a separate column instead of storing +in a map inside the `tags` column: + +``` sql +CREATE TABLE my_table ENGINE=TimeSeries SETTINGS = {'instance': 'instance', 'job': 'job'} +``` + +This statement will add columns +``` + `instance` String, + `job` String +``` +to the definition of both `my_table` and its inner [tags]{#tags-table} target table. In this case the `tags` column will not contain tags `instance` and `job`, +but the `all_tags` column will contain them. The `all_tags` column is ephemeral and its only purpose to be used in the DEFAULT expression +for the `id` column. + +The types of columns can be adjusted by specifying them explicitly: + +``` sql +CREATE TABLE my_table (instance LowCardinality(String), job LowCardinality(Nullable(String))) +ENGINE=TimeSeries SETTINGS = {'instance': 'instance', 'job': 'job'} +``` + +## Table engines of inner target tables {#inner-table-engines} + +By default inner target tables use the following table engines: +- the [data]{#data-table} table uses [MergeTree](../mergetree-family/mergetree); +- the [tags]{#tags-table} table uses [AggregatingMergeTree](../mergetree-family/aggregatingmergetree) because the same data is often inserted multiple times to this table so we need a way +to remove duplicates, and also because it's required to do aggregation for columns `min_time` and `max_time`; +- the [metrics]{#metrics-table} table uses [ReplacingMergeTree](../mergetree-family/replacingmergetree) because the same data is often inserted multiple times to this table so we need a way +to remove duplicates. + +Other table engines also can be used for inner target tables if it's specified so: + +``` sql +CREATE TABLE my_table ENGINE=TimeSeries +DATA ENGINE=ReplicatedMergeTree +TAGS ENGINE=ReplicatedAggregatingMergeTree +METRICS ENGINE=ReplicatedReplacingMergeTree +``` + +## External target tables {#external-target-tables} + +It's possible to make a `TimeSeries` table use a manually created table: + +``` sql +CREATE TABLE data_for_my_table +( + `id` UUID, + `timestamp` DateTime64(3), + `value` Float64 +) +ENGINE = MergeTree +ORDER BY (id, timestamp); + +CREATE TABLE tags_for_my_table ... + +CREATE TABLE metrics_for_my_table ... + +CREATE TABLE my_table ENGINE=TimeSeries DATA data_for_my_table TAGS tags_for_my_table METRICS metrics_for_my_table; +``` + +## Settings {#settings} + +Here is a list of settings which can be specified while defining a `TimeSeries` table: + +| Name | Type | Default | Description | +|---|---|---|---| +| `tags_to_columns` | Map | {} | Map specifying which tags should be put to separate columns in the [tags]{#tags-table} table. Syntax: `{'tag1': 'column1', 'tag2' : column2, ...}` | +| `use_all_tags_column_to_generate_id` | Bool | true | When generating an expression to calculate an identifier of a time series, this flag enables using the `all_tags` column in that calculation | +| `store_min_time_and_max_time` | Bool | true | If set to true then the table will store `min_time` and `max_time` for each time series | +| `aggregate_min_time_and_max_time` | Bool | true | When creating an inner target `tags` table, this flag enables using `SimpleAggregateFunction(min, Nullable(DateTime64(3)))` instead of just `Nullable(DateTime64(3))` as the type of the `min_time` column, and the same for the `max_time` column | +| `filter_by_min_time_and_max_time` | Bool | true | If set to true then the table will use the `min_time` and `max_time` columns for filtering time series | + +# Functions {#functions} + +Here is a list of functions supporting a `TimeSeries` table as an argument: +- [timeSeriesData](../../../sql-reference/table-functions/timeSeriesData.md) +- [timeSeriesTags](../../../sql-reference/table-functions/timeSeriesTags.md) +- [timeSeriesMetrics](../../../sql-reference/table-functions/timeSeriesMetrics.md) diff --git a/docs/en/interfaces/prometheus.md b/docs/en/interfaces/prometheus.md new file mode 100644 index 00000000000..75a68c59219 --- /dev/null +++ b/docs/en/interfaces/prometheus.md @@ -0,0 +1,160 @@ +--- +slug: /en/interfaces/prometheus +sidebar_position: 19 +sidebar_label: Prometheus protocols +--- + +# Prometheus protocols + +## Exposing metrics {#expose} + +:::note +ClickHouse Cloud does not currently support connecting to Prometheus. To be notified when this feature is supported, please contact support@clickhouse.com. +::: + +ClickHouse can expose its own metrics for scraping from Prometheus: + +```xml + + 9363 + /metrics + true + true + true + true + + +Section `` can be used to make more extended handlers. +This section is similar to [](http.md) but works for prometheus protocols: + +```xml + + 9363 + + + /metrics + + expose_metrics + true + true + true + true + + + + +``` + +Settings: + +| Name | Default | Description | +|---|---|---|---| +| `port` | none | Port for serving the exposing metrics protocol. | +| `endpoint` | `/metrics` | HTTP endpoint for scraping metrics by prometheus server. Starts with `/`. Should not be used with the `` section. | +| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](http.md) section. | +| `metrics` | true | Expose metrics from the [system.metrics](../operations/system-tables/metrics.md) table. | +| `asynchronous_metrics` | true | Expose current metrics values from the [system.asynchronous_metrics](../operations/system-tables/asynchronous_metrics.md) table. | +| `events` | true | Expose metrics from the [system.events](../operations/system-tables/events.md) table. | +| `errors` | true | Expose the number of errors by error codes occurred since the last server restart. This information could be obtained from the [system.errors](../operations/system-tables/errors.md) as well. | + +Check (replace `127.0.0.1` with the IP addr or hostname of your ClickHouse server): +```bash +curl 127.0.0.1:9363/metrics +``` + +## Remote-write protocol {#remote-write} + +ClickHouse supports the [remote-write](https://prometheus.io/docs/specs/remote_write_spec/) protocol. +Data are received by this protocol and written to a [TimeSeries](../engines/table-engines/integrations/time-series.md) table +(which should be created beforehand). + +```xml + + 9363 + + + /write + + remote_writedb_name + time_series_table
+
+
+
+
+``` + +Settings: + +| Name | Default | Description | +|---|---|---|---| +| `port` | none | Port for serving the `remote-write` protocol. | +| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](http.md) section. | +| `table` | none | The name of a [TimeSeries](../engines/table-engines/integrations/time-series.md) table to write data received by the `remote-write` protocol. This name can optionally contain the name of a database too. | +| `database` | none | The name of a database where the table specified in the `table` setting is located if it's not specified in the `table` setting. | + +## Remote-read protocol {#remote-read} + +ClickHouse supports the [remote-read](https://prometheus.io/docs/prometheus/latest/querying/remote_read_api/) protocol. +Data are read from a [TimeSeries](../engines/table-engines/integrations/time-series.md) table and sent via this protocol. + +```xml + + 9363 + + + /read + + remote_readdb_name + time_series_table
+
+
+
+
+``` + +Settings: + +| Name | Default | Description | +|---|---|---|---| +| `port` | none | Port for serving the `remote-read` protocol. | +| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](http.md) section. | +| `table` | none | The name of a [TimeSeries](../engines/table-engines/integrations/time-series.md) table to read data to send by the `remote-read` protocol. This name can optionally contain the name of a database too. | +| `database` | none | The name of a database where the table specified in the `table` setting is located if it's not specified in the `table` setting. | + +## Configuration for multiple protocols {#multiple-protocols} + +Multiple protocols can be specified together in one place: + +```xml + + 9363 + + + /metrics + + expose_metrics + true + true + true + true + + + + /write + + remote_writedb_name.time_series_table + + + + /read + + remote_readdb_name.time_series_table + + + + +``` diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index a1e3c292b04..68f61650e00 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2112,48 +2112,6 @@ The trailing slash is mandatory. /var/lib/clickhouse/ ``` -## Prometheus {#prometheus} - -:::note -ClickHouse Cloud does not currently support connecting to Prometheus. To be notified when this feature is supported, please contact support@clickhouse.com. -::: - -Exposing metrics data for scraping from [Prometheus](https://prometheus.io). - -Settings: - -- `endpoint` – HTTP endpoint for scraping metrics by prometheus server. Start from ‘/’. -- `port` – Port for `endpoint`. -- `metrics` – Expose metrics from the [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) table. -- `events` – Expose metrics from the [system.events](../../operations/system-tables/events.md#system_tables-events) table. -- `asynchronous_metrics` – Expose current metrics values from the [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) table. -- `errors` - Expose the number of errors by error codes occurred since the last server restart. This information could be obtained from the [system.errors](../../operations/system-tables/asynchronous_metrics.md#system_tables-errors) as well. - -**Example** - -``` xml - - 0.0.0.0 - 8123 - 9000 - - - /metrics - 9363 - true - true - true - true - - - -``` - -Check (replace `127.0.0.1` with the IP addr or hostname of your ClickHouse server): -```bash -curl 127.0.0.1:9363/metrics -``` - ## query_log {#query-log} Setting for logging queries received with the [log_queries=1](../../operations/settings/settings.md) setting. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 143ce836beb..c0103aca5f6 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5614,3 +5614,14 @@ Default value: `1GiB`. Disable all insert and mutations (alter table update / alter table delete / alter table drop partition). Set to true, can make this node focus on reading queries. Default value: `false`. + +## allow_experimental_time_series_table {#allow-experimental-time-series-table} + +Allows creation of tables with the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine. + +Possible values: + +- 0 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is disabled. +- 1 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is enabled. + +Default value: `0`. diff --git a/docs/en/sql-reference/table-functions/timeSeriesData.md b/docs/en/sql-reference/table-functions/timeSeriesData.md new file mode 100644 index 00000000000..aa7a9d30c2a --- /dev/null +++ b/docs/en/sql-reference/table-functions/timeSeriesData.md @@ -0,0 +1,28 @@ +--- +slug: /en/sql-reference/table-functions/timeSeriesData +sidebar_position: 145 +sidebar_label: timeSeriesData +--- + +# timeSeriesData + +`timeSeriesData(db_name.time_series_table)` - Returns the [data](../../engines/table-engines/integrations/time-series.md#data-table) table +used by table `db_name.time_series_table` which table engine is [TimeSeries](../../engines/table-engines/integrations/time-series.md): + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries DATA data_table +``` + +The function also works if the _data_ table is inner: + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries DATA INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +``` + +The following queries are equivalent: + +``` sql +SELECT * FROM timeSeriesData(db_name.time_series_table); +SELECT * FROM timeSeriesData('db_name.time_series_table'); +SELECT * FROM timeSeriesData('db_name', 'time_series_table'); +``` diff --git a/docs/en/sql-reference/table-functions/timeSeriesMetrics.md b/docs/en/sql-reference/table-functions/timeSeriesMetrics.md new file mode 100644 index 00000000000..913f1185bca --- /dev/null +++ b/docs/en/sql-reference/table-functions/timeSeriesMetrics.md @@ -0,0 +1,28 @@ +--- +slug: /en/sql-reference/table-functions/timeSeriesMetrics +sidebar_position: 145 +sidebar_label: timeSeriesMetrics +--- + +# timeSeriesMetrics + +`timeSeriesMetrics(db_name.time_series_table)` - Returns the [metrics](../../engines/table-engines/integrations/time-series.md#metrics-table) table +used by table `db_name.time_series_table` which table engine is [TimeSeries](../../engines/table-engines/integrations/time-series.md): + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries METRICS metrics_table +``` + +The function also works if the _metrics_ table is inner: + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries METRICS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +``` + +The following queries are equivalent: + +``` sql +SELECT * FROM timeSeriesMetrics(db_name.time_series_table); +SELECT * FROM timeSeriesMetrics('db_name.time_series_table'); +SELECT * FROM timeSeriesMetrics('db_name', 'time_series_table'); +``` diff --git a/docs/en/sql-reference/table-functions/timeSeriesTags.md b/docs/en/sql-reference/table-functions/timeSeriesTags.md new file mode 100644 index 00000000000..663a7dc6ac8 --- /dev/null +++ b/docs/en/sql-reference/table-functions/timeSeriesTags.md @@ -0,0 +1,28 @@ +--- +slug: /en/sql-reference/table-functions/timeSeriesTags +sidebar_position: 145 +sidebar_label: timeSeriesTags +--- + +# timeSeriesTags + +`timeSeriesTags(db_name.time_series_table)` - Returns the [tags](../../engines/table-engines/integrations/time-series.md#tags-table) table +used by table `db_name.time_series_table` which table engine is [TimeSeries](../../engines/table-engines/integrations/time-series.md): + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries TAGS tags_table +``` + +The function also works if the _tags_ table is inner: + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries TAGS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +``` + +The following queries are equivalent: + +``` sql +SELECT * FROM timeSeriesTags(db_name.time_series_table); +SELECT * FROM timeSeriesTags('db_name.time_series_table'); +SELECT * FROM timeSeriesTags('db_name', 'time_series_table'); +``` diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index d82b70cfdb4..47d30759174 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1672,6 +1672,7 @@ fuzzQuery fuzzer fuzzers gRPC +gaugehistogram gccMurmurHash gcem generateRandom @@ -2555,6 +2556,7 @@ startsWithUTF startswith statbox stateful +stateset stddev stddevPop stddevPopStable @@ -2686,6 +2688,10 @@ themself threadpool throwIf timeDiff +TimeSeries +timeSeriesData +timeSeriesMetrics +timeSeriesTags timeSlot timeSlots timeZone From bd22140fa4a0c48eb99216889ef4c1a02ba36404 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 11 Mar 2024 23:58:05 +0100 Subject: [PATCH 278/644] Add test. --- .../compose/docker_compose_prometheus.yml | 57 ++++++ tests/integration/helpers/cluster.py | 75 ++++++++ .../test_prometheus_protocols/__init__.py | 0 .../allow_experimental_time_series_table.xml | 7 + .../configs/prometheus.xml | 21 +++ .../test_prometheus_protocols/test.py | 168 ++++++++++++++++++ 6 files changed, 328 insertions(+) create mode 100644 tests/integration/compose/docker_compose_prometheus.yml create mode 100644 tests/integration/test_prometheus_protocols/__init__.py create mode 100644 tests/integration/test_prometheus_protocols/configs/allow_experimental_time_series_table.xml create mode 100644 tests/integration/test_prometheus_protocols/configs/prometheus.xml create mode 100644 tests/integration/test_prometheus_protocols/test.py diff --git a/tests/integration/compose/docker_compose_prometheus.yml b/tests/integration/compose/docker_compose_prometheus.yml new file mode 100644 index 00000000000..0a1db2138ba --- /dev/null +++ b/tests/integration/compose/docker_compose_prometheus.yml @@ -0,0 +1,57 @@ +version: '2.3' +services: + prometheus_writer: + image: prom/prometheus:v2.50.1 + hostname: ${PROMETHEUS_WRITER_HOSTNAME:-prometheus_writer} + restart: always + entrypoint: | + /bin/sh -c 'truncate -s 0 /etc/prometheus/prometheus.yml + cat << EOF >> /etc/prometheus/prometheus.yml + global: + scrape_interval: 1s + scrape_configs: + - job_name: "prometheus" + static_configs: + - targets: ["localhost:${PROMETHEUS_WRITER_PORT}"] + EOF + if [ -n "${PROMETHEUS_REMOTE_WRITE_HANDLER}" ]; then + echo "remote_write:" >> /etc/prometheus/prometheus.yml + echo " - url: \"${PROMETHEUS_REMOTE_WRITE_HANDLER}\"" >> /etc/prometheus/prometheus.yml + fi + #cat "/etc/prometheus/prometheus.yml" + /bin/prometheus --config.file="/etc/prometheus/prometheus.yml" --storage.tsdb.path="/prometheus" --web.console.libraries="/usr/share/prometheus/console_libraries" --web.console.templates="/usr/share/prometheus/consoles" --web.listen-address="0.0.0.0:${PROMETHEUS_WRITER_PORT}" &> /var/log/prometheus/prometheus.log' + expose: + - ${PROMETHEUS_WRITER_PORT} + healthcheck: + test: curl -f "ttps://localhost:${PROMETHEUS_WRITER_PORT}/api/v1/status/runtimeinfo" || exit 1 + interval: 5s + timeout: 3s + retries: 30 + volumes: + - type: ${PROMETHEUS_WRITER_LOGS_FS:-tmpfs} + source: ${PROMETHEUS_WRITER_LOGS:-} + target: /var/log/prometheus + + prometheus_reader: + image: prom/prometheus:v2.50.1 + hostname: ${PROMETHEUS_READER_HOSTNAME:-prometheus_reader} + restart: always + entrypoint: | + /bin/sh -c 'truncate -s 0 /etc/prometheus/prometheus.yml + if [ -n "${PROMETHEUS_REMOTE_READ_HANDLER}" ]; then + echo "remote_read:" >> /etc/prometheus/prometheus.yml + echo " - url: \"${PROMETHEUS_REMOTE_READ_HANDLER}\"" >> /etc/prometheus/prometheus.yml + fi + #cat "/etc/prometheus/prometheus.yml" + /bin/prometheus --config.file="/etc/prometheus/prometheus.yml" --storage.tsdb.path="/prometheus" --web.console.libraries="/usr/share/prometheus/console_libraries" --web.console.templates="/usr/share/prometheus/consoles" --web.listen-address="0.0.0.0:${PROMETHEUS_READER_PORT}" &> /var/log/prometheus/prometheus.log' + expose: + - ${PROMETHEUS_READER_PORT} + healthcheck: + test: curl -f "ttps://localhost:${PROMETHEUS_READER_PORT}/api/v1/status/runtimeinfo" || exit 1 + interval: 5s + timeout: 3s + retries: 30 + volumes: + - type: ${PROMETHEUS_READER_LOGS_FS:-tmpfs} + source: ${PROMETHEUS_READER_LOGS:-} + target: /var/log/prometheus diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 9259c720ff0..a0b545654fe 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -737,6 +737,25 @@ class ClickHouseCluster: self.jdbc_driver_dir = p.abspath(p.join(self.instances_dir, "jdbc_driver")) self.jdbc_driver_logs_dir = os.path.join(self.jdbc_driver_dir, "logs") + # available when with_prometheus == True + self.with_prometheus = False + self.prometheus_writer_host = "prometheus_writer" + self.prometheus_writer_port = 9090 + self.prometheus_writer_logs_dir = p.abspath( + p.join(self.instances_dir, "prometheus_writer/logs") + ) + self.prometheus_reader_host = "prometheus_reader" + self.prometheus_reader_port = 9091 + self.prometheus_reader_logs_dir = p.abspath( + p.join(self.instances_dir, "prometheus_reader/logs") + ) + self.prometheus_remote_write_handler_host = None + self.prometheus_remote_write_handler_port = 9092 + self.prometheus_remote_write_handler_path = "/write" + self.prometheus_remote_read_handler_host = None + self.prometheus_remote_read_handler_port = 9092 + self.prometheus_remote_read_handler_path = "/read" + self.docker_client = None self.is_up = False self.env = os.environ.copy() @@ -1619,6 +1638,42 @@ class ClickHouseCluster: ] return self.base_hive_cmd + def setup_prometheus_cmd(self, instance, env_variables, docker_compose_yml_dir): + env_variables["PROMETHEUS_WRITER_HOST"] = self.prometheus_writer_host + env_variables["PROMETHEUS_WRITER_PORT"] = str(self.prometheus_writer_port) + env_variables["PROMETHEUS_WRITER_LOGS"] = self.prometheus_writer_logs_dir + env_variables["PROMETHEUS_WRITER_LOGS_FS"] = "bind" + env_variables["PROMETHEUS_READER_HOST"] = self.prometheus_reader_host + env_variables["PROMETHEUS_READER_PORT"] = str(self.prometheus_reader_port) + env_variables["PROMETHEUS_READER_LOGS"] = self.prometheus_reader_logs_dir + env_variables["PROMETHEUS_READER_LOGS_FS"] = "bind" + if self.prometheus_remote_write_handler_host: + env_variables["PROMETHEUS_REMOTE_WRITE_HANDLER"] = ( + f"http://{self.prometheus_remote_write_handler_host}:{self.prometheus_remote_write_handler_port}/{self.prometheus_remote_write_handler_path.strip('/')}" + ) + if self.prometheus_remote_read_handler_host: + env_variables["PROMETHEUS_REMOTE_READ_HANDLER"] = ( + f"http://{self.prometheus_remote_read_handler_host}:{self.prometheus_remote_read_handler_port}/{self.prometheus_remote_read_handler_path.strip('/')}" + ) + if not self.with_prometheus: + self.with_prometheus = True + self.base_cmd.extend( + [ + "--file", + p.join(docker_compose_yml_dir, "docker_compose_prometheus.yml"), + ] + ) + self.base_prometheus_cmd = [ + "docker-compose", + "--env-file", + instance.env_file, + "--project-name", + self.project_name, + "--file", + p.join(docker_compose_yml_dir, "docker_compose_prometheus.yml"), + ] + return self.base_prometheus_cmd + def add_instance( self, name, @@ -1659,6 +1714,9 @@ class ClickHouseCluster: with_jdbc_bridge=False, with_hive=False, with_coredns=False, + with_prometheus=False, + handle_prometheus_remote_write=False, + handle_prometheus_remote_read=False, use_old_analyzer=None, hostname=None, env_variables=None, @@ -2001,6 +2059,17 @@ class ClickHouseCluster: self.setup_hive(instance, env_variables, docker_compose_yml_dir) ) + if with_prometheus: + if handle_prometheus_remote_write: + self.prometheus_remote_write_handler_host = instance.hostname + if handle_prometheus_remote_read: + self.prometheus_remote_read_handler_host = instance.hostname + cmds.append( + self.setup_prometheus_cmd( + instance, env_variables, docker_compose_yml_dir + ) + ) + logging.debug( "Cluster name:{} project_name:{}. Added instance name:{} tag:{} base_cmd:{} docker_compose_yml_dir:{}".format( self.name, @@ -3056,6 +3125,12 @@ class ClickHouseCluster: f"http://{self.jdbc_bridge_ip}:{self.jdbc_bridge_port}/ping" ) + if self.with_prometheus: + os.makedirs(self.prometheus_writer_logs_dir) + os.chmod(self.prometheus_writer_logs_dir, stat.S_IRWXU | stat.S_IRWXO) + os.makedirs(self.prometheus_reader_logs_dir) + os.chmod(self.prometheus_reader_logs_dir, stat.S_IRWXU | stat.S_IRWXO) + clickhouse_start_cmd = self.base_cmd + ["up", "-d", "--no-recreate"] logging.debug( ( diff --git a/tests/integration/test_prometheus_protocols/__init__.py b/tests/integration/test_prometheus_protocols/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_prometheus_protocols/configs/allow_experimental_time_series_table.xml b/tests/integration/test_prometheus_protocols/configs/allow_experimental_time_series_table.xml new file mode 100644 index 00000000000..d71cfcaf2c8 --- /dev/null +++ b/tests/integration/test_prometheus_protocols/configs/allow_experimental_time_series_table.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_prometheus_protocols/configs/prometheus.xml b/tests/integration/test_prometheus_protocols/configs/prometheus.xml new file mode 100644 index 00000000000..071a29620cf --- /dev/null +++ b/tests/integration/test_prometheus_protocols/configs/prometheus.xml @@ -0,0 +1,21 @@ + + + 9092 + + + /write + + remote_write + default.prometheus
+
+
+ + /read + + remote_read + default.prometheus
+
+
+
+
+
diff --git a/tests/integration/test_prometheus_protocols/test.py b/tests/integration/test_prometheus_protocols/test.py new file mode 100644 index 00000000000..488c5369742 --- /dev/null +++ b/tests/integration/test_prometheus_protocols/test.py @@ -0,0 +1,168 @@ +import pytest +import time +import requests +from http import HTTPStatus +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/prometheus.xml"], + user_configs=["configs/allow_experimental_time_series_table.xml"], + with_prometheus=True, + handle_prometheus_remote_write=True, + handle_prometheus_remote_read=True, +) + + +def execute_query_on_prometheus_writer(query, timestamp): + return execute_query_impl( + cluster.get_instance_ip(cluster.prometheus_writer_host), + cluster.prometheus_writer_port, + "/api/v1/query", + query, + timestamp, + ) + + +def execute_query_on_prometheus_reader(query, timestamp): + return execute_query_impl( + cluster.get_instance_ip(cluster.prometheus_reader_host), + cluster.prometheus_reader_port, + "/api/v1/query", + query, + timestamp, + ) + + +def execute_query_impl(host, port, path, query, timestamp): + if not path.startswith("/"): + path += "/" + url = f"http://{host}:{port}/{path.strip('/')}?query={query}&time={timestamp}" + print(f"Requesting {url}") + r = requests.get(url) + print(f"Status code: {r.status_code} {HTTPStatus(r.status_code).phrase}") + if r.status_code != requests.codes.ok: + print(f"Response: {r.text}") + raise Exception(f"Got unexpected status code {r.status_code}") + return r.json() + + +def show_query_result(query): + evaluation_time = time.time() + print(f"Evaluating query: {query}") + print(f"Evaluation time: {evaluation_time}") + result_from_writer = execute_query_on_prometheus_writer(query, evaluation_time) + print(f"Result from prometheus_writer: {result_from_writer}") + result_from_reader = execute_query_on_prometheus_reader(query, evaluation_time) + print(f"Result from prometheus_reader: {result_from_reader}") + + +def compare_query(query): + timeout = 30 + start_time = time.time() + evaluation_time = start_time + print(f"Evaluating query: {query}") + print(f"Evaluation time: {evaluation_time}") + while time.time() < start_time + timeout: + result_from_writer = execute_query_on_prometheus_writer(query, evaluation_time) + result_from_reader = execute_query_on_prometheus_reader(query, evaluation_time) + print(f"Result from prometheus_writer: {result_from_writer}") + print(f"Result from prometheus_reader: {result_from_reader}") + if result_from_writer == result_from_reader: + return + time.sleep(1) + raise Exception( + f"Got different results from prometheus_writer and prometheus_reader" + ) + + +def compare_queries(): + compare_query("up") + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def cleanup_after_test(): + try: + yield + finally: + node.query("DROP TABLE IF EXISTS prometheus SYNC") + node.query("DROP TABLE IF EXISTS original SYNC") + node.query("DROP TABLE IF EXISTS mydata SYNC") + node.query("DROP TABLE IF EXISTS mytable SYNC") + node.query("DROP TABLE IF EXISTS mymetrics SYNC") + + +def test_default(): + node.query("CREATE TABLE prometheus ENGINE=TimeSeries") + compare_queries() + + +def test_tags_to_columns(): + node.query( + "CREATE TABLE prometheus ENGINE=TimeSeries SETTINGS tags_to_columns = {'job': 'job', 'instance': 'instance'}" + ) + compare_queries() + + +def test_64bit_id(): + node.query("CREATE TABLE prometheus (id UInt64) ENGINE=TimeSeries") + compare_queries() + + +def test_custom_id_algorithm(): + node.query( + "CREATE TABLE prometheus (id FixedString(16) DEFAULT murmurHash3_128(metric_name, all_tags)) ENGINE=TimeSeries" + ) + compare_queries() + + +def test_create_as_table(): + node.query("CREATE TABLE original ENGINE=TimeSeries") + node.query("CREATE TABLE prometheus AS original") + compare_queries() + + +def test_inner_engines(): + node.query( + "CREATE TABLE prometheus ENGINE=TimeSeries " + "DATA ENGINE=MergeTree ORDER BY (id, timestamp) " + "TAGS ENGINE=AggregatingMergeTree ORDER BY (metric_name, id) " + "METRICS ENGINE=ReplacingMergeTree ORDER BY metric_family_name" + ) + compare_queries() + + +def test_external_tables(): + node.query( + "CREATE TABLE mydata (id UUID, timestamp DateTime64(3), value Float64) " + "ENGINE=MergeTree ORDER BY (id, timestamp)" + ) + node.query( + "CREATE TABLE mytags (" + "id UUID, " + "metric_name LowCardinality(String), " + "tags Map(LowCardinality(String), String), " + "min_time SimpleAggregateFunction(min, Nullable(DateTime64(3))), " + "max_time SimpleAggregateFunction(max, Nullable(DateTime64(3)))) " + "ENGINE=AggregatingMergeTree ORDER BY (metric_name, id)" + ) + node.query( + "CREATE TABLE mymetrics (metric_family_name String, type LowCardinality(String), unit LowCardinality(String), help String) " + "ENGINE=ReplacingMergeTree ORDER BY metric_family_name" + ) + node.query( + "CREATE TABLE prometheus ENGINE=TimeSeries " + "DATA mydata TAGS mytags METRICS mymetrics" + ) + compare_queries() From e85dfa9e85e383ec7acdb4703b0e00e199fa7d96 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 31 Jul 2024 19:49:28 +0200 Subject: [PATCH 279/644] Fix exception handling in PrometheusRequestHandler. --- src/Server/HTTP/sendExceptionToHTTPClient.cpp | 1 - src/Server/PrometheusRequestHandler.cpp | 68 ++++++++++++------- src/Server/PrometheusRequestHandler.h | 7 +- 3 files changed, 51 insertions(+), 25 deletions(-) diff --git a/src/Server/HTTP/sendExceptionToHTTPClient.cpp b/src/Server/HTTP/sendExceptionToHTTPClient.cpp index 022a763a9a2..321c4866e8f 100644 --- a/src/Server/HTTP/sendExceptionToHTTPClient.cpp +++ b/src/Server/HTTP/sendExceptionToHTTPClient.cpp @@ -43,7 +43,6 @@ void sendExceptionToHTTPClient( out->position() = out->buffer().begin(); out->writeln(exception_message); - out->finalize(); } } diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index 275f6f3c04b..ddbccab05df 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -33,6 +33,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int SUPPORT_IS_DISABLED; + extern const int LOGICAL_ERROR; } /// Base implementation of a prometheus protocol. @@ -343,9 +344,10 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe try { + response_finalized = false; write_event = write_event_; http_method = request.getMethod(); - chassert(!write_buffer_from_response); + chassert(!write_buffer_from_response); /// Nothing is written to the response yet. /// Make keep-alive works. if (request.getVersion() == HTTPServerRequest::HTTP_1_1) @@ -356,28 +358,24 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe impl->beforeHandlingRequest(request); impl->handleRequest(request, response); - if (write_buffer_from_response) - { - write_buffer_from_response->finalize(); - write_buffer_from_response = nullptr; - } + finalizeResponse(response); } catch (...) { tryLogCurrentException(log); - tryCallOnException(); ExecutionStatus status = ExecutionStatus::fromCurrentException("", send_stacktrace); trySendExceptionToClient(status.message, status.code, request, response); - tryCallOnException(); + tryFinalizeResponse(response); - /// `write_buffer_from_response` must be finalized already or at least tried to finalize. - write_buffer_from_response = nullptr; + tryCallOnException(); } } -WriteBuffer & PrometheusRequestHandler::getOutputStream(HTTPServerResponse & response) +WriteBufferFromHTTPServerResponse & PrometheusRequestHandler::getOutputStream(HTTPServerResponse & response) { + if (response_finalized) + throw Exception(ErrorCodes::LOGICAL_ERROR, "PrometheusRequestHandler: Response already sent"); if (write_buffer_from_response) return *write_buffer_from_response; write_buffer_from_response = std::make_unique( @@ -385,27 +383,51 @@ WriteBuffer & PrometheusRequestHandler::getOutputStream(HTTPServerResponse & res return *write_buffer_from_response; } +void PrometheusRequestHandler::finalizeResponse(HTTPServerResponse & response) +{ + if (response_finalized) + { + /// Response is already finalized or at least tried to. We don't need the write buffer anymore in either case. + write_buffer_from_response = nullptr; + } + else + { + /// We set `response_finalized = true` before actually calling `write_buffer_from_response->finalize()` + /// because we shouldn't call finalize() again even if finalize() throws an exception. + response_finalized = true; + + if (write_buffer_from_response) + std::exchange(write_buffer_from_response, {})->finalize(); + else + WriteBufferFromHTTPServerResponse{response, http_method == HTTPRequest::HTTP_HEAD, config.keep_alive_timeout, write_event}.finalize(); + } + chassert(response_finalized && !write_buffer_from_response); +} + void PrometheusRequestHandler::trySendExceptionToClient(const String & exception_message, int exception_code, HTTPServerRequest & request, HTTPServerResponse & response) { + if (response_finalized) + return; /// Response is already finalized (or tried to). We can't write the error message to the response in either case. + try { - sendExceptionToHTTPClient(exception_message, exception_code, request, response, write_buffer_from_response.get(), log); + sendExceptionToHTTPClient(exception_message, exception_code, request, response, &getOutputStream(response), log); } catch (...) { tryLogCurrentException(log, "Couldn't send exception to client"); + } +} - if (write_buffer_from_response) - { - try - { - write_buffer_from_response->finalize(); - } - catch (...) - { - tryLogCurrentException(log, "Cannot flush data to client (after sending exception)"); - } - } +void PrometheusRequestHandler::tryFinalizeResponse(HTTPServerResponse & response) +{ + try + { + finalizeResponse(response); + } + catch (...) + { + tryLogCurrentException(log, "Cannot flush data to client (after sending exception)"); } } diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index 6df718f2a05..3a2dac0b7ad 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -26,7 +26,11 @@ private: void createImpl(); /// Returns the write buffer used for the current HTTP response. - WriteBuffer & getOutputStream(HTTPServerResponse & response); + WriteBufferFromHTTPServerResponse & getOutputStream(HTTPServerResponse & response); + + /// Finalizes the output stream and sends the response to the client. + void finalizeResponse(HTTPServerResponse & response); + void tryFinalizeResponse(HTTPServerResponse & response); /// Writes the current exception to the response. void trySendExceptionToClient(const String & exception_message, int exception_code, HTTPServerRequest & request, HTTPServerResponse & response); @@ -50,6 +54,7 @@ private: String http_method; bool send_stacktrace = false; std::unique_ptr write_buffer_from_response; + bool response_finalized = false; ProfileEvents::Event write_event; }; From 4ef382360d96fde13a787b0ebc43aefa44c822ab Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 2 Aug 2024 16:35:11 +0200 Subject: [PATCH 280/644] Fix compilation. --- src/Parsers/ASTFunction.h | 16 ---------------- .../TimeSeries/PrometheusRemoteWriteProtocol.cpp | 14 +++++++++++--- .../TimeSeriesDefinitionNormalizer.cpp | 5 +++-- .../TimeSeries/TimeSeriesInnerTablesCreator.cpp | 4 ++-- .../TimeSeries/TimeSeriesInnerTablesCreator.h | 4 ++-- 5 files changed, 18 insertions(+), 25 deletions(-) diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index 5a14d66f31c..1b4a5928d1c 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -116,20 +116,4 @@ inline bool tryGetFunctionNameInto(const ASTPtr & ast, String & name) { return t /// Checks if function is a lambda function definition `lambda((x, y), x + y)` bool isASTLambdaFunction(const ASTFunction & function); -/// Makes an ASTFunction to represent a data type. -template -std::shared_ptr makeASTDataType(const String & type_name, Args &&... args) -{ - auto function = std::make_shared(); - function->name = type_name; - function->no_empty_args = true; - if (sizeof...(args)) - { - function->arguments = std::make_shared(); - function->children.push_back(function->arguments); - function->arguments->children = { std::forward(args)... }; - } - return function; -} - } diff --git a/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp index 23d28894acc..1e8164152a5 100644 --- a/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp +++ b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -107,7 +108,7 @@ namespace ColumnsDescription{id_column_description}, context); - auto adding_missing_defaults_actions = std::make_shared(adding_missing_defaults_dag); + auto adding_missing_defaults_actions = std::make_shared(std::move(adding_missing_defaults_dag)); pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, adding_missing_defaults_actions); @@ -118,7 +119,7 @@ namespace header_with_id.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Position); auto actions = std::make_shared( - convert_actions_dag, + std::move(convert_actions_dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)); pipe.addSimpleTransform([&](const Block & stream_header) { @@ -528,7 +529,14 @@ namespace ContextMutablePtr insert_context = Context::createCopy(context); insert_context->setCurrentQueryId(context->getCurrentQueryId() + ":" + String{toString(table_kind)}); - InterpreterInsertQuery interpreter(insert_query, insert_context); + InterpreterInsertQuery interpreter( + insert_query, + insert_context, + /* allow_materialized= */ false, + /* no_squash= */ false, + /* no_destination= */ false, + /* async_insert= */ false); + BlockIO io = interpreter.execute(); PushingPipelineExecutor executor(io.pipeline); diff --git a/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp index 49dc2f3d1c1..f9e7290e514 100644 --- a/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp +++ b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -169,7 +170,7 @@ void TimeSeriesDefinitionNormalizer::addMissingColumns(ASTCreateQuery & create) auto get_string_to_string_map_type = [&] { return makeASTDataType("Map", get_string_type(), get_string_type()); }; auto get_lc_string_to_string_map_type = [&] { return makeASTDataType("Map", get_lc_string_type(), get_string_type()); }; - auto make_nullable = [&](std::shared_ptr type) + auto make_nullable = [&](std::shared_ptr type) { if (type->name == "Nullable") return type; @@ -185,7 +186,7 @@ void TimeSeriesDefinitionNormalizer::addMissingColumns(ASTCreateQuery & create) make_new_column(TimeSeriesColumnNames::Timestamp, get_datetime_type()); auto timestamp_column = typeid_cast>(columns[position - 1]); - auto timestamp_type = typeid_cast>(timestamp_column->type->ptr()); + auto timestamp_type = typeid_cast>(timestamp_column->type->ptr()); if (!is_next_column_named(TimeSeriesColumnNames::Value)) make_new_column(TimeSeriesColumnNames::Value, get_float_type()); diff --git a/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp index 130d49c4c68..5f616982a6f 100644 --- a/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp +++ b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp @@ -150,7 +150,7 @@ StorageID TimeSeriesInnerTablesCreator::getInnerTableID(ViewTarget::Kind inner_t std::shared_ptr TimeSeriesInnerTablesCreator::getInnerTableCreateQuery( ViewTarget::Kind inner_table_kind, const UUID & inner_table_uuid, - const std::shared_ptr inner_storage_def) const + const std::shared_ptr & inner_storage_def) const { auto manual_create_query = std::make_shared(); @@ -173,7 +173,7 @@ std::shared_ptr TimeSeriesInnerTablesCreator::getInnerTableCreat StorageID TimeSeriesInnerTablesCreator::createInnerTable( ViewTarget::Kind inner_table_kind, const UUID & inner_table_uuid, - const std::shared_ptr inner_storage_def) const + const std::shared_ptr & inner_storage_def) const { /// We will make a query to create the inner target table. auto create_context = Context::createCopy(getContext()); diff --git a/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.h b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.h index a59bd2107bb..5778dd77398 100644 --- a/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.h +++ b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.h @@ -31,12 +31,12 @@ public: /// Generates a CREATE TABLE query for an inner table. std::shared_ptr getInnerTableCreateQuery(ViewTarget::Kind inner_table_kind, const UUID & inner_table_uuid, - const std::shared_ptr inner_storage_def) const; + const std::shared_ptr & inner_storage_def) const; /// Creates an inner table. StorageID createInnerTable(ViewTarget::Kind inner_table_kind, const UUID & inner_table_uuid, - const std::shared_ptr inner_storage_def) const; + const std::shared_ptr & inner_storage_def) const; private: const StorageID time_series_storage_id; From 9f1753bb4f825349ac4238650bc87d7792f99cc8 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 2 Aug 2024 16:58:04 +0200 Subject: [PATCH 281/644] Corrections after review. --- .../table-engines/integrations/time-series.md | 4 +-- src/Core/Settings.h | 2 +- src/Server/PrometheusRequestHandler.cpp | 12 +++---- src/Storages/StorageTimeSeries.cpp | 34 ++++++------------- 4 files changed, 20 insertions(+), 32 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/time-series.md b/docs/en/engines/table-engines/integrations/time-series.md index 4830fd61d27..b9e47e8d2c9 100644 --- a/docs/en/engines/table-engines/integrations/time-series.md +++ b/docs/en/engines/table-engines/integrations/time-series.md @@ -213,8 +213,8 @@ CREATE TABLE my_table ## The `tags` and `all_tags` columns {#tags-and-all-tags} There are two columns containing maps of tags - `tags` and `all_tags`. In this example they mean the same, however they can be different -if setting `tags_to_columns` is used. This setting allows to specify that a specific tag should be stored in a separate column instead of storing -in a map inside the `tags` column: +if setting `tags_to_columns` is used. This setting allows you to specify that a specific tag should be stored in a separate column instead of storing +it in a map inside the `tags` column: ``` sql CREATE TABLE my_table ENGINE=TimeSeries SETTINGS = {'instance': 'instance', 'job': 'job'} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 115554d44c5..82e0e320cf1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -903,7 +903,7 @@ class IColumn; M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions", 0) \ M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ - M(Bool, allow_experimental_time_series_table, false, "Allows to use the TimeSeries table engine. Disabled by default, because this feature is experimental", 0) \ + M(Bool, allow_experimental_time_series_table, false, "Allows experimental TimeSeries table engine", 0) \ M(Bool, allow_experimental_variant_type, false, "Allow Variant data type", 0) \ M(Bool, allow_experimental_dynamic_type, false, "Allow Dynamic data type", 0) \ M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index ddbccab05df..bbd9a978c91 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -151,25 +151,25 @@ protected: if (!roles.empty()) context->setCurrentRoles(roles); - auto param_could_be_skipped = [&] (const String & name) + /// Settings can be overridden in the URL query. + auto is_setting_like_parameter = [&] (const String & name) { /// Empty parameter appears when URL like ?&a=b or a=b&&c=d. Just skip them for user's convenience. if (name.empty()) - return true; + return false; /// Some parameters (database, default_format, everything used in the code above) do not /// belong to the Settings class. static const NameSet reserved_param_names{"user", "password", "quota_key", "stacktrace", "role", "query_id"}; - return reserved_param_names.contains(name); + return !reserved_param_names.contains(name); }; - /// Settings can be overridden in the query. SettingsChanges settings_changes; for (const auto & [key, value] : *params) { - if (!param_could_be_skipped(key)) + if (is_setting_like_parameter(key)) { - /// Other than query parameters are treated as settings. + /// This query parameter should be considered as a ClickHouse setting. settings_changes.push_back({key, value}); } } diff --git a/src/Storages/StorageTimeSeries.cpp b/src/Storages/StorageTimeSeries.cpp index b1d566564ea..f5a2c0c59a2 100644 --- a/src/Storages/StorageTimeSeries.cpp +++ b/src/Storages/StorageTimeSeries.cpp @@ -371,9 +371,8 @@ void StorageTimeSeries::alter(const AlterCommands & params, ContextPtr local_con } -void StorageTimeSeries::renameInMemory(const StorageID & new_table_id) +void StorageTimeSeries::renameInMemory(const StorageID & /* new_table_id */) { - UNUSED(new_table_id); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Renaming is not supported by storage {} yet", getName()); } @@ -406,33 +405,22 @@ void StorageTimeSeries::restoreDataFromBackup(RestorerFromBackup & restorer, con void StorageTimeSeries::read( - QueryPlan & query_plan, - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr local_context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - size_t num_streams) + QueryPlan & /* query_plan */, + const Names & /* column_names */, + const StorageSnapshotPtr & /* storage_snapshot */, + SelectQueryInfo & /* query_info */, + ContextPtr /* local_context */, + QueryProcessingStage::Enum /* processed_stage */, + size_t /* max_block_size */, + size_t /* num_streams */) { - UNUSED(query_plan); - UNUSED(column_names); - UNUSED(storage_snapshot); - UNUSED(query_info); - UNUSED(local_context); - UNUSED(processed_stage); - UNUSED(max_block_size); - UNUSED(num_streams); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "SELECT is not supported by storage {} yet", getName()); } -SinkToStoragePtr StorageTimeSeries::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool async_insert) +SinkToStoragePtr StorageTimeSeries::write( + const ASTPtr & /* query */, const StorageMetadataPtr & /* metadata_snapshot */, ContextPtr /* local_context */, bool /* async_insert */) { - UNUSED(query); - UNUSED(metadata_snapshot); - UNUSED(local_context); - UNUSED(async_insert); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "INSERT is not supported by storage {} yet", getName()); } From d5629655c77653f51e44e1bfa9f8935b000da891 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sun, 4 Aug 2024 21:52:29 +0000 Subject: [PATCH 282/644] fix --- .../MergeTree/MergeTreeIndexBloomFilter.cpp | 26 ++++++++++++++++++- ..._bloom_filter_not_supported_func.reference | 2 ++ .../03215_bloom_filter_not_supported_func.sql | 14 ++++++++++ 3 files changed, 41 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03215_bloom_filter_not_supported_func.reference create mode 100644 tests/queries/0_stateless/03215_bloom_filter_not_supported_func.sql diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index c6a00751f25..7b873b0e3f2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -366,7 +366,31 @@ bool MergeTreeIndexConditionBloomFilter::extractAtomFromTree(const RPNBuilderTre } } - return traverseFunction(node, out, nullptr /*parent*/); + if (node.isFunction()) + { + /// Similar to the logic of KeyCondition, restrict the usage of bloom filter, in case of func like cast(c=1 or c=9999 as Bool). + const std::unordered_set atom_map + { + "equals", + "notEquals", + "has", + "mapContains", + "indexOf", + "hasAny", + "hasAll", + "in", + "notIn", + "globalIn", + "globalNotIn" + }; + + auto func_name = node.toFunctionNode().getFunctionName(); + if (atom_map.find(func_name) == std::end(atom_map)) + return false; + } + + bool res = traverseFunction(node, out, nullptr /*parent*/); + return res; } bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNode & node, RPNElement & out, const RPNBuilderTreeNode * parent) diff --git a/tests/queries/0_stateless/03215_bloom_filter_not_supported_func.reference b/tests/queries/0_stateless/03215_bloom_filter_not_supported_func.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/03215_bloom_filter_not_supported_func.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/03215_bloom_filter_not_supported_func.sql b/tests/queries/0_stateless/03215_bloom_filter_not_supported_func.sql new file mode 100644 index 00000000000..3d094244892 --- /dev/null +++ b/tests/queries/0_stateless/03215_bloom_filter_not_supported_func.sql @@ -0,0 +1,14 @@ +drop table if exists t; + +create table t ( + c Int32, + index x1 (c) type bloom_filter +) engine=MergeTree order by c as select 1; + +SELECT count() FROM t WHERE cast(c=1 or c=9999 as Bool) +settings use_skip_indexes=0; + +SELECT count() FROM t WHERE cast(c=1 or c=9999 as Bool) +settings use_skip_indexes=1; + +drop table t; \ No newline at end of file From f816158dbc000c4fb02ba413f5b47349795995c4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Aug 2024 09:11:55 +0000 Subject: [PATCH 283/644] Address review comments --- src/Storages/MessageQueueSink.cpp | 4 ++-- tests/integration/test_storage_kafka/test.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MessageQueueSink.cpp b/src/Storages/MessageQueueSink.cpp index d4dabd60ef8..104e3506ade 100644 --- a/src/Storages/MessageQueueSink.cpp +++ b/src/Storages/MessageQueueSink.cpp @@ -46,6 +46,8 @@ void MessageQueueSink::consume(Chunk & chunk) if (columns.empty()) return; + /// The formatter might hold pointers to buffer (e.g. if PeekableWriteBuffer is used), which means the formatter + /// needs to be reset after buffer might reallocate its memory. In this exact case after restarting the buffer. if (row_format) { size_t row = 0; @@ -77,6 +79,4 @@ void MessageQueueSink::consume(Chunk & chunk) format->resetFormatter(); } } - - } diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index dd0bf1bf28f..8793ab72a16 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -5098,7 +5098,7 @@ def test_multiple_read_in_materialized_views(kafka_cluster, max_retries=15): def test_kafka_produce_http_interface_row_based_format(kafka_cluster): - # reproduction of #https://github.com/ClickHouse/ClickHouse/issues/61060 with validating the written messages + # reproduction of #61060 with validating the written messages admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) From 1a3e9d147441f86d7a00b78873b0a07b6f292e1f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Aug 2024 11:15:37 +0000 Subject: [PATCH 284/644] Style fixes --- src/Storages/Kafka/KafkaConfigLoader.cpp | 5 +++++ src/Storages/Kafka/StorageKafkaUtils.cpp | 1 - 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Storages/Kafka/KafkaConfigLoader.cpp b/src/Storages/Kafka/KafkaConfigLoader.cpp index 3d31a987395..000e08e2276 100644 --- a/src/Storages/Kafka/KafkaConfigLoader.cpp +++ b/src/Storages/Kafka/KafkaConfigLoader.cpp @@ -19,6 +19,11 @@ extern const Metric KafkaLibrdkafkaThreads; namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + template struct KafkaInterceptors { diff --git a/src/Storages/Kafka/StorageKafkaUtils.cpp b/src/Storages/Kafka/StorageKafkaUtils.cpp index c510303f45e..a2e3683f769 100644 --- a/src/Storages/Kafka/StorageKafkaUtils.cpp +++ b/src/Storages/Kafka/StorageKafkaUtils.cpp @@ -56,7 +56,6 @@ using namespace std::chrono_literals; namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int SUPPORT_IS_DISABLED; From 048e3f56e864e1cd36b5e7df7e73482f71c8a9a9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Aug 2024 11:15:48 +0000 Subject: [PATCH 285/644] Remove redundant scope --- src/Storages/Kafka/StorageKafka2.cpp | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index f58d629dd9b..3574b46e3b0 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -652,12 +652,10 @@ void StorageKafka2::dropReplica() return; } - { - my_keeper->tryRemoveChildrenRecursive(replica_path); + my_keeper->tryRemoveChildrenRecursive(replica_path); - if (my_keeper->tryRemove(replica_path) != Coordination::Error::ZOK) - LOG_ERROR(log, "Replica was not completely removed from Keeper, {} still exists and may contain some garbage.", replica_path); - } + if (my_keeper->tryRemove(replica_path) != Coordination::Error::ZOK) + LOG_ERROR(log, "Replica was not completely removed from Keeper, {} still exists and may contain some garbage.", replica_path); /// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line. Strings replicas; From fd1e354e8503a968eff9dbe614a47c1135f67bdd Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 5 Aug 2024 13:43:06 +0200 Subject: [PATCH 286/644] fix flaky check for integration tests --- .../test_storage_azure_blob_storage/test.py | 82 +++++++++++++------ .../test_cluster.py | 18 ++-- tests/integration/test_storage_hdfs/test.py | 47 +++++++---- 3 files changed, 94 insertions(+), 53 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 6966abfee4f..15a1f6db2c1 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -135,6 +135,7 @@ def test_create_table_connection_string(cluster): Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_create_connection_string', 'CSV') """, ) + azure_query(node, "DROP TABLE IF EXISTS test_create_table_conn_string") def test_create_table_account_string(cluster): @@ -144,6 +145,7 @@ def test_create_table_account_string(cluster): f"CREATE TABLE test_create_table_account_url (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," f"'cont', 'test_create_connection_string', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", ) + azure_query(node, "DROP TABLE IF EXISTS test_create_table_account_url") def test_simple_write_account_string(cluster): @@ -157,6 +159,7 @@ def test_simple_write_account_string(cluster): azure_query(node, "INSERT INTO test_simple_write VALUES (1, 'a')") print(get_azure_file_content("test_simple_write.csv", port)) assert get_azure_file_content("test_simple_write.csv", port) == '1,"a"\n' + azure_query(node, "DROP TABLE test_simple_write") def test_simple_write_connection_string(cluster): @@ -170,6 +173,7 @@ def test_simple_write_connection_string(cluster): azure_query(node, "INSERT INTO test_simple_write_connection_string VALUES (1, 'a')") print(get_azure_file_content("test_simple_write_c.csv", port)) assert get_azure_file_content("test_simple_write_c.csv", port) == '1,"a"\n' + azure_query(node, "DROP TABLE test_simple_write_connection_string") def test_simple_write_named_collection_1(cluster): @@ -185,7 +189,7 @@ def test_simple_write_named_collection_1(cluster): ) print(get_azure_file_content("test_simple_write_named.csv", port)) assert get_azure_file_content("test_simple_write_named.csv", port) == '1,"a"\n' - azure_query(node, "TRUNCATE TABLE test_simple_write_named_collection_1") + azure_query(node, "DROP TABLE test_simple_write_named_collection_1") def test_simple_write_named_collection_2(cluster): @@ -202,6 +206,7 @@ def test_simple_write_named_collection_2(cluster): ) print(get_azure_file_content("test_simple_write_named_2.csv", port)) assert get_azure_file_content("test_simple_write_named_2.csv", port) == '1,"a"\n' + azure_query(node, "DROP TABLE test_simple_write_named_collection_2") def test_partition_by(cluster): @@ -223,6 +228,7 @@ def test_partition_by(cluster): assert "1,2,3\n" == get_azure_file_content("test_3.csv", port) assert "3,2,1\n" == get_azure_file_content("test_1.csv", port) assert "78,43,45\n" == get_azure_file_content("test_45.csv", port) + azure_query(node, "DROP TABLE test_partitioned_write") def test_partition_by_string_column(cluster): @@ -243,6 +249,7 @@ def test_partition_by_string_column(cluster): assert '1,"foo/bar"\n' == get_azure_file_content("test_foo/bar.csv", port) assert '3,"йцук"\n' == get_azure_file_content("test_йцук.csv", port) assert '78,"你好"\n' == get_azure_file_content("test_你好.csv", port) + azure_query(node, "DROP TABLE test_partitioned_string_write") def test_partition_by_const_column(cluster): @@ -261,6 +268,7 @@ def test_partition_by_const_column(cluster): ) azure_query(node, f"INSERT INTO test_partitioned_const_write VALUES {values}") assert values_csv == get_azure_file_content("test_88.csv", port) + azure_query(node, "DROP TABLE test_partitioned_const_write") def test_truncate(cluster): @@ -276,6 +284,7 @@ def test_truncate(cluster): azure_query(node, "TRUNCATE TABLE test_truncate") with pytest.raises(Exception): print(get_azure_file_content("test_truncate.csv", port)) + azure_query(node, "DROP TABLE test_truncate") def test_simple_read_write(cluster): @@ -292,6 +301,7 @@ def test_simple_read_write(cluster): assert get_azure_file_content("test_simple_read_write.csv", port) == '1,"a"\n' print(azure_query(node, "SELECT * FROM test_simple_read_write")) assert azure_query(node, "SELECT * FROM test_simple_read_write") == "1\ta\n" + azure_query(node, "DROP TABLE test_simple_read_write") def test_create_new_files_on_insert(cluster): @@ -344,6 +354,7 @@ def test_overwrite(cluster): result = azure_query(node, f"select count() from test_overwrite") assert int(result) == 200 + azure_query(node, f"DROP TABLE test_overwrite") def test_insert_with_path_with_globs(cluster): @@ -356,6 +367,7 @@ def test_insert_with_path_with_globs(cluster): node.query_and_get_error( f"insert into table function test_insert_globs SELECT number, randomString(100) FROM numbers(500)" ) + azure_query(node, f"DROP TABLE test_insert_globs") def test_put_get_with_globs(cluster): @@ -364,6 +376,7 @@ def test_put_get_with_globs(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" + used_names = [] for i in range(10): for j in range(10): path = "{}/{}_{}/{}.csv".format( @@ -372,6 +385,8 @@ def test_put_get_with_globs(cluster): max_path = max(path, max_path) values = f"({i},{j},{i + j})" + used_names.append(f"test_put_{i}_{j}") + azure_query( node, f"CREATE TABLE test_put_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, " @@ -392,6 +407,9 @@ def test_put_get_with_globs(cluster): bucket="cont", max_path=max_path ) ] + azure_query(node, "DROP TABLE test_glob_select") + for name in used_names: + azure_query(node, f"DROP TABLE {name}") def test_azure_glob_scheherazade(cluster): @@ -400,12 +418,14 @@ def test_azure_glob_scheherazade(cluster): values = "(1, 1, 1)" nights_per_job = 1001 // 30 jobs = [] + used_names = [] for night in range(0, 1001, nights_per_job): def add_tales(start, end): for i in range(start, end): path = "night_{}/tale.csv".format(i) unique_num = random.randint(1, 10000) + used_names.append(f"test_scheherazade_{i}_{unique_num}") azure_query( node, f"CREATE TABLE test_scheherazade_{i}_{unique_num} ({table_format}) Engine = AzureBlobStorage(azure_conf2, " @@ -433,6 +453,9 @@ def test_azure_glob_scheherazade(cluster): ) query = "select count(), sum(column1), sum(column2), sum(column3) from test_glob_select_scheherazade" assert azure_query(node, query).splitlines() == ["1001\t1001\t1001\t1001"] + azure_query(node, "DROP TABLE test_glob_select_scheherazade") + for name in used_names: + azure_query(node, f"DROP TABLE {name}") @pytest.mark.parametrize( @@ -506,6 +529,8 @@ def test_schema_inference_no_globs(cluster): assert azure_query(node, query).splitlines() == [ "499500\t2890\t332833500\ttest_schema_inference_no_globs.csv\tcont/test_schema_inference_no_globs.csv" ] + azure_query(node, f"DROP TABLE test_schema_inference_src") + azure_query(node, f"DROP TABLE test_select_inference") def test_schema_inference_from_globs(cluster): @@ -514,6 +539,7 @@ def test_schema_inference_from_globs(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" + used_names = [] for i in range(10): for j in range(10): path = "{}/{}_{}/{}.csv".format( @@ -521,6 +547,7 @@ def test_schema_inference_from_globs(cluster): ) max_path = max(path, max_path) values = f"({i},{j},{i + j})" + used_names.append(f"test_schema_{i}_{j}") azure_query( node, @@ -546,6 +573,9 @@ def test_schema_inference_from_globs(cluster): bucket="cont", max_path=max_path ) ] + azure_query(node, "DROP TABLE test_glob_select_inference") + for name in used_names: + azure_query(node, f"DROP TABLE {name}") def test_simple_write_account_string_table_function(cluster): @@ -595,7 +625,7 @@ def test_simple_write_named_collection_1_table_function(cluster): azure_query( node, - "TRUNCATE TABLE drop_table", + "DROP TABLE drop_table", ) @@ -605,7 +635,7 @@ def test_simple_write_named_collection_2_table_function(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')", + f" container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a') SETTINGS azure_truncate_on_insert=1", ) print(get_azure_file_content("test_simple_write_named_2_tf.csv", port)) assert get_azure_file_content("test_simple_write_named_2_tf.csv", port) == '1,"a"\n' @@ -628,7 +658,7 @@ def test_put_get_with_globs_tf(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values} SETTINGS azure_truncate_on_insert=1", ) query = ( f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, " @@ -649,7 +679,7 @@ def test_schema_inference_no_globs_tf(cluster): query = ( f"insert into table function azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " f"container='cont', blob_path='test_schema_inference_no_globs_tf.csv', format='CSVWithNames', structure='{table_format}') " - f"SELECT number, toString(number), number * number FROM numbers(1000)" + f"SELECT number, toString(number), number * number FROM numbers(1000) SETTINGS azure_truncate_on_insert=1" ) azure_query(node, query) @@ -680,7 +710,7 @@ def test_schema_inference_from_globs_tf(cluster): query = ( f"insert into table function azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " - f"container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}" + f"container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values} SETTINGS azure_truncate_on_insert=1" ) azure_query(node, query) @@ -708,7 +738,7 @@ def test_partition_by_tf(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " f"'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', " - f"'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", + f"'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values} SETTINGS azure_truncate_on_insert=1", ) assert "1,2,3\n" == get_azure_file_content("test_partition_tf_3.csv", port) @@ -727,7 +757,7 @@ def test_filter_using_file(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', 'cont', '{filename}', " f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', " - f"'{table_format}') PARTITION BY {partition_by} VALUES {values}", + f"'{table_format}') PARTITION BY {partition_by} VALUES {values} SETTINGS azure_truncate_on_insert=1", ) query = ( @@ -745,7 +775,7 @@ def test_read_subcolumns(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.tsv', " f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," - f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)", + f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3) SETTINGS azure_truncate_on_insert=1", ) azure_query( @@ -795,7 +825,7 @@ def test_read_subcolumn_time(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumn_time.tsv', " f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," - f" 'a UInt32') select (42)", + f" 'a UInt32') select (42) SETTINGS azure_truncate_on_insert=1", ) res = node.query( @@ -825,7 +855,7 @@ def test_function_signatures(cluster): account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV', 'auto', 'column1 UInt32') VALUES (1),(2),(3)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV', 'auto', 'column1 UInt32') VALUES (1),(2),(3) SETTINGS azure_truncate_on_insert=1", ) # " - connection_string, container_name, blobpath\n" @@ -939,12 +969,12 @@ def test_union_schema_inference_mode(cluster): account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference1.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'a UInt32') VALUES (1)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference1.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'a UInt32') VALUES (1) SETTINGS azure_truncate_on_insert=1", ) azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference2.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'b UInt32') VALUES (2)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference2.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'b UInt32') VALUES (2) SETTINGS azure_truncate_on_insert=1", ) node.query("system drop schema cache for azure") @@ -981,7 +1011,7 @@ def test_union_schema_inference_mode(cluster): assert result == "a\tNullable(Int64)\n" "b\tNullable(Int64)\n" azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference3.jsonl', '{account_name}', '{account_key}', 'CSV', 'auto', 's String') VALUES ('Error')", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference3.jsonl', '{account_name}', '{account_key}', 'CSV', 'auto', 's String') VALUES ('Error') SETTINGS azure_truncate_on_insert=1", ) error = azure_query( @@ -1003,7 +1033,7 @@ def test_schema_inference_cache(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.jsonl', '{account_name}', '{account_key}') " - f"select * from numbers(100)", + f"select * from numbers(100) SETTINGS azure_truncate_on_insert=1", ) time.sleep(1) @@ -1210,19 +1240,19 @@ def test_filtering_by_file_or_path(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_filter1.tsv', 'devstoreaccount1', " - f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1", + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1 SETTINGS azure_truncate_on_insert=1", ) azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_filter2.tsv', 'devstoreaccount1', " - f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 2", + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 2 SETTINGS azure_truncate_on_insert=1", ) azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_filter3.tsv', 'devstoreaccount1', " - f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 3", + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 3 SETTINGS azure_truncate_on_insert=1", ) node.query( @@ -1246,19 +1276,19 @@ def test_size_virtual_column(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_size_virtual_column1.tsv', 'devstoreaccount1', " - f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1", + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1 SETTINGS azure_truncate_on_insert=1", ) azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_size_virtual_column2.tsv', 'devstoreaccount1', " - f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 11", + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 11 SETTINGS azure_truncate_on_insert=1", ) azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_size_virtual_column3.tsv', 'devstoreaccount1', " - f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 111", + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 111 SETTINGS azure_truncate_on_insert=1", ) result = azure_query( @@ -1281,7 +1311,7 @@ def test_format_detection(cluster): account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection0', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt64, y String') select number as x, 'str_' || toString(number) from numbers(0)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection0', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt64, y String') select number as x, 'str_' || toString(number) from numbers(0) SETTINGS azure_truncate_on_insert=1", ) azure_query( @@ -1351,7 +1381,7 @@ def test_write_to_globbed_partitioned_path(cluster): account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" error = azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_data_*_{{_partition_id}}', '{account_name}', '{account_key}', 'CSV', 'auto', 'x UInt64') partition by 42 select 42", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_data_*_{{_partition_id}}', '{account_name}', '{account_key}', 'CSV', 'auto', 'x UInt64') partition by 42 select 42 SETTINGS azure_truncate_on_insert=1", expect_error="true", ) @@ -1475,7 +1505,7 @@ def test_hive_partitioning_with_one_parameter(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values} SETTINGS azure_truncate_on_insert=1", ) query = ( @@ -1512,7 +1542,7 @@ def test_hive_partitioning_with_two_parameters(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2} SETTINGS azure_truncate_on_insert=1", ) query = ( @@ -1558,7 +1588,7 @@ def test_hive_partitioning_without_setting(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2} SETTINGS azure_truncate_on_insert=1", ) query = ( diff --git a/tests/integration/test_storage_azure_blob_storage/test_cluster.py b/tests/integration/test_storage_azure_blob_storage/test_cluster.py index 6c5e2d20ca5..4d63016cf9a 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_cluster.py +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -71,7 +71,7 @@ def test_select_all(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cluster_select_all.csv', 'devstoreaccount1'," f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') " - f"VALUES (1, 'a'), (2, 'b')", + f"VALUES (1, 'a'), (2, 'b') SETTINGS azure_truncate_on_insert=1", ) print(get_azure_file_content("test_cluster_select_all.csv", port)) @@ -100,7 +100,7 @@ def test_count(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cluster_count.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', " - f"'auto', 'key UInt64') VALUES (1), (2)", + f"'auto', 'key UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", ) print(get_azure_file_content("test_cluster_count.csv", port)) @@ -128,7 +128,7 @@ def test_union_all(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_parquet_union_all', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', " - f"'auto', 'a Int32, b String') VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", + f"'auto', 'a Int32, b String') VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd') SETTINGS azure_truncate_on_insert=1", ) pure_azure = azure_query( @@ -179,7 +179,7 @@ def test_skip_unavailable_shards(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_skip_unavailable.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " - f"'auto', 'a UInt64') VALUES (1), (2)", + f"'auto', 'a UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", ) result = azure_query( node, @@ -199,7 +199,7 @@ def test_unset_skip_unavailable_shards(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_unset_skip_unavailable.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " - f"'auto', 'a UInt64') VALUES (1), (2)", + f"'auto', 'a UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", ) result = azure_query( node, @@ -217,7 +217,7 @@ def test_cluster_with_named_collection(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cluster_with_named_collection.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " - f"'auto', 'a UInt64') VALUES (1), (2)", + f"'auto', 'a UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", ) pure_azure = azure_query( @@ -248,7 +248,7 @@ def test_partition_parallel_reading_with_cluster(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', '{filename}', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') " - f"PARTITION BY {partition_by} VALUES {values}", + f"PARTITION BY {partition_by} VALUES {values} SETTINGS azure_truncate_on_insert=1", ) assert "1,2,3\n" == get_azure_file_content("test_tf_3.csv", port) @@ -272,12 +272,12 @@ def test_format_detection(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection0', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt32, y String') select number as x, 'str_' || toString(number) from numbers(10)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection0', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt32, y String') select number as x, 'str_' || toString(number) from numbers(10) SETTINGS azure_truncate_on_insert=1", ) azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt32, y String') select number as x, 'str_' || toString(number) from numbers(10, 10)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt32, y String') select number as x, 'str_' || toString(number) from numbers(10, 10) SETTINGS azure_truncate_on_insert=1", ) expected_desc_result = azure_query( diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 79914855782..ca072f59e4b 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -41,6 +41,7 @@ def test_read_write_storage(started_cluster): node1.query("insert into SimpleHDFSStorage values (1, 'Mark', 72.53)") assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" assert node1.query("select * from SimpleHDFSStorage") == "1\tMark\t72.53\n" + node1.query("drop table if exists SimpleHDFSStorage") def test_read_write_storage_with_globs(started_cluster): @@ -94,6 +95,11 @@ def test_read_write_storage_with_globs(started_cluster): print(ex) assert "in readonly mode" in str(ex) + node1.query("DROP TABLE HDFSStorageWithRange") + node1.query("DROP TABLE HDFSStorageWithEnum") + node1.query("DROP TABLE HDFSStorageWithQuestionMark") + node1.query("DROP TABLE HDFSStorageWithAsterisk") + def test_storage_with_multidirectory_glob(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -335,6 +341,7 @@ def test_virtual_columns(started_cluster): ) == expected ) + node1.query("DROP TABLE virual_cols") def test_read_files_with_spaces(started_cluster): @@ -356,6 +363,7 @@ def test_read_files_with_spaces(started_cluster): ) assert node1.query("select * from test order by id") == "1\n2\n3\n" fs.delete(dir, recursive=True) + node1.query("DROP TABLE test") def test_truncate_table(started_cluster): @@ -427,7 +435,7 @@ def test_seekable_formats(started_cluster): f"hdfs('hdfs://hdfs1:9000/parquet', 'Parquet', 'a Int32, b String')" ) node1.query( - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000)" + f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000) SETTINGS hdfs_truncate_on_insert=1" ) result = node1.query(f"SELECT count() FROM {table_function}") @@ -435,7 +443,7 @@ def test_seekable_formats(started_cluster): table_function = f"hdfs('hdfs://hdfs1:9000/orc', 'ORC', 'a Int32, b String')" node1.query( - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000)" + f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000) SETTINGS hdfs_truncate_on_insert=1" ) result = node1.query(f"SELECT count() FROM {table_function}") assert int(result) == 5000000 @@ -459,7 +467,7 @@ def test_read_table_with_default(started_cluster): def test_schema_inference(started_cluster): node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/native', 'Native', 'a Int32, b String') SELECT number, randomString(100) FROM numbers(5000000)" + f"insert into table function hdfs('hdfs://hdfs1:9000/native', 'Native', 'a Int32, b String') SELECT number, randomString(100) FROM numbers(5000000) SETTINGS hdfs_truncate_on_insert=1" ) result = node1.query(f"desc hdfs('hdfs://hdfs1:9000/native', 'Native')") @@ -512,6 +520,7 @@ def test_hdfs_directory_not_exist(started_cluster): assert "" == node1.query( "select * from HDFSStorageWithNotExistDir settings hdfs_ignore_file_doesnt_exist=1" ) + node1.query("DROP TABLE HDFSStorageWithNotExistDir") def test_overwrite(started_cluster): @@ -531,6 +540,7 @@ def test_overwrite(started_cluster): result = node1.query(f"select count() from test_overwrite") assert int(result) == 10 + node1.query(f"DROP TABLE test_overwrite") def test_multiple_inserts(started_cluster): @@ -567,6 +577,7 @@ def test_multiple_inserts(started_cluster): result = node1.query(f"select count() from test_multiple_inserts") assert int(result) == 60 + node1.query(f"DROP TABLE test_multiple_inserts") def test_format_detection(started_cluster): @@ -580,10 +591,10 @@ def test_format_detection(started_cluster): def test_schema_inference_with_globs(started_cluster): node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data1.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL" + f"insert into table function hdfs('hdfs://hdfs1:9000/data1.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL SETTINGS hdfs_truncate_on_insert=1" ) node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data2.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select 0" + f"insert into table function hdfs('hdfs://hdfs1:9000/data2.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select 0 SETTINGS hdfs_truncate_on_insert=1" ) result = node1.query( @@ -597,7 +608,7 @@ def test_schema_inference_with_globs(started_cluster): assert sorted(result.split()) == ["0", "\\N"] node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data3.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL" + f"insert into table function hdfs('hdfs://hdfs1:9000/data3.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL SETTINGS hdfs_truncate_on_insert=1" ) filename = "data{1,3}.jsoncompacteachrow" @@ -609,7 +620,7 @@ def test_schema_inference_with_globs(started_cluster): assert "All attempts to extract table structure from files failed" in result node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data0.jsoncompacteachrow', 'TSV', 'x String') select '[123;]'" + f"insert into table function hdfs('hdfs://hdfs1:9000/data0.jsoncompacteachrow', 'TSV', 'x String') select '[123;]' SETTINGS hdfs_truncate_on_insert=1" ) result = node1.query_and_get_error( @@ -621,7 +632,7 @@ def test_schema_inference_with_globs(started_cluster): def test_insert_select_schema_inference(started_cluster): node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/test.native.zst') select toUInt64(1) as x" + f"insert into table function hdfs('hdfs://hdfs1:9000/test.native.zst') select toUInt64(1) as x SETTINGS hdfs_truncate_on_insert=1" ) result = node1.query(f"desc hdfs('hdfs://hdfs1:9000/test.native.zst')") @@ -664,7 +675,7 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_2', 'Parquet', 'a Int32, b String')" ) - node1.query(f"insert into table function {table_function} SELECT 1, 'kek'") + node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "parquet_2" @@ -672,7 +683,7 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_3', 'Parquet', 'a Int32, _path String')" ) - node1.query(f"insert into table function {table_function} SELECT 1, 'kek'") + node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "kek" @@ -969,11 +980,11 @@ def test_read_subcolumns(started_cluster): node = started_cluster.instances["node1"] node.query( - f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" + f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3) SETTINGS hdfs_truncate_on_insert=1" ) node.query( - f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" + f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3) SETTINGS hdfs_truncate_on_insert=1" ) res = node.query( @@ -1019,11 +1030,11 @@ def test_union_schema_inference_mode(started_cluster): node = started_cluster.instances["node1"] node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference1.jsonl') select 1 as a" + "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference1.jsonl') select 1 as a SETTINGS hdfs_truncate_on_insert=1" ) node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference2.jsonl') select 2 as b" + "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference2.jsonl') select 2 as b SETTINGS hdfs_truncate_on_insert=1" ) node.query("system drop schema cache for hdfs") @@ -1055,7 +1066,7 @@ def test_union_schema_inference_mode(started_cluster): ) assert result == "a\tNullable(Int64)\n" "b\tNullable(Int64)\n" node.query( - f"insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference3.jsonl', TSV) select 'Error'" + f"insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference3.jsonl', TSV) select 'Error' SETTINGS hdfs_truncate_on_insert=1" ) error = node.query_and_get_error( @@ -1068,11 +1079,11 @@ def test_format_detection(started_cluster): node = started_cluster.instances["node1"] node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection0', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(0)" + "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection0', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(0) SETTINGS hdfs_truncate_on_insert=1" ) node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection1', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(10)" + "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection1', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(10) SETTINGS hdfs_truncate_on_insert=1" ) expected_desc_result = node.query( @@ -1136,7 +1147,7 @@ def test_write_to_globbed_partitioned_path(started_cluster): node = started_cluster.instances["node1"] error = node.query_and_get_error( - "insert into function hdfs('hdfs://hdfs1:9000/test_data_*_{_partition_id}.csv') partition by 42 select 42" + "insert into function hdfs('hdfs://hdfs1:9000/test_data_*_{_partition_id}.csv') partition by 42 select 42 SETTINGS hdfs_truncate_on_insert=1" ) assert "DATABASE_ACCESS_DENIED" in error From 7bb236d66ee30fe4c0080f1d7a5a358cdc2a81af Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Aug 2024 14:43:39 +0200 Subject: [PATCH 287/644] Fix style --- src/Storages/Kafka/StorageKafkaUtils.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Storages/Kafka/StorageKafkaUtils.cpp b/src/Storages/Kafka/StorageKafkaUtils.cpp index a2e3683f769..cdc32d775eb 100644 --- a/src/Storages/Kafka/StorageKafkaUtils.cpp +++ b/src/Storages/Kafka/StorageKafkaUtils.cpp @@ -39,11 +39,6 @@ # include #endif // USE_KRB5 -namespace CurrentMetrics -{ -extern const Metric KafkaLibrdkafkaThreads; -} - namespace ProfileEvents { extern const Event KafkaConsumerErrors; From fd562086ae989a6955a3355f4298465d91b3934d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Aug 2024 15:37:23 +0200 Subject: [PATCH 288/644] Fix bad merge --- src/Core/SettingsChangesHistory.cpp | 262 +--------------------------- 1 file changed, 1 insertion(+), 261 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index a538151798d..e689382c5c4 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,267 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.8", {{"allow_experimental_kafka_offsets_storage_in_keeper", false, false, "Allow the usage of experimental Kafka storage engine that stores the committed offsets in ClickHouse Keeper"}, - }}, - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, - {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, - {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, - {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, - {"input_format_json_case_insensitive_column_matching", false, false, "Ignore case when matching JSON keys with CH columns."}, - {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, - {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, - {"collect_hash_table_stats_during_joins", false, true, "New setting."}, - {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, - {"input_format_orc_reader_time_zone_name", "GMT", "GMT", "The time zone name for ORC row reader, the default ORC row reader's time zone is GMT."}, - {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, - {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, - {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, - {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, - {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, - {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, - {"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"}, - {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, - {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, - {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, - {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."} - }}, - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, - {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, - {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, - {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, - {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, - {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, - {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, - {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, - {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, - {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, - {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, - {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, - {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, - {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"http_max_chunk_size", 0, 0, "Internal limitation"}, - {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, - {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, - {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - }}, - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, - {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, - {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, - {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, - {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, - {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, - {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, - {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, - {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, - {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, - {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, - {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, - {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, - {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, - {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, - {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, - {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, - {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, - {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication in dependent materialized view cannot work together with async inserts."}, - {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"log_processors_profiles", false, true, "Enable by default"}, - {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, - {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, - {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, - {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, - {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, - {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, - {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, - {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, - {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, - {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, - {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, - {"allow_get_client_http_header", false, false, "Introduced a new function."}, - {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, - {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, - {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, - {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, - {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, - {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, - {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, - {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, - {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, - {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, - {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, - {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, - {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, - {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, - {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, - {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, - {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, - {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, - {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, - {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, - {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, - {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, - {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, {"24.12", { } @@ -338,6 +77,7 @@ static std::initializer_list Date: Fri, 2 Aug 2024 18:23:56 +0200 Subject: [PATCH 289/644] repro for #67478 --- .../test_storage_policies/configs/disks.xml | 8 ++ .../integration/test_storage_policies/test.py | 79 +++++++++++++++++++ 2 files changed, 87 insertions(+) diff --git a/tests/integration/test_storage_policies/configs/disks.xml b/tests/integration/test_storage_policies/configs/disks.xml index 3331fee4e4f..dc60d93208c 100644 --- a/tests/integration/test_storage_policies/configs/disks.xml +++ b/tests/integration/test_storage_policies/configs/disks.xml @@ -1,4 +1,12 @@ + + system + blob_storage_log
+ toYYYYMM(event_date) + 7500 + event_date + INTERVAL 30 DAY +
+ diff --git a/tests/integration/test_storage_policies/test.py b/tests/integration/test_storage_policies/test.py index 389146b2171..08a5cc71b99 100644 --- a/tests/integration/test_storage_policies/test.py +++ b/tests/integration/test_storage_policies/test.py @@ -38,3 +38,82 @@ def test_storage_policy_configuration_change(started_cluster): "/etc/clickhouse-server/config.d/disks.xml", ) node.start_clickhouse() + + +def test_disk_is_immutable(started_cluster): + node.query("DROP TABLE IF EXISTS test_1") + + node.query( + """ + create table test_1 (a Int32) + engine = MergeTree() + order by tuple() + settings + disk=disk( + name='not_uniq_disk_name', + type = object_storage, + object_storage_type = local_blob_storage, + path='./03215_data_test_1/') + """ + ) + + node.query("INSERT INTO test_1 VALUES (1)") + node.query("SYSTEM FLUSH LOGS;") + + print( + node.query( + "SELECT 'test_1', * FROM system.blob_storage_log" + ) + ) + + print( + node.query( + "SELECT 'test_1', * FROM test_1" + ) + ) + + node.query("DROP TABLE test_1 SYNC") + node.query("DROP TABLE IF EXISTS test_2") + + node.query( + """ + create table test_2 (a Int32) + engine = MergeTree() + order by tuple() + settings + disk=disk( + name='not_uniq_disk_name', + type = object_storage, + object_storage_type = local_blob_storage, + path='./03215_data_test_2/') + """ + ) + + node.query("INSERT INTO test_2 VALUES (1)") + node.query("SYSTEM FLUSH LOGS;") + + print( + node.query( + "SELECT 'test_2', * FROM system.blob_storage_log" + ) + ) + + print( + node.query( + "SELECT 'test_2', * FROM test_2" + ) + ) + + node.restart_clickhouse() + + print( + node.query( + "SELECT 'test_2', * FROM system.blob_storage_log" + ) + ) + + print( + node.query( + "SELECT 'test_2', * FROM test_2" + ) + ) From 5d9d5bf919527822b4a3d42e4f1595ccd374ead9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Aug 2024 18:14:22 +0000 Subject: [PATCH 290/644] Automatic style fix --- .../integration/test_storage_policies/test.py | 36 ++++--------------- 1 file changed, 6 insertions(+), 30 deletions(-) diff --git a/tests/integration/test_storage_policies/test.py b/tests/integration/test_storage_policies/test.py index 08a5cc71b99..f65096389af 100644 --- a/tests/integration/test_storage_policies/test.py +++ b/tests/integration/test_storage_policies/test.py @@ -60,17 +60,9 @@ def test_disk_is_immutable(started_cluster): node.query("INSERT INTO test_1 VALUES (1)") node.query("SYSTEM FLUSH LOGS;") - print( - node.query( - "SELECT 'test_1', * FROM system.blob_storage_log" - ) - ) + print(node.query("SELECT 'test_1', * FROM system.blob_storage_log")) - print( - node.query( - "SELECT 'test_1', * FROM test_1" - ) - ) + print(node.query("SELECT 'test_1', * FROM test_1")) node.query("DROP TABLE test_1 SYNC") node.query("DROP TABLE IF EXISTS test_2") @@ -92,28 +84,12 @@ def test_disk_is_immutable(started_cluster): node.query("INSERT INTO test_2 VALUES (1)") node.query("SYSTEM FLUSH LOGS;") - print( - node.query( - "SELECT 'test_2', * FROM system.blob_storage_log" - ) - ) + print(node.query("SELECT 'test_2', * FROM system.blob_storage_log")) - print( - node.query( - "SELECT 'test_2', * FROM test_2" - ) - ) + print(node.query("SELECT 'test_2', * FROM test_2")) node.restart_clickhouse() - print( - node.query( - "SELECT 'test_2', * FROM system.blob_storage_log" - ) - ) + print(node.query("SELECT 'test_2', * FROM system.blob_storage_log")) - print( - node.query( - "SELECT 'test_2', * FROM test_2" - ) - ) + print(node.query("SELECT 'test_2', * FROM test_2")) From d080f863ea41420ecbb1c5d65769d74e21a46aba Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 5 Aug 2024 17:07:17 +0200 Subject: [PATCH 291/644] fix black --- tests/integration/test_storage_hdfs/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 9ee8ac4cdfd..856715f28c8 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -675,7 +675,9 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_2', 'Parquet', 'a Int32, b String')" ) - node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") + node1.query( + f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" + ) result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "parquet_2" @@ -683,7 +685,9 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_3', 'Parquet', 'a Int32, _path String')" ) - node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") + node1.query( + f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" + ) result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "kek" From d8aa219783b6715f5424772bf25c092a26be5e2d Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 5 Aug 2024 17:43:29 +0200 Subject: [PATCH 292/644] fix build --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 03e0f4f0dc8..2080e8fbf0d 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -77,7 +77,7 @@ static std::initializer_list Date: Mon, 5 Aug 2024 18:31:16 +0000 Subject: [PATCH 293/644] set max_threads --- .../queries/0_stateless/03161_lightweight_delete_projection.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 28e5612a529..618f3ac0cb8 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -1,5 +1,5 @@ -SET lightweight_deletes_sync = 2, alter_sync = 2; +SET max_threads = 1, lightweight_deletes_sync = 2, alter_sync = 2; DROP TABLE IF EXISTS users_compact; From cb6baefa948ef1270ce9454f72075ac10bf6e729 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 5 Aug 2024 21:49:36 +0100 Subject: [PATCH 294/644] better --- src/Common/ShellCommand.cpp | 2 +- src/Processors/Sources/ShellCommandSource.cpp | 12 +++--------- tests/integration/test_executable_dictionary/test.py | 2 -- .../test_executable_table_function/test.py | 1 - 4 files changed, 4 insertions(+), 13 deletions(-) diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index 79b0d667863..0d41669816c 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -239,7 +239,7 @@ std::unique_ptr ShellCommand::executeImpl( LOG_TRACE( getLogger(), - "Started shell command '{}' with pid {} and file descriptors: read {}, write {}", + "Started shell command '{}' with pid {} and file descriptors: out {}, err {}", filename, pid, res->out.getFD(), diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 923bdfad8f8..23359367a9b 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -76,13 +76,11 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond { Stopwatch watch; -#if defined(DEBUG_OR_SANITIZER_BUILD) auto describe_fd = [](const auto & pollfd) { return fmt::format("(fd={}, flags={})", pollfd.fd, fcntl(pollfd.fd, F_GETFL)); }; - LOG_TRACE( + LOG_TEST( getLogger("TimeoutReadBufferFromFileDescriptor"), "Polling descriptors: {}", fmt::join(std::span(pfds, pfds + num) | std::views::transform(describe_fd), ", ")); -#endif res = poll(pfds, static_cast(num), static_cast(timeout_milliseconds)); @@ -94,13 +92,11 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond const auto elapsed = watch.elapsedMilliseconds(); if (timeout_milliseconds <= elapsed) { -#if defined(DEBUG_OR_SANITIZER_BUILD) - LOG_TRACE( + LOG_TEST( getLogger("TimeoutReadBufferFromFileDescriptor"), "Timeout exceeded: elapsed={}, timeout={}", elapsed, timeout_milliseconds); -#endif break; } timeout_milliseconds -= elapsed; @@ -111,14 +107,12 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond } } -#if defined(DEBUG_OR_SANITIZER_BUILD) auto describe_fd = [](const auto & pollfd) { return fmt::format("(fd={}, flags={})", pollfd.fd, fcntl(pollfd.fd, F_GETFL)); }; - LOG_TRACE( + LOG_TEST( getLogger("TimeoutReadBufferFromFileDescriptor"), "Poll for descriptors: {} returned {}", fmt::join(std::span(pfds, pfds + num) | std::views::transform(describe_fd), ", "), res); -#endif return res; } diff --git a/tests/integration/test_executable_dictionary/test.py b/tests/integration/test_executable_dictionary/test.py index 2a6af75e751..22f3442bb95 100644 --- a/tests/integration/test_executable_dictionary/test.py +++ b/tests/integration/test_executable_dictionary/test.py @@ -235,7 +235,6 @@ def test_executable_implicit_input_signalled_python(started_cluster): ) -@pytest.mark.repeat(50) def test_executable_input_slow_python(started_cluster): skip_test_msan(node) assert node.query_and_get_error( @@ -246,7 +245,6 @@ def test_executable_input_slow_python(started_cluster): ) -@pytest.mark.repeat(50) def test_executable_implicit_input_slow_python(started_cluster): skip_test_msan(node) assert node.query_and_get_error( diff --git a/tests/integration/test_executable_table_function/test.py b/tests/integration/test_executable_table_function/test.py index a79616fc008..801a3c7c14a 100644 --- a/tests/integration/test_executable_table_function/test.py +++ b/tests/integration/test_executable_table_function/test.py @@ -139,7 +139,6 @@ def test_executable_function_input_signalled_python(started_cluster): assert node.query(query.format(source="(SELECT id FROM test_data_table)")) == "" -@pytest.mark.repeat(50) def test_executable_function_input_slow_python(started_cluster): skip_test_msan(node) From e6f566e49d78080a954ca992d8d5e0f5fb1bb1e2 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 6 Aug 2024 13:23:12 +0800 Subject: [PATCH 295/644] Small refactors in ORC output format --- .../Formats/Impl/ORCBlockOutputFormat.cpp | 99 +++++++------------ 1 file changed, 33 insertions(+), 66 deletions(-) diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 6f543a05fba..bd89ae0fa86 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include @@ -203,25 +204,15 @@ template void ORCBlockOutputFormat::writeNumbers( orc::ColumnVectorBatch & orc_column, const IColumn & column, - const PaddedPODArray * null_bytemap, + const PaddedPODArray * /*null_bytemap*/, ConvertFunc convert) { NumberVectorBatch & number_orc_column = dynamic_cast(orc_column); const auto & number_column = assert_cast &>(column); - number_orc_column.resize(number_column.size()); + number_orc_column.data.resize(number_column.size()); for (size_t i = 0; i != number_column.size(); ++i) - { - if (null_bytemap && (*null_bytemap)[i]) - { - number_orc_column.notNull[i] = 0; - continue; - } - - number_orc_column.notNull[i] = 1; number_orc_column.data[i] = convert(number_column.getElement(i)); - } - number_orc_column.numElements = number_column.size(); } template @@ -229,7 +220,7 @@ void ORCBlockOutputFormat::writeDecimals( orc::ColumnVectorBatch & orc_column, const IColumn & column, DataTypePtr & type, - const PaddedPODArray * null_bytemap, + const PaddedPODArray * /*null_bytemap*/, ConvertFunc convert) { DecimalVectorBatch & decimal_orc_column = dynamic_cast(orc_column); @@ -238,71 +229,49 @@ void ORCBlockOutputFormat::writeDecimals( decimal_orc_column.precision = decimal_type->getPrecision(); decimal_orc_column.scale = decimal_type->getScale(); decimal_orc_column.resize(decimal_column.size()); - for (size_t i = 0; i != decimal_column.size(); ++i) - { - if (null_bytemap && (*null_bytemap)[i]) - { - decimal_orc_column.notNull[i] = 0; - continue; - } - decimal_orc_column.notNull[i] = 1; + decimal_orc_column.values.resize(decimal_column.size()); + for (size_t i = 0; i != decimal_column.size(); ++i) decimal_orc_column.values[i] = convert(decimal_column.getElement(i).value); - } - decimal_orc_column.numElements = decimal_column.size(); } template void ORCBlockOutputFormat::writeStrings( orc::ColumnVectorBatch & orc_column, const IColumn & column, - const PaddedPODArray * null_bytemap) + const PaddedPODArray * /*null_bytemap*/) { orc::StringVectorBatch & string_orc_column = dynamic_cast(orc_column); const auto & string_column = assert_cast(column); - string_orc_column.resize(string_column.size()); + string_orc_column.data.resize(string_column.size()); + string_orc_column.length.resize(string_column.size()); for (size_t i = 0; i != string_column.size(); ++i) { - if (null_bytemap && (*null_bytemap)[i]) - { - string_orc_column.notNull[i] = 0; - continue; - } - - string_orc_column.notNull[i] = 1; const std::string_view & string = string_column.getDataAt(i).toView(); string_orc_column.data[i] = const_cast(string.data()); string_orc_column.length[i] = string.size(); } - string_orc_column.numElements = string_column.size(); } template void ORCBlockOutputFormat::writeDateTimes( orc::ColumnVectorBatch & orc_column, const IColumn & column, - const PaddedPODArray * null_bytemap, + const PaddedPODArray * /*null_bytemap*/, GetSecondsFunc get_seconds, GetNanosecondsFunc get_nanoseconds) { orc::TimestampVectorBatch & timestamp_orc_column = dynamic_cast(orc_column); const auto & timestamp_column = assert_cast(column); - timestamp_orc_column.resize(timestamp_column.size()); + timestamp_orc_column.data.resize(timestamp_column.size()); + timestamp_orc_column.nanoseconds.resize(timestamp_column.size()); for (size_t i = 0; i != timestamp_column.size(); ++i) { - if (null_bytemap && (*null_bytemap)[i]) - { - timestamp_orc_column.notNull[i] = 0; - continue; - } - - timestamp_orc_column.notNull[i] = 1; timestamp_orc_column.data[i] = static_cast(get_seconds(timestamp_column.getElement(i))); timestamp_orc_column.nanoseconds[i] = static_cast(get_nanoseconds(timestamp_column.getElement(i))); } - timestamp_orc_column.numElements = timestamp_column.size(); } void ORCBlockOutputFormat::writeColumn( @@ -311,9 +280,19 @@ void ORCBlockOutputFormat::writeColumn( DataTypePtr & type, const PaddedPODArray * null_bytemap) { - orc_column.notNull.resize(column.size()); + orc_column.numElements = column.size(); if (null_bytemap) - orc_column.hasNulls = true; + { + orc_column.hasNulls = !memoryIsZero(null_bytemap->data(), 0, null_bytemap->size()); + if (orc_column.hasNulls) + { + orc_column.notNull.resize(null_bytemap->size()); + for (size_t i = 0; i < null_bytemap->size(); ++i) + orc_column.notNull[i] = !(*null_bytemap)[i]; + } + } + else + orc_column.hasNulls = false; /// ORC doesn't have unsigned types, so cast everything to signed and sign-extend to Int64 to /// make the ORC library calculate min and max correctly. @@ -471,6 +450,7 @@ void ORCBlockOutputFormat::writeColumn( } case TypeIndex::Nullable: { + chassert(!null_bytemap); const auto & nullable_column = assert_cast(column); const PaddedPODArray & new_null_bytemap = assert_cast &>(*nullable_column.getNullMapColumnPtr()).getData(); auto nested_type = removeNullable(type); @@ -485,19 +465,15 @@ void ORCBlockOutputFormat::writeColumn( const ColumnArray::Offsets & offsets = list_column.getOffsets(); size_t column_size = list_column.size(); - list_orc_column.resize(column_size); + list_orc_column.offsets.resize(column_size + 1); /// The length of list i in ListVectorBatch is offsets[i+1] - offsets[i]. list_orc_column.offsets[0] = 0; for (size_t i = 0; i != column_size; ++i) - { list_orc_column.offsets[i + 1] = offsets[i]; - list_orc_column.notNull[i] = 1; - } orc::ColumnVectorBatch & nested_orc_column = *list_orc_column.elements; - writeColumn(nested_orc_column, list_column.getData(), nested_type, null_bytemap); - list_orc_column.numElements = column_size; + writeColumn(nested_orc_column, list_column.getData(), nested_type, nullptr); break; } case TypeIndex::Tuple: @@ -505,10 +481,8 @@ void ORCBlockOutputFormat::writeColumn( orc::StructVectorBatch & struct_orc_column = dynamic_cast(orc_column); const auto & tuple_column = assert_cast(column); auto nested_types = assert_cast(type.get())->getElements(); - for (size_t i = 0; i != tuple_column.size(); ++i) - struct_orc_column.notNull[i] = 1; for (size_t i = 0; i != tuple_column.tupleSize(); ++i) - writeColumn(*struct_orc_column.fields[i], tuple_column.getColumn(i), nested_types[i], null_bytemap); + writeColumn(*struct_orc_column.fields[i], tuple_column.getColumn(i), nested_types[i], nullptr); break; } case TypeIndex::Map: @@ -520,25 +494,21 @@ void ORCBlockOutputFormat::writeColumn( size_t column_size = list_column.size(); - map_orc_column.resize(list_column.size()); + map_orc_column.offsets.resize(column_size + 1); /// The length of list i in ListVectorBatch is offsets[i+1] - offsets[i]. map_orc_column.offsets[0] = 0; for (size_t i = 0; i != column_size; ++i) - { map_orc_column.offsets[i + 1] = offsets[i]; - map_orc_column.notNull[i] = 1; - } + const auto nested_columns = assert_cast(list_column.getDataPtr().get())->getColumns(); orc::ColumnVectorBatch & keys_orc_column = *map_orc_column.keys; auto key_type = map_type.getKeyType(); - writeColumn(keys_orc_column, *nested_columns[0], key_type, null_bytemap); + writeColumn(keys_orc_column, *nested_columns[0], key_type, nullptr); orc::ColumnVectorBatch & values_orc_column = *map_orc_column.elements; auto value_type = map_type.getValueType(); - writeColumn(values_orc_column, *nested_columns[1], value_type, null_bytemap); - - map_orc_column.numElements = column_size; + writeColumn(values_orc_column, *nested_columns[1], value_type, nullptr); break; } default: @@ -575,10 +545,7 @@ void ORCBlockOutputFormat::consume(Chunk chunk) size_t columns_num = chunk.getNumColumns(); size_t rows_num = chunk.getNumRows(); - /// getMaxColumnSize is needed to write arrays. - /// The size of the batch must be no less than total amount of array elements - /// and no less than the number of rows (ORC writes a null bit for every row). - std::unique_ptr batch = writer->createRowBatch(getMaxColumnSize(chunk)); + std::unique_ptr batch = writer->createRowBatch(chunk.getNumRows()); orc::StructVectorBatch & root = dynamic_cast(*batch); auto columns = chunk.detachColumns(); From 11fd263be6e24ee4cdc3a51ac497510c82837fa5 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 1 Aug 2024 08:35:05 +0000 Subject: [PATCH 296/644] implement DROP DETACHED PARTITION ALL Signed-off-by: Duc Canh Le --- src/Storages/MergeTree/MergeTreeData.cpp | 7 +++++-- .../03203_drop_detached_partition_all.reference | 5 +++++ .../0_stateless/03203_drop_detached_partition_all.sql | 8 ++++++++ 3 files changed, 18 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03203_drop_detached_partition_all.reference create mode 100644 tests/queries/0_stateless/03203_drop_detached_partition_all.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 30a4a7caa0f..3d3ae2e63ea 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6253,10 +6253,13 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, ContextPtr } else { - String partition_id = getPartitionIDFromQuery(partition, local_context); + String partition_id; + bool all = partition->as()->all; + if (!all) + partition_id = getPartitionIDFromQuery(partition, local_context); DetachedPartsInfo detached_parts = getDetachedParts(); for (const auto & part_info : detached_parts) - if (part_info.valid_name && part_info.partition_id == partition_id + if (part_info.valid_name && (all || part_info.partition_id == partition_id) && part_info.prefix != "attaching" && part_info.prefix != "deleting") renamed_parts.addPart(part_info.dir_name, "deleting_" + part_info.dir_name, part_info.disk); } diff --git a/tests/queries/0_stateless/03203_drop_detached_partition_all.reference b/tests/queries/0_stateless/03203_drop_detached_partition_all.reference new file mode 100644 index 00000000000..c0f52d1d898 --- /dev/null +++ b/tests/queries/0_stateless/03203_drop_detached_partition_all.reference @@ -0,0 +1,5 @@ +1 1 +2 2 +3 3 +3 +0 diff --git a/tests/queries/0_stateless/03203_drop_detached_partition_all.sql b/tests/queries/0_stateless/03203_drop_detached_partition_all.sql new file mode 100644 index 00000000000..e29eb4ae36b --- /dev/null +++ b/tests/queries/0_stateless/03203_drop_detached_partition_all.sql @@ -0,0 +1,8 @@ +DROP TABLE IF EXISTS t_03203; +CREATE TABLE t_03203 (p UInt64, v UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY v; +INSERT INTO t_03203 VALUES (1, 1), (2, 2), (3, 3); +SELECT * FROM t_03203 ORDER BY p, v; +ALTER TABLE t_03203 DETACH PARTITION ALL; +SELECT count() FROM system.detached_parts WHERE database = currentDatabase() AND table = 't_03203'; +ALTER TABLE t_03203 DROP DETACHED PARTITION ALL SETTINGS allow_drop_detached = 1; +SELECT count() FROM system.detached_parts WHERE database = currentDatabase() AND table = 't_03203'; From c6c2fce9d22739b1881d5b4814dbbabe8ab8f09b Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 6 Aug 2024 07:51:35 +0000 Subject: [PATCH 297/644] update document Signed-off-by: Duc Canh Le --- docs/en/sql-reference/statements/alter/partition.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 778816f8934..1bb7817364a 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -9,6 +9,7 @@ The following operations with [partitions](/docs/en/engines/table-engines/merget - [DETACH PARTITION\|PART](#detach-partitionpart) — Moves a partition or part to the `detached` directory and forget it. - [DROP PARTITION\|PART](#drop-partitionpart) — Deletes a partition or part. +- [DROP DETACHED PARTITION\|PART](#drop-detached-partitionpart) - Delete a part or all parts of a partition from `detached`. - [FORGET PARTITION](#forget-partition) — Deletes a partition metadata from zookeeper if it's empty. - [ATTACH PARTITION\|PART](#attach-partitionpart) — Adds a partition or part from the `detached` directory to the table. - [ATTACH PARTITION FROM](#attach-partition-from) — Copies the data partition from one table to another and adds. @@ -68,7 +69,7 @@ ALTER TABLE mt DROP PART 'all_4_4_0'; ## DROP DETACHED PARTITION\|PART ``` sql -ALTER TABLE table_name [ON CLUSTER cluster] DROP DETACHED PARTITION|PART partition_expr +ALTER TABLE table_name [ON CLUSTER cluster] DROP DETACHED PARTITION|PART ALL|partition_expr ``` Removes the specified part or all parts of the specified partition from `detached`. From aa3d8086c32ce2b5a90fbe4788579cae970ec32f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 6 Aug 2024 12:30:39 +0200 Subject: [PATCH 298/644] fix integration tests --- .../test_storage_azure_blob_storage/test.py | 37 ++++++++++++------- .../test_cluster.py | 21 +++++++---- tests/integration/test_storage_hdfs/test.py | 10 ++--- 3 files changed, 41 insertions(+), 27 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 15a1f6db2c1..092c124855c 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -635,7 +635,8 @@ def test_simple_write_named_collection_2_table_function(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a') SETTINGS azure_truncate_on_insert=1", + f" container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')", + settings={"azure_truncate_on_insert": 1}, ) print(get_azure_file_content("test_simple_write_named_2_tf.csv", port)) assert get_azure_file_content("test_simple_write_named_2_tf.csv", port) == '1,"a"\n' @@ -658,7 +659,8 @@ def test_put_get_with_globs_tf(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values} SETTINGS azure_truncate_on_insert=1", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + settings={"azure_truncate_on_insert": 1}, ) query = ( f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, " @@ -710,9 +712,9 @@ def test_schema_inference_from_globs_tf(cluster): query = ( f"insert into table function azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " - f"container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values} SETTINGS azure_truncate_on_insert=1" + f"container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}" ) - azure_query(node, query) + azure_query(node, query, settings={"azure_truncate_on_insert": 1}) query = ( f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, " @@ -738,7 +740,8 @@ def test_partition_by_tf(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " f"'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', " - f"'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values} SETTINGS azure_truncate_on_insert=1", + f"'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", + settings={"azure_truncate_on_insert": 1}, ) assert "1,2,3\n" == get_azure_file_content("test_partition_tf_3.csv", port) @@ -757,7 +760,8 @@ def test_filter_using_file(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', 'cont', '{filename}', " f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', " - f"'{table_format}') PARTITION BY {partition_by} VALUES {values} SETTINGS azure_truncate_on_insert=1", + f"'{table_format}') PARTITION BY {partition_by} VALUES {values}", + settings={"azure_truncate_on_insert": 1}, ) query = ( @@ -855,7 +859,8 @@ def test_function_signatures(cluster): account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV', 'auto', 'column1 UInt32') VALUES (1),(2),(3) SETTINGS azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV', 'auto', 'column1 UInt32') VALUES (1),(2),(3)", + settings={"azure_truncate_on_insert": 1}, ) # " - connection_string, container_name, blobpath\n" @@ -969,12 +974,14 @@ def test_union_schema_inference_mode(cluster): account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference1.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'a UInt32') VALUES (1) SETTINGS azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference1.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'a UInt32') VALUES (1)", + settings={"azure_truncate_on_insert": 1}, ) azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference2.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'b UInt32') VALUES (2) SETTINGS azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference2.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'b UInt32') VALUES (2)", + settings={"azure_truncate_on_insert": 1}, ) node.query("system drop schema cache for azure") @@ -1011,7 +1018,8 @@ def test_union_schema_inference_mode(cluster): assert result == "a\tNullable(Int64)\n" "b\tNullable(Int64)\n" azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference3.jsonl', '{account_name}', '{account_key}', 'CSV', 'auto', 's String') VALUES ('Error') SETTINGS azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference3.jsonl', '{account_name}', '{account_key}', 'CSV', 'auto', 's String') VALUES ('Error')", + settings={"azure_truncate_on_insert": 1}, ) error = azure_query( @@ -1505,7 +1513,8 @@ def test_hive_partitioning_with_one_parameter(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values} SETTINGS azure_truncate_on_insert=1", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + settings={"azure_truncate_on_insert": 1}, ) query = ( @@ -1542,7 +1551,8 @@ def test_hive_partitioning_with_two_parameters(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2} SETTINGS azure_truncate_on_insert=1", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + settings={"azure_truncate_on_insert": 1}, ) query = ( @@ -1588,7 +1598,8 @@ def test_hive_partitioning_without_setting(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2} SETTINGS azure_truncate_on_insert=1", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + settings={"azure_truncate_on_insert": 1}, ) query = ( diff --git a/tests/integration/test_storage_azure_blob_storage/test_cluster.py b/tests/integration/test_storage_azure_blob_storage/test_cluster.py index 4d63016cf9a..04baf007c69 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_cluster.py +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -71,7 +71,8 @@ def test_select_all(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cluster_select_all.csv', 'devstoreaccount1'," f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') " - f"VALUES (1, 'a'), (2, 'b') SETTINGS azure_truncate_on_insert=1", + f"VALUES (1, 'a'), (2, 'b')", + settings={"azure_truncate_on_insert": 1}, ) print(get_azure_file_content("test_cluster_select_all.csv", port)) @@ -100,7 +101,8 @@ def test_count(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cluster_count.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', " - f"'auto', 'key UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", + f"'auto', 'key UInt64') VALUES (1), (2)", + settings={"azure_truncate_on_insert": 1}, ) print(get_azure_file_content("test_cluster_count.csv", port)) @@ -128,7 +130,8 @@ def test_union_all(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_parquet_union_all', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', " - f"'auto', 'a Int32, b String') VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd') SETTINGS azure_truncate_on_insert=1", + f"'auto', 'a Int32, b String') VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", + settings={"azure_truncate_on_insert": 1}, ) pure_azure = azure_query( @@ -179,7 +182,8 @@ def test_skip_unavailable_shards(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_skip_unavailable.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " - f"'auto', 'a UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", + f"'auto', 'a UInt64') VALUES (1), (2)", + settings={"azure_truncate_on_insert": 1}, ) result = azure_query( node, @@ -199,7 +203,8 @@ def test_unset_skip_unavailable_shards(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_unset_skip_unavailable.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " - f"'auto', 'a UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", + f"'auto', 'a UInt64') VALUES (1), (2)", + settings={"azure_truncate_on_insert": 1}, ) result = azure_query( node, @@ -217,7 +222,8 @@ def test_cluster_with_named_collection(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cluster_with_named_collection.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " - f"'auto', 'a UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", + f"'auto', 'a UInt64') VALUES (1), (2)", + settings={"azure_truncate_on_insert": 1}, ) pure_azure = azure_query( @@ -248,7 +254,8 @@ def test_partition_parallel_reading_with_cluster(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', '{filename}', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') " - f"PARTITION BY {partition_by} VALUES {values} SETTINGS azure_truncate_on_insert=1", + f"PARTITION BY {partition_by} VALUES {values}", + settings={"azure_truncate_on_insert": 1}, ) assert "1,2,3\n" == get_azure_file_content("test_tf_3.csv", port) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 856715f28c8..3fef6bc46cf 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -341,7 +341,7 @@ def test_virtual_columns(started_cluster): ) == expected ) - node1.query("DROP TABLE virual_cols") + node1.query("DROP TABLE virtual_cols") def test_read_files_with_spaces(started_cluster): @@ -675,9 +675,7 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_2', 'Parquet', 'a Int32, b String')" ) - node1.query( - f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" - ) + node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "parquet_2" @@ -685,9 +683,7 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_3', 'Parquet', 'a Int32, _path String')" ) - node1.query( - f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" - ) + node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "kek" From 995187006a8c2500ddb7fa234f3443c75d900be4 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 5 Aug 2024 20:23:41 +0200 Subject: [PATCH 299/644] rework custom table's disk usage --- src/Disks/DiskFomAST.cpp | 191 ++++++++++++++++++ src/Disks/DiskFomAST.h | 15 ++ src/Disks/DiskSelector.h | 4 +- src/Disks/IDisk.h | 9 +- src/Disks/StoragePolicy.h | 2 + src/Disks/getOrCreateDiskFromAST.cpp | 121 ----------- src/Disks/getOrCreateDiskFromAST.h | 18 -- src/Interpreters/Context.cpp | 14 +- src/Interpreters/Context.h | 1 + src/Parsers/FieldFromAST.cpp | 1 - src/Storages/MergeTree/MergeTreeSettings.cpp | 10 +- .../test_storage_policies/configs/disks.xml | 8 - .../integration/test_storage_policies/test.py | 55 ----- ...02963_test_flexible_disk_configuration.sql | 34 +++- 14 files changed, 264 insertions(+), 219 deletions(-) create mode 100644 src/Disks/DiskFomAST.cpp create mode 100644 src/Disks/DiskFomAST.h delete mode 100644 src/Disks/getOrCreateDiskFromAST.cpp delete mode 100644 src/Disks/getOrCreateDiskFromAST.h diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp new file mode 100644 index 00000000000..c8a4f88547f --- /dev/null +++ b/src/Disks/DiskFomAST.cpp @@ -0,0 +1,191 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_DISK; +} + +std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string & serialization, ContextPtr context, bool attach) +{ + Poco::Util::AbstractConfiguration::Keys disk_settings_keys; + config->keys(disk_settings_keys); + + + // Check that no settings are defined when disk from the config is referred. + if (disk_settings_keys.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Disk function has no arguments. Invalid disk description."); + + if (disk_settings_keys.size() == 1 && disk_settings_keys.front() == "name" && !attach) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Disk function `{}` has to have the other arguments which describe the disk. Invalid disk description.", + serialization); + } + + std::string disk_name; + if (config->has("name")) + { + disk_name = config->getString("name"); + } + + if (!disk_name.empty()) + { + if (disk_name.starts_with(DiskSelector::CUSTOM_DISK_PREFIX)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Disk name `{}` could not start with `{}`", + disk_name, DiskSelector::CUSTOM_DISK_PREFIX); + + if (auto disk = context->tryGetDisk(disk_name)) + { + /// the disk is defined by config + if (disk->isCustomDisk()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Disk with name `{}` already exist as a custom disk but the name does not start with `{}`", + disk_name, + DiskSelector::CUSTOM_DISK_PREFIX); + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk `{}` is already exist. It is impossible to redefine it.", disk_name); + } + } + + auto disk_settings_hash = sipHash128(serialization.data(), serialization.size()); + + std::string custom_disk_name; + if (disk_name.empty()) + { + /// We need a unique name for a created custom disk, but it needs to be the same + /// after table is reattached or server is restarted, so take a hash of the disk + /// configuration serialized ast as a disk name suffix. + custom_disk_name = toString(DiskSelector::CUSTOM_DISK_PREFIX) + "noname_" + toString(disk_settings_hash); + } + else + { + custom_disk_name = toString(DiskSelector::CUSTOM_DISK_PREFIX) + disk_name; + } + + auto result_disk = context->getOrCreateDisk(custom_disk_name, [&](const DisksMap & disks_map) -> DiskPtr { + auto disk = DiskFactory::instance().create( + disk_name, *config, /* config_path */"", context, disks_map, /* attach */attach, /* custom_disk */true); + /// Mark that disk can be used without storage policy. + disk->markDiskAsCustom(disk_settings_hash); + return disk; + }); + + if (!result_disk->isCustomDisk()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Disk with name `{}` expected to be custom disk", disk_name); + + if (result_disk->getCustomDiskSettings() != disk_settings_hash && !attach) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "The disk `{}` is already configured as a custom disk in another table. It can't be redefined with different settings.", + disk_name); + + if (!attach && !result_disk->isRemote()) + { + static constexpr auto custom_local_disks_base_dir_in_config = "custom_local_disks_base_directory"; + auto disk_path_expected_prefix = context->getConfigRef().getString(custom_local_disks_base_dir_in_config, ""); + + if (disk_path_expected_prefix.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Base path for custom local disks must be defined in config file by `{}`", + custom_local_disks_base_dir_in_config); + + if (!pathStartsWith(result_disk->getPath(), disk_path_expected_prefix)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Path of the custom local disk must be inside `{}` directory", + disk_path_expected_prefix); + } + + return custom_disk_name; +} + +class DiskConfigurationFlattener +{ +public: + struct Data + { + ContextPtr context; + bool attach; + }; + + static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } + + static void visit(ASTPtr & ast, Data & data) + { + if (isDiskFunction(ast)) + { + const auto * function = ast->as(); + const auto * function_args_expr = assert_cast(function->arguments.get()); + const auto & function_args = function_args_expr->children; + auto config = getDiskConfigurationFromAST(function_args, data.context); + auto disk_setting_string = serializeAST(*function); + auto disk_name = getOrCreateCustomDisk(config, disk_setting_string, data.context, data.attach); + ast = std::make_shared(disk_name); + } + } +}; + + +std::string DiskFomAST::createCustomDisk(const ASTPtr & disk_function_ast, ContextPtr context, bool attach) +{ + if (!isDiskFunction(disk_function_ast)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected a disk function"); + + auto ast = disk_function_ast->clone(); + + using FlattenDiskConfigurationVisitor = InDepthNodeVisitor; + FlattenDiskConfigurationVisitor::Data data{context, attach}; + FlattenDiskConfigurationVisitor{data}.visit(ast); + + auto disk_name = assert_cast(*ast).value.get(); + return disk_name; +} + +std::string DiskFomAST::getConfigDefinedDisk(const std::string &disk_name, ContextPtr context) +{ + if (disk_name.starts_with(DiskSelector::CUSTOM_DISK_PREFIX)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Disk name `{}` could not start with `{}`", + disk_name, DiskSelector::CUSTOM_DISK_PREFIX); + + if (auto result = context->tryGetDisk(disk_name)) + return disk_name; + + std::string custom_disk_name = DiskSelector::CUSTOM_DISK_PREFIX + disk_name; + if (auto result = context->tryGetDisk(custom_disk_name)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Disk name `{}` is a custom disk that is used in other table." + "That disk could not be used by a reference. The custom disk should be fully specified with a disk function.", + disk_name); + + throw Exception(ErrorCodes::UNKNOWN_DISK, "Unknown disk {}", disk_name); +} + +} diff --git a/src/Disks/DiskFomAST.h b/src/Disks/DiskFomAST.h new file mode 100644 index 00000000000..3a70484eda0 --- /dev/null +++ b/src/Disks/DiskFomAST.h @@ -0,0 +1,15 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +namespace DiskFomAST +{ + std::string getConfigDefinedDisk(const std::string & name, ContextPtr context); + std::string createCustomDisk(const ASTPtr & disk_function, ContextPtr context, bool attach); +} + +} diff --git a/src/Disks/DiskSelector.h b/src/Disks/DiskSelector.h index 49a1be5cf50..0f7424460a2 100644 --- a/src/Disks/DiskSelector.h +++ b/src/Disks/DiskSelector.h @@ -6,6 +6,8 @@ #include #include +#include +#include namespace DB { @@ -18,7 +20,7 @@ using DiskSelectorPtr = std::shared_ptr; class DiskSelector { public: - static constexpr auto TMP_INTERNAL_DISK_PREFIX = "__tmp_internal_"; + static constexpr auto CUSTOM_DISK_PREFIX = "__"; explicit DiskSelector(std::unordered_set skip_types_ = {}) : skip_types(skip_types_) { } DiskSelector(const DiskSelector & from) = default; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 412ad27e94f..78d5f37e3a7 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -464,9 +464,9 @@ public: virtual void chmod(const String & /*path*/, mode_t /*mode*/) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Disk does not support chmod"); } /// Was disk created to be used without storage configuration? - bool isCustomDisk() const { return is_custom_disk; } - - void markDiskAsCustom() { is_custom_disk = true; } + bool isCustomDisk() const { return custom_disk_settings_hash != 0; } + UInt128 getCustomDiskSettings() const { return custom_disk_settings_hash; } + void markDiskAsCustom(UInt128 settings_hash) { custom_disk_settings_hash = settings_hash; } virtual DiskPtr getDelegateDiskIfExists() const { return nullptr; } @@ -504,7 +504,8 @@ protected: private: ThreadPool copying_thread_pool; - bool is_custom_disk = false; + // 0 means the disk is not custom, the disk is predefined in the config + UInt128 custom_disk_settings_hash = 0; /// Check access to the disk. void checkAccess(); diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index 501e033abc3..e23598214b3 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -13,6 +13,7 @@ #include #include +#include #include #include #include @@ -119,6 +120,7 @@ class StoragePolicySelector { public: static constexpr auto TMP_STORAGE_POLICY_PREFIX = "__"; + static_assert(std::string_view(DiskSelector::CUSTOM_DISK_PREFIX) == std::string_view(TMP_STORAGE_POLICY_PREFIX)); StoragePolicySelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks); diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp deleted file mode 100644 index fd43f31a009..00000000000 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ /dev/null @@ -1,121 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -namespace -{ - std::string getOrCreateDiskFromDiskAST(const ASTFunction & function, ContextPtr context, bool attach) - { - const auto * function_args_expr = assert_cast(function.arguments.get()); - const auto & function_args = function_args_expr->children; - auto config = getDiskConfigurationFromAST(function_args, context); - - std::string disk_name; - if (config->has("name")) - { - disk_name = config->getString("name"); - } - else - { - /// We need a unique name for a created custom disk, but it needs to be the same - /// after table is reattached or server is restarted, so take a hash of the disk - /// configuration serialized ast as a disk name suffix. - auto disk_setting_string = serializeAST(function); - disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX - + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); - } - - auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { - auto disk = DiskFactory::instance().create( - disk_name, *config, /* config_path */"", context, disks_map, /* attach */attach, /* custom_disk */true); - /// Mark that disk can be used without storage policy. - disk->markDiskAsCustom(); - return disk; - }); - - if (!result_disk->isCustomDisk()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk with name `{}` already exist", disk_name); - - if (!attach && !result_disk->isRemote()) - { - static constexpr auto custom_local_disks_base_dir_in_config = "custom_local_disks_base_directory"; - auto disk_path_expected_prefix = context->getConfigRef().getString(custom_local_disks_base_dir_in_config, ""); - - if (disk_path_expected_prefix.empty()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Base path for custom local disks must be defined in config file by `{}`", - custom_local_disks_base_dir_in_config); - - if (!pathStartsWith(result_disk->getPath(), disk_path_expected_prefix)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Path of the custom local disk must be inside `{}` directory", - disk_path_expected_prefix); - } - - return disk_name; - } - - class DiskConfigurationFlattener - { - public: - struct Data - { - ContextPtr context; - bool attach; - }; - - static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } - - static void visit(ASTPtr & ast, Data & data) - { - if (isDiskFunction(ast)) - { - auto disk_name = getOrCreateDiskFromDiskAST(*ast->as(), data.context, data.attach); - ast = std::make_shared(disk_name); - } - } - }; - - /// Visits children first. - using FlattenDiskConfigurationVisitor = InDepthNodeVisitor; -} - - -std::string getOrCreateDiskFromDiskAST(const ASTPtr & disk_function, ContextPtr context, bool attach) -{ - if (!isDiskFunction(disk_function)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected a disk function"); - - auto ast = disk_function->clone(); - - FlattenDiskConfigurationVisitor::Data data{context, attach}; - FlattenDiskConfigurationVisitor{data}.visit(ast); - - auto disk_name = assert_cast(*ast).value.get(); - LOG_TRACE(getLogger("getOrCreateDiskFromDiskAST"), "Result disk name: {}", disk_name); - return disk_name; -} - -} diff --git a/src/Disks/getOrCreateDiskFromAST.h b/src/Disks/getOrCreateDiskFromAST.h deleted file mode 100644 index 61e1decbee9..00000000000 --- a/src/Disks/getOrCreateDiskFromAST.h +++ /dev/null @@ -1,18 +0,0 @@ -#pragma once -#include -#include -#include - -namespace DB -{ - -class ASTFunction; - -/** - * Create a DiskPtr from disk AST function like disk(), - * add it to DiskSelector by a unique (but always the same for given configuration) disk name - * and return this name. - */ -std::string getOrCreateDiskFromDiskAST(const ASTPtr & disk_function, ContextPtr context, bool attach); - -} diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5413b568068..0acbef26805 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4,6 +4,7 @@ #include #include #include +#include "Common/Logger.h" #include #include #include @@ -4395,6 +4396,15 @@ DiskPtr Context::getDisk(const String & name) const return disk_selector->get(name); } +DiskPtr Context::tryGetDisk(const String & name) const +{ + std::lock_guard lock(shared->storage_policies_mutex); + + auto disk_selector = getDiskSelector(lock); + + return disk_selector->tryGet(name); +} + DiskPtr Context::getOrCreateDisk(const String & name, DiskCreator creator) const { std::lock_guard lock(shared->storage_policies_mutex); @@ -4422,9 +4432,11 @@ StoragePolicyPtr Context::getStoragePolicy(const String & name) const StoragePolicyPtr Context::getStoragePolicyFromDisk(const String & disk_name) const { + LOG_DEBUG(getLogger("StoragePolicy"), "getStoragePolicyFromDisk disk_name {}", disk_name); + std::lock_guard lock(shared->storage_policies_mutex); - const std::string storage_policy_name = StoragePolicySelector::TMP_STORAGE_POLICY_PREFIX + disk_name; + const std::string storage_policy_name = disk_name.starts_with(DiskSelector::CUSTOM_DISK_PREFIX) ? disk_name : StoragePolicySelector::TMP_STORAGE_POLICY_PREFIX + disk_name; auto storage_policy_selector = getStoragePolicySelector(lock); StoragePolicyPtr storage_policy = storage_policy_selector->tryGet(storage_policy_name); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index d5e35c3e4b3..586eff768df 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1186,6 +1186,7 @@ public: /// Provides storage disks DiskPtr getDisk(const String & name) const; + DiskPtr tryGetDisk(const String & name) const; using DiskCreator = std::function; DiskPtr getOrCreateDisk(const String & name, DiskCreator creator) const; diff --git a/src/Parsers/FieldFromAST.cpp b/src/Parsers/FieldFromAST.cpp index ad1eab49eeb..64aeae1b570 100644 --- a/src/Parsers/FieldFromAST.cpp +++ b/src/Parsers/FieldFromAST.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index c968ad84936..f72b24e3270 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -64,10 +64,14 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr conte auto ast = dynamic_cast(custom.getImpl()).ast; if (ast && isDiskFunction(ast)) { - auto disk_name = getOrCreateDiskFromDiskAST(ast, context, is_attach); - LOG_TRACE(getLogger("MergeTreeSettings"), "Created custom disk {}", disk_name); + auto disk_name = DiskFomAST::createCustomDisk(ast, context, is_attach); + LOG_DEBUG(getLogger("MergeTreeSettings"), "Created custom disk {}", disk_name); value = disk_name; } + else + { + value = DiskFomAST::getConfigDefinedDisk(value.safeGet(), context); + } } if (has("storage_policy")) diff --git a/tests/integration/test_storage_policies/configs/disks.xml b/tests/integration/test_storage_policies/configs/disks.xml index dc60d93208c..3331fee4e4f 100644 --- a/tests/integration/test_storage_policies/configs/disks.xml +++ b/tests/integration/test_storage_policies/configs/disks.xml @@ -1,12 +1,4 @@ - - system - blob_storage_log
- toYYYYMM(event_date) - 7500 - event_date + INTERVAL 30 DAY -
- diff --git a/tests/integration/test_storage_policies/test.py b/tests/integration/test_storage_policies/test.py index f65096389af..389146b2171 100644 --- a/tests/integration/test_storage_policies/test.py +++ b/tests/integration/test_storage_policies/test.py @@ -38,58 +38,3 @@ def test_storage_policy_configuration_change(started_cluster): "/etc/clickhouse-server/config.d/disks.xml", ) node.start_clickhouse() - - -def test_disk_is_immutable(started_cluster): - node.query("DROP TABLE IF EXISTS test_1") - - node.query( - """ - create table test_1 (a Int32) - engine = MergeTree() - order by tuple() - settings - disk=disk( - name='not_uniq_disk_name', - type = object_storage, - object_storage_type = local_blob_storage, - path='./03215_data_test_1/') - """ - ) - - node.query("INSERT INTO test_1 VALUES (1)") - node.query("SYSTEM FLUSH LOGS;") - - print(node.query("SELECT 'test_1', * FROM system.blob_storage_log")) - - print(node.query("SELECT 'test_1', * FROM test_1")) - - node.query("DROP TABLE test_1 SYNC") - node.query("DROP TABLE IF EXISTS test_2") - - node.query( - """ - create table test_2 (a Int32) - engine = MergeTree() - order by tuple() - settings - disk=disk( - name='not_uniq_disk_name', - type = object_storage, - object_storage_type = local_blob_storage, - path='./03215_data_test_2/') - """ - ) - - node.query("INSERT INTO test_2 VALUES (1)") - node.query("SYSTEM FLUSH LOGS;") - - print(node.query("SELECT 'test_2', * FROM system.blob_storage_log")) - - print(node.query("SELECT 'test_2', * FROM test_2")) - - node.restart_clickhouse() - - print(node.query("SELECT 'test_2', * FROM system.blob_storage_log")) - - print(node.query("SELECT 'test_2', * FROM test_2")) diff --git a/tests/queries/0_stateless/02963_test_flexible_disk_configuration.sql b/tests/queries/0_stateless/02963_test_flexible_disk_configuration.sql index 7ebef866360..6b285d423e7 100644 --- a/tests/queries/0_stateless/02963_test_flexible_disk_configuration.sql +++ b/tests/queries/0_stateless/02963_test_flexible_disk_configuration.sql @@ -2,13 +2,33 @@ drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple() -settings disk=disk(name='test1', type = object_storage, object_storage_type = local_blob_storage, path='./02963_test1/'); +settings disk=disk(name='02963_custom_disk', type = object_storage, object_storage_type = local_blob_storage, path='./02963_test1/'); -drop table test; +drop table if exists test; +create table test (a Int32) engine = MergeTree() order by tuple() +settings disk=disk(name='02963_custom_disk', type = object_storage, object_storage_type = local_blob_storage, path='./02963_test2/'); -- { serverError BAD_ARGUMENTS } + +drop table if exists test; +create table test (a Int32) engine = MergeTree() order by tuple() +settings disk=disk(name='02963_custom_disk'); -- { serverError BAD_ARGUMENTS } + +drop table if exists test; +create table test (a Int32) engine = MergeTree() order by tuple() +settings disk='02963_custom_disk'; -- { serverError BAD_ARGUMENTS } + +drop table if exists test; +create table test (a Int32) engine = MergeTree() order by tuple() +settings disk=disk(name='s3_disk_02963'); -- { serverError BAD_ARGUMENTS } + +drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple() settings disk='s3_disk_02963'; -drop table test; +drop table if exists test; +create table test (a Int32) engine = MergeTree() order by tuple() +settings disk=disk(name='s3_disk_02963', type = object_storage, object_storage_type = local_blob_storage, path='./02963_test2/'); -- { serverError BAD_ARGUMENTS } + +drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple() settings disk=disk(name='test1', type = object_storage, @@ -17,7 +37,7 @@ settings disk=disk(name='test1', access_key_id = clickhouse, secret_access_key = clickhouse); -drop table test; +drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple() settings disk=disk(name='test2', type = object_storage, @@ -27,7 +47,7 @@ settings disk=disk(name='test2', access_key_id = clickhouse, secret_access_key = clickhouse); -drop table test; +drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple() settings disk=disk(name='test3', type = object_storage, @@ -37,8 +57,8 @@ settings disk=disk(name='test3', endpoint = 'http://localhost:11111/test/common/', access_key_id = clickhouse, secret_access_key = clickhouse); -drop table test; +drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple() settings disk=disk(name='test4', type = object_storage, @@ -48,8 +68,8 @@ settings disk=disk(name='test4', endpoint = 'http://localhost:11111/test/common/', access_key_id = clickhouse, secret_access_key = clickhouse); -drop table test; +drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple() settings disk=disk(name='test5', type = object_storage, From 4e2b6a5e4927da8a02cf15b374d0e351108d1d88 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 6 Aug 2024 13:06:34 +0200 Subject: [PATCH 300/644] Fix docs and a test --- .../engines/table-engines/integrations/time-series.md | 10 +++++++--- docs/en/interfaces/prometheus.md | 1 + tests/integration/test_prometheus_protocols/test.py | 5 +++++ 3 files changed, 13 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/time-series.md b/docs/en/engines/table-engines/integrations/time-series.md index b9e47e8d2c9..2914bf4bf37 100644 --- a/docs/en/engines/table-engines/integrations/time-series.md +++ b/docs/en/engines/table-engines/integrations/time-series.md @@ -8,7 +8,7 @@ sidebar_label: TimeSeries A table engine storing time series, i.e. a set of values associated with timestamps and tags (or labels): -``` +```text metric_name1[tag1=value1, tag2=value2, ...] = {timestamp1: value1, timestamp2: value2, ...} metric_name2[...] = ... ``` @@ -39,6 +39,7 @@ CREATE TABLE my_table ENGINE=TimeSeries ``` Then this table can be used with the following protocols (a port must be assigned in the server configuration): + - [prometheus remote-write](../../../interfaces/prometheus.md#remote-write) - [prometheus remote-read](../../../interfaces/prometheus.md#remote-read) @@ -53,6 +54,7 @@ The target tables can be either specified explicitly in the `CREATE TABLE` query or the `TimeSeries` table engine can generate inner target tables automatically. The target tables are the following: + 1. The _data_ table {#data-table} contains time series associated with some identifier. The _data_ table must have columns: @@ -71,7 +73,7 @@ The _tags_ table must have columns: | `metric_name` | [x] | `LowCardinality(String)` | `String` or `LowCardinality(String)` | The name of a metric | | `` | [ ] | `String` | `String` or `LowCardinality(String)` or `LowCardinality(Nullable(String))` | The value of a specific tag, the tag's name and the name of a corresponding column are specified in the [tags_to_columns](#settings) setting | | `tags` | [x] | `Map(LowCardinality(String), String)` | `Map(String, String)` or `Map(LowCardinality(String), String)` or `Map(LowCardinality(String), LowCardinality(String))` | Map of tags excluding the tag `__name__` containing the name of a metric and excluding tags with names enumerated in the [tags_to_columns](#settings) setting | -| `all_tags` | [ ] | `Map(String, String)` | `Map(String, String)` or `Map(LowCardinality(String), String)` or `Map(LowCardinality(String), LowCardinality(String))` | Ephemeral column, each row is a map of all the tags excluding only the tag `__name__` containing the name of a metric. The only purpose of that column is to be used while calculating `id` | +| `all_tags` | [ ] | `Map(String, LowCardinality(String))` | `Map(String, String)` or `Map(LowCardinality(String), String)` or `Map(LowCardinality(String), LowCardinality(String))` | Ephemeral column, each row is a map of all the tags excluding only the tag `__name__` containing the name of a metric. The only purpose of that column is to be used while calculating `id` | | `min_time` | [ ] | `Nullable(DateTime64(3))` | `DateTime64(X)` or `Nullable(DateTime64(X))` | Minimum timestamp of time series with that `id`. The column is created if [store_min_time_and_max_time](#settings) is `true` | | `max_time` | [ ] | `Nullable(DateTime64(3))` | `DateTime64(X)` or `Nullable(DateTime64(X))` | Maximum timestamp of time series with that `id`. The column is created if [store_min_time_and_max_time](#settings) is `true` | @@ -239,6 +241,7 @@ ENGINE=TimeSeries SETTINGS = {'instance': 'instance', 'job': 'job'} ## Table engines of inner target tables {#inner-table-engines} By default inner target tables use the following table engines: + - the [data]{#data-table} table uses [MergeTree](../mergetree-family/mergetree); - the [tags]{#tags-table} table uses [AggregatingMergeTree](../mergetree-family/aggregatingmergetree) because the same data is often inserted multiple times to this table so we need a way to remove duplicates, and also because it's required to do aggregation for columns `min_time` and `max_time`; @@ -287,9 +290,10 @@ Here is a list of settings which can be specified while defining a `TimeSeries` | `aggregate_min_time_and_max_time` | Bool | true | When creating an inner target `tags` table, this flag enables using `SimpleAggregateFunction(min, Nullable(DateTime64(3)))` instead of just `Nullable(DateTime64(3))` as the type of the `min_time` column, and the same for the `max_time` column | | `filter_by_min_time_and_max_time` | Bool | true | If set to true then the table will use the `min_time` and `max_time` columns for filtering time series | -# Functions {#functions} +## Functions {#functions} Here is a list of functions supporting a `TimeSeries` table as an argument: + - [timeSeriesData](../../../sql-reference/table-functions/timeSeriesData.md) - [timeSeriesTags](../../../sql-reference/table-functions/timeSeriesTags.md) - [timeSeriesMetrics](../../../sql-reference/table-functions/timeSeriesMetrics.md) diff --git a/docs/en/interfaces/prometheus.md b/docs/en/interfaces/prometheus.md index 75a68c59219..5eac99f685e 100644 --- a/docs/en/interfaces/prometheus.md +++ b/docs/en/interfaces/prometheus.md @@ -58,6 +58,7 @@ Settings: | `errors` | true | Expose the number of errors by error codes occurred since the last server restart. This information could be obtained from the [system.errors](../operations/system-tables/errors.md) as well. | Check (replace `127.0.0.1` with the IP addr or hostname of your ClickHouse server): + ```bash curl 127.0.0.1:9363/metrics ``` diff --git a/tests/integration/test_prometheus_protocols/test.py b/tests/integration/test_prometheus_protocols/test.py index 488c5369742..6adb3da56c3 100644 --- a/tests/integration/test_prometheus_protocols/test.py +++ b/tests/integration/test_prometheus_protocols/test.py @@ -144,6 +144,11 @@ def test_inner_engines(): def test_external_tables(): + node.query("DROP TABLE IF EXISTS mydata") + node.query("DROP TABLE IF EXISTS mytags") + node.query("DROP TABLE IF EXISTS mymetrics") + node.query("DROP TABLE IF EXISTS prometheus") + node.query( "CREATE TABLE mydata (id UUID, timestamp DateTime64(3), value Float64) " "ENGINE=MergeTree ORDER BY (id, timestamp)" From 27cdbb54d73f8f4b82d63850c1e6f6fd5669646e Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 6 Aug 2024 13:10:03 +0200 Subject: [PATCH 301/644] fix black --- tests/integration/test_storage_hdfs/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 3fef6bc46cf..77921b885b0 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -675,7 +675,9 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_2', 'Parquet', 'a Int32, b String')" ) - node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") + node1.query( + f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" + ) result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "parquet_2" @@ -683,7 +685,9 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_3', 'Parquet', 'a Int32, _path String')" ) - node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") + node1.query( + f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" + ) result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "kek" From 97f1cfc232c331504754db4ba4221a282c690bb6 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Wed, 24 Jul 2024 14:32:35 +0000 Subject: [PATCH 302/644] add ** glob to hdfs docs --- docs/en/sql-reference/table-functions/hdfs.md | 1 + docs/ru/sql-reference/table-functions/file.md | 1 + docs/ru/sql-reference/table-functions/hdfs.md | 1 + tests/integration/test_storage_hdfs/test.py | 16 ++++++++++++++++ 4 files changed, 19 insertions(+) diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 28cba5ccc6a..405ac477846 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -44,6 +44,7 @@ LIMIT 2 Paths may use globbing. Files must match the whole path pattern, not only the suffix or prefix. - `*` — Represents arbitrarily many characters except `/` but including the empty string. +- `**` — Represents all files inside a folder recursively. - `?` — Represents an arbitrary single character. - `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. The strings can contain the `/` symbol. - `{N..M}` — Represents any number `>= N` and `<= M`. diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index 546a674d41a..f3e8b0f46b7 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -81,6 +81,7 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U Обрабатываться будут те и только те файлы, которые существуют в файловой системе и удовлетворяют всему шаблону пути. - `*` — заменяет любое количество любых символов кроме `/`, включая отсутствие символов. +- `**` — Заменяет любое количество любых символов, включая `/`, то есть осуществляет рекурсивный поиск по вложенным директориям. - `?` — заменяет ровно один любой символ. - `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. Эти строки также могут содержать символ `/`. - `{N..M}` — заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули). diff --git a/docs/ru/sql-reference/table-functions/hdfs.md b/docs/ru/sql-reference/table-functions/hdfs.md index 6dcb1a21791..13f1bdc43af 100644 --- a/docs/ru/sql-reference/table-functions/hdfs.md +++ b/docs/ru/sql-reference/table-functions/hdfs.md @@ -47,6 +47,7 @@ LIMIT 2 - `*` — Заменяет любое количество любых символов (кроме `/`), включая отсутствие символов. +- `**` — Заменяет любое количество любых символов, включая `/`, то есть осуществляет рекурсивный поиск по вложенным директориям. - `?` — Заменяет ровно один любой символ. - `{some_string,another_string,yet_another_one}` — Заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. Эти строки также могут содержать символ `/`. - `{N..M}` — Заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули). diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index ccd2c7eaf11..9b2dacd3d5d 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -61,6 +61,14 @@ def test_read_write_storage_with_globs(started_cluster): hdfs_api.write_data("/storage" + i, i + "\tMark\t72.53\n") assert hdfs_api.read_data("/storage" + i) == i + "\tMark\t72.53\n" + node1.query( + "create table HDFSStorageWithDoubleAsterisk (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/**.tsv', 'TSV')" + ) + + for i in ["1", "2", "3"]: + hdfs_api.write_data(f"/subdir/file{i}.tsv", f"{i}\tMark\t72.53\n") + assert hdfs_api.read_data(f"/subdir/file{i}.tsv") == f"{i}\tMark\t72.53\n" + assert ( node1.query( "select count(*) from HDFSStorageWithRange settings s3_throw_on_zero_files_match=1" @@ -70,6 +78,7 @@ def test_read_write_storage_with_globs(started_cluster): assert node1.query("select count(*) from HDFSStorageWithEnum") == "3\n" assert node1.query("select count(*) from HDFSStorageWithQuestionMark") == "3\n" assert node1.query("select count(*) from HDFSStorageWithAsterisk") == "3\n" + assert node1.query("select count(*) from HDFSStorageWithDoubleAsterisk") == "3\n" try: node1.query("insert into HDFSStorageWithEnum values (1, 'NEW', 4.2)") @@ -92,6 +101,13 @@ def test_read_write_storage_with_globs(started_cluster): print(ex) assert "in readonly mode" in str(ex) + try: + node1.query("insert into HDFSStorageWithDoubleAsterisk values (1, 'NEW', 4.2)") + assert False, "Exception have to be thrown" + except Exception as ex: + print(ex) + assert "in readonly mode" in str(ex) + def test_storage_with_multidirectory_glob(started_cluster): hdfs_api = started_cluster.hdfs_api From 7d45424d318f84a2d035a01290d263a620859bca Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Fri, 26 Jul 2024 08:41:14 +0000 Subject: [PATCH 303/644] specify files in test: --- tests/integration/test_storage_hdfs/test.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 9b2dacd3d5d..51d3eed91df 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -62,12 +62,15 @@ def test_read_write_storage_with_globs(started_cluster): assert hdfs_api.read_data("/storage" + i) == i + "\tMark\t72.53\n" node1.query( - "create table HDFSStorageWithDoubleAsterisk (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/**.tsv', 'TSV')" + "create table HDFSStorageWithDoubleAsterisk (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/**.doublestar.tsv', 'TSV')" ) for i in ["1", "2", "3"]: - hdfs_api.write_data(f"/subdir/file{i}.tsv", f"{i}\tMark\t72.53\n") - assert hdfs_api.read_data(f"/subdir/file{i}.tsv") == f"{i}\tMark\t72.53\n" + hdfs_api.write_data(f"/subdir{i}/file{i}.doublestar.tsv", f"{i}\tMark\t72.53\n") + assert ( + hdfs_api.read_data(f"/subdir{i}/file{i}.doublestar.tsv") + == f"{i}\tMark\t72.53\n" + ) assert ( node1.query( From 913b9028649f668020850de91182d975d2a2916a Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 6 Aug 2024 12:51:16 +0200 Subject: [PATCH 304/644] CI: Fix for setting Mergeable Check from sync --- .github/workflows/jepsen.yml | 2 +- tests/ci/commit_status_helper.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/jepsen.yml b/.github/workflows/jepsen.yml index 035ba2e5b98..d4d676ceead 100644 --- a/.github/workflows/jepsen.yml +++ b/.github/workflows/jepsen.yml @@ -66,7 +66,7 @@ jobs: - name: Check Workflow results run: | export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" - cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' + cat > "$WORKFLOW_RESULT_FILE" << 'EOF' ${{ toJson(needs) }} EOF python3 ./tests/ci/ci_buddy.py --check-wf-status diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 908ac4a7dca..7e3a0b4965c 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -497,9 +497,9 @@ def trigger_mergeable_check( description = format_description(description) if set_from_sync: - # update Mergeable Check from sync WF only if its status already present or its new status is not SUCCESS + # update Mergeable Check from sync WF only if its status already present or its new status is FAILURE # to avoid false-positives - if mergeable_status or state != SUCCESS: + if mergeable_status or state == FAILURE: set_mergeable_check(commit, description, state) elif mergeable_status is None or mergeable_status.description != description: set_mergeable_check(commit, description, state) From 2776a515ba36be6946919bc43ba267797d929cdd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 12:29:54 +0000 Subject: [PATCH 305/644] Cosmetics I --- src/Storages/Statistics/Statistics.cpp | 12 ++++++------ src/Storages/Statistics/StatisticsCountMinSketch.cpp | 4 ++-- src/Storages/Statistics/StatisticsCountMinSketch.h | 4 ++-- src/Storages/Statistics/StatisticsTDigest.cpp | 4 ++-- src/Storages/Statistics/StatisticsTDigest.h | 4 ++-- src/Storages/Statistics/StatisticsUniq.cpp | 4 ++-- src/Storages/Statistics/StatisticsUniq.h | 4 ++-- 7 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index ade3326288a..2a17101478a 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -204,15 +204,15 @@ void MergeTreeStatisticsFactory::registerValidator(StatisticsType stats_type, Va MergeTreeStatisticsFactory::MergeTreeStatisticsFactory() { - registerValidator(StatisticsType::TDigest, tdigestValidator); - registerCreator(StatisticsType::TDigest, tdigestCreator); + registerValidator(StatisticsType::TDigest, tdigestStatisticsValidator); + registerCreator(StatisticsType::TDigest, tdigestStatisticsCreator); - registerValidator(StatisticsType::Uniq, uniqValidator); - registerCreator(StatisticsType::Uniq, uniqCreator); + registerValidator(StatisticsType::Uniq, uniqStatisticsValidator); + registerCreator(StatisticsType::Uniq, uniqStatisticsCreator); #if USE_DATASKETCHES - registerValidator(StatisticsType::CountMinSketch, countMinSketchValidator); - registerCreator(StatisticsType::CountMinSketch, countMinSketchCreator); + registerValidator(StatisticsType::CountMinSketch, countMinSketchStatisticsValidator); + registerCreator(StatisticsType::CountMinSketch, countMinSketchStatisticsCreator); #endif } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index e69bbc1515b..50d3b6e515c 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -84,7 +84,7 @@ void StatisticsCountMinSketch::deserialize(ReadBuffer & buf) } -void countMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void countMinSketchStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -92,7 +92,7 @@ void countMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr da throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName()); } -StatisticsPtr countMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { return std::make_shared(stat, data_type); } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.h b/src/Storages/Statistics/StatisticsCountMinSketch.h index 6c8b74f8c35..d10bc78a88e 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.h +++ b/src/Storages/Statistics/StatisticsCountMinSketch.h @@ -31,8 +31,8 @@ private: }; -void countMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr countMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void countMinSketchStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr); } diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index 66150e00fdb..7c5ea443201 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -57,7 +57,7 @@ Float64 StatisticsTDigest::estimateEqual(const Field & val) const throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName()); } -void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void tdigestStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -65,7 +65,7 @@ void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName()); } -StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr) +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr) { return std::make_shared(stat); } diff --git a/src/Storages/Statistics/StatisticsTDigest.h b/src/Storages/Statistics/StatisticsTDigest.h index 614973e5d8b..d41b0648aa4 100644 --- a/src/Storages/Statistics/StatisticsTDigest.h +++ b/src/Storages/Statistics/StatisticsTDigest.h @@ -23,7 +23,7 @@ private: QuantileTDigest t_digest; }; -void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void tdigestStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr); } diff --git a/src/Storages/Statistics/StatisticsUniq.cpp b/src/Storages/Statistics/StatisticsUniq.cpp index 8f60ffcf0b5..c259f09e0ae 100644 --- a/src/Storages/Statistics/StatisticsUniq.cpp +++ b/src/Storages/Statistics/StatisticsUniq.cpp @@ -52,7 +52,7 @@ UInt64 StatisticsUniq::estimateCardinality() const return column->getUInt(0); } -void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void uniqStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -60,7 +60,7 @@ void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName()); } -StatisticsPtr uniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { return std::make_shared(stat, data_type); } diff --git a/src/Storages/Statistics/StatisticsUniq.h b/src/Storages/Statistics/StatisticsUniq.h index faabde8d47c..743714e1e1b 100644 --- a/src/Storages/Statistics/StatisticsUniq.h +++ b/src/Storages/Statistics/StatisticsUniq.h @@ -27,7 +27,7 @@ private: }; -void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr uniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); +void uniqStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); } From 47e4ea598267d33e5db72623069629a9025b0c0d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 14:31:22 +0200 Subject: [PATCH 306/644] Make 02984_form_format.sh parallelizable --- tests/queries/0_stateless/02984_form_format.sh | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02984_form_format.sh b/tests/queries/0_stateless/02984_form_format.sh index 471b48e0f68..814fe1a0f2c 100755 --- a/tests/queries/0_stateless/02984_form_format.sh +++ b/tests/queries/0_stateless/02984_form_format.sh @@ -1,6 +1,4 @@ #!/bin/bash -# Tags: no-parallel - CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh @@ -8,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) FILE_NAME="data.tmp" FORM_DATA="${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/${FILE_NAME}" mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -touch $FORM_DATA +touch $FORM_DATA # Simple tests echo -ne "col1=42&col2=Hello%2C%20World%21" > $FORM_DATA From 996699c78cbc4709a69ff866e487bba640fa0cfd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 14:34:15 +0200 Subject: [PATCH 307/644] Make 01456_modify_column_type_via_add_drop_update parallelizable --- .../01456_modify_column_type_via_add_drop_update.sql | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/01456_modify_column_type_via_add_drop_update.sql b/tests/queries/0_stateless/01456_modify_column_type_via_add_drop_update.sql index b7cbfc92c26..a2e4804f12e 100644 --- a/tests/queries/0_stateless/01456_modify_column_type_via_add_drop_update.sql +++ b/tests/queries/0_stateless/01456_modify_column_type_via_add_drop_update.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel - DROP TABLE IF EXISTS tbl; CREATE TABLE tbl(a String, b UInt32, c Float64, d Int64, e UInt8) ENGINE=MergeTree ORDER BY tuple(); INSERT INTO tbl SELECT number, number * 2, number * 3, number * 4, number * 5 FROM system.numbers LIMIT 10; From d09c82ff76186a93b0c521c67ab30d9101bc4769 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 12:33:18 +0000 Subject: [PATCH 308/644] Cosmetics II --- .../mergetree-family/mergetree.md | 2 +- src/Storages/Statistics/Statistics.cpp | 20 ++++++++++--------- .../Statistics/StatisticsCountMinSketch.cpp | 10 +++++----- .../Statistics/StatisticsCountMinSketch.h | 6 +++--- src/Storages/Statistics/StatisticsTDigest.cpp | 10 +++++----- src/Storages/Statistics/StatisticsTDigest.h | 6 +++--- src/Storages/Statistics/StatisticsUniq.cpp | 10 +++++----- src/Storages/Statistics/StatisticsUniq.h | 6 +++--- 8 files changed, 36 insertions(+), 34 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 7ffbd9a5bae..183b94f4641 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1005,7 +1005,7 @@ They can be used for prewhere optimization only if we enable `set allow_statisti ## Column-level Settings {#column-level-settings} -Certain MergeTree settings can be override at column level: +Certain MergeTree settings can be overridden at column level: - `max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. - `min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 2a17101478a..771304405a6 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -89,15 +89,17 @@ Float64 IStatistics::estimateLess(const Field & /*val*/) const throw Exception(ErrorCodes::LOGICAL_ERROR, "Less-than estimation is not implemented for this type of statistics"); } -/// ------------------------------------- -/// Implementation of the estimation: -/// Note: Each statistics object supports certain types predicates natively, e.g. -/// - TDigest: '< X' (less-than predicates) -/// - Count-min sketches: '= X' (equal predicates) -/// - Uniq (HyperLogLog): 'count distinct(*)' (column cardinality) -/// If multiple statistics objects are available per column, it is sometimes also possible to combine them in a clever way. -/// For that reason, all estimation are performed in a central place (here), and we don't simply pass the predicate to the first statistics -/// object that supports it natively. +/// Notes: +/// - Statistics object usually only support estimation for certain types of predicates, e.g. +/// - TDigest: '< X' (less-than predicates) +/// - Count-min sketches: '= X' (equal predicates) +/// - Uniq (HyperLogLog): 'count distinct(*)' (column cardinality) +/// +/// If multiple statistics objects in a column support estimating a predicate, we want to try statistics in order of descending accuracy +/// (e.g. MinMax statistics are simpler than TDigest statistics and thus worse for estimating 'less' predicates). +/// +/// Sometimes, it is possible to combine multiple statistics in a clever way. For that reason, all estimation are performed in a central +/// place (here), and we don't simply pass the predicate to the first statistics object that supports it natively. Float64 ColumnStatistics::estimateLess(const Field & val) const { diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index 50d3b6e515c..dce5b39ae56 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -25,8 +25,8 @@ extern const int ILLEGAL_STATISTICS; static constexpr auto num_hashes = 7uz; static constexpr auto num_buckets = 2718uz; -StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) - : IStatistics(stat_) +StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & description, DataTypePtr data_type_) + : IStatistics(description) , sketch(num_hashes, num_buckets) , data_type(data_type_) { @@ -84,7 +84,7 @@ void StatisticsCountMinSketch::deserialize(ReadBuffer & buf) } -void countMinSketchStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void countMinSketchStatisticsValidator(const SingleStatisticsDescription & /*description*/, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -92,9 +92,9 @@ void countMinSketchStatisticsValidator(const SingleStatisticsDescription &, Data throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName()); } -StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & description, DataTypePtr data_type) { - return std::make_shared(stat, data_type); + return std::make_shared(description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.h b/src/Storages/Statistics/StatisticsCountMinSketch.h index d10bc78a88e..af01408f2a3 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.h +++ b/src/Storages/Statistics/StatisticsCountMinSketch.h @@ -14,7 +14,7 @@ namespace DB class StatisticsCountMinSketch : public IStatistics { public: - StatisticsCountMinSketch(const SingleStatisticsDescription & stat_, DataTypePtr data_type_); + StatisticsCountMinSketch(const SingleStatisticsDescription & description, DataTypePtr data_type_); Float64 estimateEqual(const Field & val) const override; @@ -31,8 +31,8 @@ private: }; -void countMinSketchStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void countMinSketchStatisticsValidator(const SingleStatisticsDescription & description, DataTypePtr data_type); +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & description, DataTypePtr data_type); } diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index 7c5ea443201..73ab6c84b4e 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -10,8 +10,8 @@ extern const int ILLEGAL_STATISTICS; extern const int LOGICAL_ERROR; } -StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & stat_) - : IStatistics(stat_) +StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & description) + : IStatistics(description) { } @@ -57,7 +57,7 @@ Float64 StatisticsTDigest::estimateEqual(const Field & val) const throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName()); } -void tdigestStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void tdigestStatisticsValidator(const SingleStatisticsDescription & /*description*/, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -65,9 +65,9 @@ void tdigestStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName()); } -StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr) +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, DataTypePtr /*data_type*/) { - return std::make_shared(stat); + return std::make_shared(description); } } diff --git a/src/Storages/Statistics/StatisticsTDigest.h b/src/Storages/Statistics/StatisticsTDigest.h index d41b0648aa4..47d6c93f64c 100644 --- a/src/Storages/Statistics/StatisticsTDigest.h +++ b/src/Storages/Statistics/StatisticsTDigest.h @@ -9,7 +9,7 @@ namespace DB class StatisticsTDigest : public IStatistics { public: - explicit StatisticsTDigest(const SingleStatisticsDescription & stat_); + explicit StatisticsTDigest(const SingleStatisticsDescription & description); void update(const ColumnPtr & column) override; @@ -23,7 +23,7 @@ private: QuantileTDigest t_digest; }; -void tdigestStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void tdigestStatisticsValidator(const SingleStatisticsDescription & description, DataTypePtr data_type); +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, DataTypePtr data_type); } diff --git a/src/Storages/Statistics/StatisticsUniq.cpp b/src/Storages/Statistics/StatisticsUniq.cpp index c259f09e0ae..e737f9987a5 100644 --- a/src/Storages/Statistics/StatisticsUniq.cpp +++ b/src/Storages/Statistics/StatisticsUniq.cpp @@ -11,8 +11,8 @@ namespace ErrorCodes extern const int ILLEGAL_STATISTICS; } -StatisticsUniq::StatisticsUniq(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type) - : IStatistics(stat_) +StatisticsUniq::StatisticsUniq(const SingleStatisticsDescription & description, const DataTypePtr & data_type) + : IStatistics(description) { arena = std::make_unique(); AggregateFunctionProperties properties; @@ -52,7 +52,7 @@ UInt64 StatisticsUniq::estimateCardinality() const return column->getUInt(0); } -void uniqStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void uniqStatisticsValidator(const SingleStatisticsDescription & /*description*/, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -60,9 +60,9 @@ void uniqStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr da throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName()); } -StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & description, DataTypePtr data_type) { - return std::make_shared(stat, data_type); + return std::make_shared(description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsUniq.h b/src/Storages/Statistics/StatisticsUniq.h index 743714e1e1b..6b511d4f496 100644 --- a/src/Storages/Statistics/StatisticsUniq.h +++ b/src/Storages/Statistics/StatisticsUniq.h @@ -10,7 +10,7 @@ namespace DB class StatisticsUniq : public IStatistics { public: - StatisticsUniq(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type); + StatisticsUniq(const SingleStatisticsDescription & description, const DataTypePtr & data_type); ~StatisticsUniq() override; void update(const ColumnPtr & column) override; @@ -27,7 +27,7 @@ private: }; -void uniqStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); +void uniqStatisticsValidator(const SingleStatisticsDescription & description, DataTypePtr data_type); +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & description, DataTypePtr data_type); } From f3ee25036f9c5796a9018699d575f94bf75a50b5 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Thu, 2 May 2024 17:39:54 +0000 Subject: [PATCH 309/644] Building aarch64 builds with '-no-pie' to allow better introspection --- CMakeLists.txt | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index f796e6c4616..0d862b23e3a 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -428,12 +428,14 @@ if (NOT SANITIZE) set (CMAKE_POSITION_INDEPENDENT_CODE OFF) endif() -if (OS_LINUX AND NOT (ARCH_AARCH64 OR ARCH_S390X) AND NOT SANITIZE) +if (NOT OS_ANDROID AND OS_LINUX AND NOT ARCH_S390X AND NOT SANITIZE) # Slightly more efficient code can be generated - # It's disabled for ARM because otherwise ClickHouse cannot run on Android. + # Disabled for Android, because otherwise ClickHouse cannot run on Android. set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -fno-pie") set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -fno-pie") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -no-pie -Wl,-no-pie") +else () + message (WARNING "ClickHouse is built as PIE, system.trace_log will contain invalid addresses after server restart.") endif () if (ENABLE_TESTS) From 99b18d31db32a077678661bd9ba84fb52ff49333 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 14 Jul 2024 15:30:38 +0200 Subject: [PATCH 310/644] Update gdb to 15.1 (by compiling from sources) Right now there are couple of gdb bugs that makes CI unstable: - https://sourceware.org/bugzilla/show_bug.cgi?id=29185 - https://bugzilla.redhat.com/show_bug.cgi?id=1492496 But ubuntu 22.04 does not have 14+ anywhere, the ~ubuntu-toolchain-r/test contains only gdb 13, so there is no other options except for compiling it from sources. But there also other reasons to update it - optimizations, looks like older gdb versions does not use index fully - 5.6sec vs 56sec: # 15.1 $ time command gdb -batch -ex 'disas main' clickhouse ... real 0m5.692s user 0m29.948s sys 0m1.190s # 12.1 (from ubuntu 22.04) real 0m56.709s user 0m59.307s sys 0m0.585s Also note, that we cannot compile gdb in the fasttest (that contains compiler) since some images does not includes full toolchain, for instance gdb is added in the following images: - test-util -> test-base -> lots of other images (no toolchain) - performance-comparison (no toolchain) - integration-test (no toolchain) - integration-tests-runner (no toolchain) Signed-off-by: Azat Khuzhin --- docker/images.json | 4 +++ docker/packager/binary-builder/Dockerfile | 5 ++++ docker/packager/gdb/Dockerfile | 30 +++++++++++++++++++++++ 3 files changed, 39 insertions(+) create mode 100644 docker/packager/gdb/Dockerfile diff --git a/docker/images.json b/docker/images.json index 716b76ee217..eae4ad30a79 100644 --- a/docker/images.json +++ b/docker/images.json @@ -7,6 +7,10 @@ "name": "clickhouse/cctools", "dependent": [] }, + "docker/packager/gdb": { + "name": "clickhouse/gdb", + "dependent": [] + }, "docker/test/compatibility/centos": { "name": "clickhouse/test-old-centos", "dependent": [] diff --git a/docker/packager/binary-builder/Dockerfile b/docker/packager/binary-builder/Dockerfile index 7d6acdcd856..647ab8758a5 100644 --- a/docker/packager/binary-builder/Dockerfile +++ b/docker/packager/binary-builder/Dockerfile @@ -6,6 +6,11 @@ ENV CXX=clang++-${LLVM_VERSION} # If the cctools is updated, then first build it in the CI, then update here in a different commit COPY --from=clickhouse/cctools:d9e3596e706b /cctools /cctools +# TODO: same for gdb and in other places as well +# +# NOTE: here it will add circular dependency but it will be fixed after [1] +# +# [1]: https://github.com/ClickHouse/ClickHouse/issues/66493 # Rust toolchain and libraries ENV RUSTUP_HOME=/rust/rustup diff --git a/docker/packager/gdb/Dockerfile b/docker/packager/gdb/Dockerfile new file mode 100644 index 00000000000..1b5c3bbcb2e --- /dev/null +++ b/docker/packager/gdb/Dockerfile @@ -0,0 +1,30 @@ +# docker build -t clickhouse/gdb . + +ARG FROM_TAG=latest +FROM clickhouse/fasttest:$FROM_TAG + +ENV CC=clang-${LLVM_VERSION} +ENV CXX=clang++-${LLVM_VERSION} +# ld from binutils is 2.38, which has the following error: +# +# DWARF error: invalid or unhandled FORM value: 0x23 +# +ENV LD=ld.lld-${LLVM_VERSION} + +ARG GDB_VERSION=15.1 + +# gdb dependencies +RUN apt-get update \ + && apt-get install --yes \ + libgmp-dev \ + libmpfr-dev \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* + +RUN wget https://sourceware.org/pub/gdb/releases/gdb-$GDB_VERSION.tar.gz \ + && tar -xvf gdb-$GDB_VERSION.tar.gz \ + && cd gdb-$GDB_VERSION \ + && ./configure --prefix=/usr \ + && make -j $(nproc) \ + && make install \ + && rm -fr gdb-$GDB_VERSION gdb-$GDB_VERSION.tar.gz From 459dd1ff2db90f85d9c1f72329ff2f31117c13a7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 14 Jul 2024 16:35:49 +0200 Subject: [PATCH 311/644] Add a comment of the image name for cctools Signed-off-by: Azat Khuzhin --- docker/packager/cctools/Dockerfile | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/packager/cctools/Dockerfile b/docker/packager/cctools/Dockerfile index d986c6a3c86..56a4a65bcc9 100644 --- a/docker/packager/cctools/Dockerfile +++ b/docker/packager/cctools/Dockerfile @@ -1,3 +1,5 @@ +# docker build -t clickhouse/cctools . + # This is a hack to significantly reduce the build time of the clickhouse/binary-builder # It's based on the assumption that we don't care of the cctools version so much # It event does not depend on the clickhouse/fasttest in the `docker/images.json` From 16fcd5b825b50ba88de8ef42b37d40eac1af5596 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 14 Jul 2024 15:48:12 +0200 Subject: [PATCH 312/644] Remove .gdb-index (do not give any benefit for gdb 15.1) Signed-off-by: Azat Khuzhin --- CMakeLists.txt | 8 -------- 1 file changed, 8 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index f796e6c4616..f3c6b2abb30 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -187,14 +187,6 @@ else () set(NO_WHOLE_ARCHIVE --no-whole-archive) endif () -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") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--gdb-index") - message (STATUS "Adding .gdb-index via --gdb-index linker option.") - endif () -endif() - if (NOT (SANITIZE_COVERAGE OR WITH_COVERAGE) AND (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" From 424f19d6c0d003d2a9e2c03017c7fa7dc9dd9233 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 5 Aug 2024 22:49:27 +0200 Subject: [PATCH 313/644] Move gdb into cctools Signed-off-by: Azat Khuzhin --- docker/images.json | 4 ---- docker/packager/cctools/Dockerfile | 24 ++++++++++++++++++++++++ docker/packager/gdb/Dockerfile | 30 ------------------------------ 3 files changed, 24 insertions(+), 34 deletions(-) delete mode 100644 docker/packager/gdb/Dockerfile diff --git a/docker/images.json b/docker/images.json index eae4ad30a79..716b76ee217 100644 --- a/docker/images.json +++ b/docker/images.json @@ -7,10 +7,6 @@ "name": "clickhouse/cctools", "dependent": [] }, - "docker/packager/gdb": { - "name": "clickhouse/gdb", - "dependent": [] - }, "docker/test/compatibility/centos": { "name": "clickhouse/test-old-centos", "dependent": [] diff --git a/docker/packager/cctools/Dockerfile b/docker/packager/cctools/Dockerfile index 56a4a65bcc9..570a42d42d5 100644 --- a/docker/packager/cctools/Dockerfile +++ b/docker/packager/cctools/Dockerfile @@ -32,5 +32,29 @@ RUN git clone https://github.com/tpoechtrager/cctools-port.git \ && cd ../.. \ && rm -rf cctools-port +# +# GDB +# +# ld from binutils is 2.38, which has the following error: +# +# DWARF error: invalid or unhandled FORM value: 0x23 +# +ENV LD=ld.lld-${LLVM_VERSION} +ARG GDB_VERSION=15.1 +RUN apt-get update \ + && apt-get install --yes \ + libgmp-dev \ + libmpfr-dev \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* +RUN wget https://sourceware.org/pub/gdb/releases/gdb-$GDB_VERSION.tar.gz \ + && tar -xvf gdb-$GDB_VERSION.tar.gz \ + && cd gdb-$GDB_VERSION \ + && ./configure --prefix=/opt/gdb \ + && make -j $(nproc) \ + && make install \ + && rm -fr gdb-$GDB_VERSION gdb-$GDB_VERSION.tar.gz + FROM scratch COPY --from=builder /cctools /cctools +COPY --from=builder /opt/gdb /opt/gdb diff --git a/docker/packager/gdb/Dockerfile b/docker/packager/gdb/Dockerfile deleted file mode 100644 index 1b5c3bbcb2e..00000000000 --- a/docker/packager/gdb/Dockerfile +++ /dev/null @@ -1,30 +0,0 @@ -# docker build -t clickhouse/gdb . - -ARG FROM_TAG=latest -FROM clickhouse/fasttest:$FROM_TAG - -ENV CC=clang-${LLVM_VERSION} -ENV CXX=clang++-${LLVM_VERSION} -# ld from binutils is 2.38, which has the following error: -# -# DWARF error: invalid or unhandled FORM value: 0x23 -# -ENV LD=ld.lld-${LLVM_VERSION} - -ARG GDB_VERSION=15.1 - -# gdb dependencies -RUN apt-get update \ - && apt-get install --yes \ - libgmp-dev \ - libmpfr-dev \ - && apt-get clean \ - && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* - -RUN wget https://sourceware.org/pub/gdb/releases/gdb-$GDB_VERSION.tar.gz \ - && tar -xvf gdb-$GDB_VERSION.tar.gz \ - && cd gdb-$GDB_VERSION \ - && ./configure --prefix=/usr \ - && make -j $(nproc) \ - && make install \ - && rm -fr gdb-$GDB_VERSION gdb-$GDB_VERSION.tar.gz From 815fdc43ac333a75adff646fef073ea591494d13 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 6 Aug 2024 14:36:02 +0000 Subject: [PATCH 314/644] Revert "Merge pull request #67800 from ClickHouse/revert-66510" This reverts commit 45c4a71ccb62bac6728d0e583fd04c0fc4f45a6f, reversing changes made to bb71c1eea8e6019a5a21b6add08c2244764ddea5. --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 21 ++++++++++++------- src/Storages/VirtualColumnUtils.h | 10 ++++++++- ..._with_non_deterministic_function.reference | 2 ++ ..._count_with_non_deterministic_function.sql | 4 ++++ 5 files changed, 30 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/03203_count_with_non_deterministic_function.reference create mode 100644 tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 49888596fbb..ce27ad24e10 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1146,7 +1146,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( auto metadata_snapshot = getInMemoryMetadataPtr(); auto virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, {parts[0]}); - auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag.getOutputs().at(0), nullptr); + auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag.getOutputs().at(0), nullptr, /*allow_non_deterministic_functions=*/ false); if (!filter_dag) return {}; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index ba1f4488005..90c2c7f93c1 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -275,7 +275,8 @@ bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node) static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( const ActionsDAG::Node * node, const Block * allowed_inputs, - ActionsDAG::Nodes & additional_nodes) + ActionsDAG::Nodes & additional_nodes, + bool allow_non_deterministic_functions) { if (node->type == ActionsDAG::ActionType::FUNCTION) { @@ -284,8 +285,14 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( auto & node_copy = additional_nodes.emplace_back(*node); node_copy.children.clear(); for (const auto * child : node->children) - if (const auto * child_copy = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes)) + if (const auto * child_copy = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, allow_non_deterministic_functions)) node_copy.children.push_back(child_copy); + /// Expression like (now_allowed AND allowed) is not allowed if allow_non_deterministic_functions = true. This is important for + /// trivial count optimization, otherwise we can get incorrect results. For example, if the query is + /// SELECT count() FROM table WHERE _partition_id = '0' AND rowNumberInBlock() = 1, we cannot apply + /// trivial count. + else if (!allow_non_deterministic_functions) + return nullptr; if (node_copy.children.empty()) return nullptr; @@ -311,7 +318,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( { auto & node_copy = additional_nodes.emplace_back(*node); for (auto & child : node_copy.children) - if (child = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes); !child) + if (child = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, allow_non_deterministic_functions); !child) return nullptr; return &node_copy; @@ -325,7 +332,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( auto index_hint_dag = index_hint->getActions().clone(); ActionsDAG::NodeRawConstPtrs atoms; for (const auto & output : index_hint_dag.getOutputs()) - if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes)) + if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes, allow_non_deterministic_functions)) atoms.push_back(child_copy); if (!atoms.empty()) @@ -359,13 +366,13 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( return node; } -std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs) +std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool allow_non_deterministic_functions) { if (!predicate) return {}; ActionsDAG::Nodes additional_nodes; - const auto * res = splitFilterNodeForAllowedInputs(predicate, allowed_inputs, additional_nodes); + const auto * res = splitFilterNodeForAllowedInputs(predicate, allowed_inputs, additional_nodes, allow_non_deterministic_functions); if (!res) return {}; @@ -374,7 +381,7 @@ std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context) { - auto dag = splitFilterDagForAllowedInputs(predicate, &block); + auto dag = splitFilterDagForAllowedInputs(predicate, &block, /*allow_non_deterministic_functions=*/ false); if (dag) filterBlockWithExpression(buildFilterExpression(std::move(*dag), context), block); } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index d75dc70ae44..abf46dc23a4 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -41,7 +41,15 @@ void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context); bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node); /// Extract a part of predicate that can be evaluated using only columns from input_names. -std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs); +/// When allow_non_deterministic_functions is true then even if the predicate contains non-deterministic +/// functions, we still allow to extract a part of the predicate, otherwise we return nullptr. +/// allow_non_deterministic_functions must be false when we are going to use the result to filter parts in +/// MergeTreeData::totalRowsByPartitionPredicateImp. For example, if the query is +/// `SELECT count() FROM table WHERE _partition_id = '0' AND rowNumberInBlock() = 1` +/// The predicate will be `_partition_id = '0' AND rowNumberInBlock() = 1`, and `rowNumberInBlock()` is +/// non-deterministic. If we still extract the part `_partition_id = '0'` for filtering parts, then trivial +/// count optimization will be mistakenly applied to the query. +std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool allow_non_deterministic_functions = true); /// Extract from the input stream a set of `name` column values template diff --git a/tests/queries/0_stateless/03203_count_with_non_deterministic_function.reference b/tests/queries/0_stateless/03203_count_with_non_deterministic_function.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/03203_count_with_non_deterministic_function.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql b/tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql new file mode 100644 index 00000000000..bb3269da597 --- /dev/null +++ b/tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql @@ -0,0 +1,4 @@ +CREATE TABLE t (p UInt8, x UInt64) Engine = MergeTree PARTITION BY p ORDER BY x; +INSERT INTO t SELECT 0, number FROM numbers(10) SETTINGS max_block_size = 100; +SELECT count() FROM t WHERE p = 0 AND rowNumberInAllBlocks() = 1 SETTINGS allow_experimental_analyzer = 0; +SELECT count() FROM t WHERE p = 0 AND rowNumberInAllBlocks() = 1 SETTINGS allow_experimental_analyzer = 1; From ea1575f60aa41d62c3d22211d8dfb5e187b2194e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 21:14:04 +0200 Subject: [PATCH 315/644] tests: avoid leaving processes leftovers Previously processes cleanup on i.e. SIGINT simply did not work, because the launcher kills only processes in process group, while tests are launched with start_new_session=True for Popen(), which creates own process group. This is needed for killing process group in case of test timeout. So instead, look at the parent pid, and kill the child process groups. Also add some logging to make it more explicit which processes will be killed. Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 99 +++++++++++++++++++++++++++++++------------ 1 file changed, 71 insertions(+), 28 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 907d773337a..5e70b37e232 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -358,14 +358,78 @@ def clickhouse_execute_json( return rows +def kill_process_group(pgid): + print(f"Killing process group {pgid}") + print(f"Processes in process group {pgid}:") + print( + subprocess.check_output( + f"pgrep --pgroup {pgid} -a", shell=True, stderr=subprocess.STDOUT + ).decode("utf-8"), + end="", + ) + try: + # NOTE: this still may leave some processes, that had been + # created by timeout(1), since it also creates new process + # group. But this should not be a problem with default + # options, since the default time for each test is 10min, + # and this is way more bigger then the timeout for each + # timeout(1) invocation. + # + # But as a workaround we are sending SIGTERM first, and + # only after SIGKILL, that way timeout(1) will have an + # ability to terminate childrens (though not always since + # signals are asynchronous). + os.killpg(pgid, signal.SIGTERM) + # This may not be enough, but this is at least something + # (and anyway it is OK to spend 0.1 second more in case of + # test timeout). + sleep(0.1) + os.killpg(pgid, signal.SIGKILL) + except OSError as e: + if e.errno == ESRCH: + print(f"Got ESRCH while killing {pgid}. Ignoring.") + else: + raise + print(f"Process group {pgid} should be killed") + + +def cleanup_child_processes(pid): + pgid = os.getpgid(os.getpid()) + print(f"Child processes of {pid}:") + print( + subprocess.check_output( + f"pgrep --parent {pid} -a", shell=True, stderr=subprocess.STDOUT + ).decode("utf-8"), + end="", + ) + # Due to start_new_session=True, it is not enough to kill by PGID, we need + # to look at children processes as well. + # But we are hoping that nobody create session in the tests (though it is + # possible via timeout(), but we assuming that they will be killed by + # timeout). + processes = subprocess.check_output( + f"pgrep --parent {pid}", shell=True, stderr=subprocess.STDOUT + ) + processes = processes.decode("utf-8") + processes = processes.strip() + processes = processes.split("\n") + processes = map(lambda x: int(x.strip()), processes) + processes = list(processes) + for child in processes: + child_pgid = os.getpgid(child) + if child_pgid != pgid: + kill_process_group(child_pgid) + + # SIGKILL should not be sent, since this will kill the script itself + os.killpg(pgid, signal.SIGTERM) + + +# send signal to all processes in group to avoid hung check triggering +# (to avoid terminating clickhouse-test itself, the signal should be ignored) def stop_tests(): - # send signal to all processes in group to avoid hung check triggering - # (to avoid terminating clickhouse-test itself, the signal should be ignored) - print("Sending signals") signal.signal(signal.SIGTERM, signal.SIG_IGN) - os.killpg(os.getpgid(os.getpid()), signal.SIGTERM) - signal.signal(signal.SIGTERM, signal.SIG_DFL) - print("Sending signals DONE") + cleanup_child_processes(os.getpid()) + signal.signal(signal.SIGTERM, signal_handler) def get_db_engine(args, database_name): @@ -1258,28 +1322,7 @@ class TestCase: if proc: if proc.returncode is None: - try: - pgid = os.getpgid(proc.pid) - # NOTE: this still may leave some processes, that had been - # created by timeout(1), since it also creates new process - # group. But this should not be a problem with default - # options, since the default time for each test is 10min, - # and this is way more bigger then the timeout for each - # timeout(1) invocation. - # - # But as a workaround we are sending SIGTERM first, and - # only after SIGKILL, that way timeout(1) will have an - # ability to terminate childrens (though not always since - # signals are asynchronous). - os.killpg(pgid, signal.SIGTERM) - # This may not be enough, but this is at least something - # (and anyway it is OK to spend 0.1 second more in case of - # test timeout). - sleep(0.1) - os.killpg(pgid, signal.SIGKILL) - except OSError as e: - if e.errno != ESRCH: - raise + kill_process_group(os.getpgid(proc.pid)) if stderr: description += stderr From f9dcce6da3b9468abf5cdc27915c4c093cd231d7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 21:23:37 +0200 Subject: [PATCH 316/644] tests: omit python stacktace in case of signals/server died It is simply useless and only create output that only distracts. Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 5e70b37e232..10a537f665d 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -3438,11 +3438,12 @@ def parse_args(): class Terminated(KeyboardInterrupt): - pass + def __init__(self, signal): + self.signal = signal -def signal_handler(sig, frame): - raise Terminated(f"Terminated with {sig} signal") +def signal_handler(signal, frame): + raise Terminated(signal) if __name__ == "__main__": @@ -3594,4 +3595,14 @@ if __name__ == "__main__": if args.replace_replicated_with_shared: args.s3_storage = True - main(args) + try: + main(args) + except ServerDied as e: + print(f"{e}", file=sys.stderr) + sys.exit(1) + except Terminated as e: + print(f"Terminated with {e.signal} signal", file=sys.stderr) + sys.exit(128 + e.signal) + except KeyboardInterrupt: + print("Interrupted") + sys.exit(128 + signal.SIGINT) From a478ad24a96b28a5cab77c01e77d5d510cddfabb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 21:26:15 +0200 Subject: [PATCH 317/644] tests: try to catch stacktraces from client in case of test timeouts This is to catch issues like [1]. [1]: https://github.com/ClickHouse/ClickHouse/issues/67736 Signed-off-by: Azat Khuzhin --- src/Common/SignalHandlers.cpp | 1 + tests/clickhouse-test | 20 +++++++++++++++++--- 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index e025e49e0a3..c4358da2453 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -629,6 +629,7 @@ void HandledSignals::setupTerminateHandler() void HandledSignals::setupCommonDeadlySignalHandlers() { /// SIGTSTP is added for debugging purposes. To output a stack trace of any running thread at anytime. + /// NOTE: that it is also used by clickhouse-test wrapper addSignalHandler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP, SIGTRAP}, signalHandler, true); #if defined(SANITIZER) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 10a537f665d..20e0ce0b150 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -368,6 +368,13 @@ def kill_process_group(pgid): end="", ) try: + # Let's try to dump stacktrace in client (useful to catch issues there) + os.killpg(pgid, signal.SIGTSTP) + # Wait some time for clickhouse utilities to gather stacktrace + if RELEASE_BUILD: + sleep(0.5) + else: + sleep(5) # NOTE: this still may leave some processes, that had been # created by timeout(1), since it also creates new process # group. But this should not be a problem with default @@ -380,9 +387,8 @@ def kill_process_group(pgid): # ability to terminate childrens (though not always since # signals are asynchronous). os.killpg(pgid, signal.SIGTERM) - # This may not be enough, but this is at least something - # (and anyway it is OK to spend 0.1 second more in case of - # test timeout). + # We need minimal delay to let processes handle SIGTERM - 0.1 (this may + # not be enough, but at least something) sleep(0.1) os.killpg(pgid, signal.SIGKILL) except OSError as e: @@ -2396,7 +2402,13 @@ class BuildFlags: POLYMORPHIC_PARTS = "polymorphic-parts" +# Release and non-sanitizer build +RELEASE_BUILD = False + + def collect_build_flags(args): + global RELEASE_BUILD + result = [] value = clickhouse_execute( @@ -2421,6 +2433,8 @@ def collect_build_flags(args): elif b"RelWithDebInfo" in value or b"Release" in value: result.append(BuildFlags.RELEASE) + RELEASE_BUILD = result == [BuildFlags.RELEASE] + value = clickhouse_execute( args, "SELECT value FROM system.settings WHERE name = 'allow_deprecated_database_ordinary'", From a6ccf1986936b3cd65dd016d7259e25eb35f35d9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 21:55:03 +0200 Subject: [PATCH 318/644] tests: capture stderr/stdout/debuglog after terminating test It was simply wrong before, but now, with capturing stacktrace that can take sometime it is a must. Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 67 +++++++++++++++++++++---------------------- 1 file changed, 33 insertions(+), 34 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 20e0ce0b150..a8c8b3614c8 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1318,18 +1318,35 @@ class TestCase: return None - def process_result_impl( - self, proc, stdout: str, stderr: str, debug_log: str, total_time: float - ): + def process_result_impl(self, proc, total_time: float): + if proc: + if proc.returncode is None: + kill_process_group(os.getpgid(proc.pid)) + description = "" + debug_log = "" + if os.path.exists(self.testcase_args.debug_log_file): + with open(self.testcase_args.debug_log_file, "rb") as stream: + debug_log += self.testcase_args.debug_log_file + ":\n" + debug_log += str(stream.read(), errors="replace", encoding="utf-8") + debug_log += "\n" + + stdout = "" + if os.path.exists(self.stdout_file): + with open(self.stdout_file, "rb") as stdfd: + stdout = str(stdfd.read(), errors="replace", encoding="utf-8") + + stderr = "" + if os.path.exists(self.stderr_file): + with open(self.stderr_file, "rb") as stdfd: + stderr += str(stdfd.read(), errors="replace", encoding="utf-8") + if debug_log: debug_log = "\n".join(debug_log.splitlines()[:100]) if proc: if proc.returncode is None: - kill_process_group(os.getpgid(proc.pid)) - if stderr: description += stderr if debug_log: @@ -1658,13 +1675,6 @@ class TestCase: # Whether the test timed out will be decided later pass - debug_log = "" - if os.path.exists(self.testcase_args.debug_log_file): - with open(self.testcase_args.debug_log_file, "rb") as stream: - debug_log += self.testcase_args.debug_log_file + ":\n" - debug_log += str(stream.read(), errors="replace", encoding="utf-8") - debug_log += "\n" - total_time = (datetime.now() - start_time).total_seconds() # Normalize randomized database names in stdout, stderr files. @@ -1716,17 +1726,7 @@ class TestCase: "https://localhost:8443/", ) - stdout = "" - if os.path.exists(self.stdout_file): - with open(self.stdout_file, "rb") as stdfd: - stdout = str(stdfd.read(), errors="replace", encoding="utf-8") - - stderr = "" - if os.path.exists(self.stderr_file): - with open(self.stderr_file, "rb") as stdfd: - stderr += str(stdfd.read(), errors="replace", encoding="utf-8") - - return proc, stdout, stderr, debug_log, total_time + return proc, total_time def run(self, args, suite, client_options, server_logs_level): start_time = datetime.now() @@ -1758,14 +1758,14 @@ class TestCase: if not is_valid_utf_8(self.case_file) or ( self.reference_file and not is_valid_utf_8(self.reference_file) ): - proc, stdout, stderr, debug_log, total_time = self.run_single_test( + proc, total_time = self.run_single_test( server_logs_level, client_options ) - result = self.process_result_impl( - proc, stdout, stderr, debug_log, total_time + result = self.process_result_impl(proc, total_time) + result.check_if_need_retry( + args, result.description, result.description, self.runs_count ) - result.check_if_need_retry(args, stdout, stderr, self.runs_count) # to avoid breaking CSV parser result.description = result.description.replace("\0", "") else: @@ -1783,17 +1783,16 @@ class TestCase: ): ( proc, - stdout, - stderr, - debug_log, total_time, ) = self.run_single_test(server_logs_level, client_options) - result = self.process_result_impl( - proc, stdout, stderr, debug_log, total_time - ) + result = self.process_result_impl(proc, total_time) + result.check_if_need_retry( - args, stdout, stderr, self.runs_count + args, + result.description, + result.description, + self.runs_count, ) # to avoid breaking CSV parser result.description = result.description.replace("\0", "") From b76fb165d11a7d39b36b8f4e13355c2488ab9e58 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 22:13:29 +0200 Subject: [PATCH 319/644] tests: fix pylint issue in clickhouse_execute_http() Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a8c8b3614c8..239375d7fec 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -267,7 +267,7 @@ def clickhouse_execute_http( max_http_retries=5, retry_error_codes=False, ): - if args.secure: + if base_args.secure: client = http.client.HTTPSConnection( host=base_args.tcp_host, port=base_args.http_port, timeout=timeout ) From ef7d12db6625b0a4e2cdf7ae5b6eb192e5d773af Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 5 Aug 2024 19:51:30 +0200 Subject: [PATCH 320/644] tests: change the process group earlier to avoid killing self Previously it was possible to have original pgid from the spawned threads, that could lead to killing the caller script and in case of CI it could be init process [1]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/67737/e68c9c8d16f37f6c25739076c9b071ed97952269/stress_test__asan_/stress_test_run_21.txt Repro: $ echo "SELECT '1" > tests/queries/0_stateless/00001_select_1.sql # break the test $ cat /tmp/test.sh ./tests/clickhouse-test 0001_select --test-runs 3 --max-failures-chain 1 --no-random-settings --no-random-merge-tree-settings Before this change: $ /tmp/test.sh Using queries from '/src/ch/worktrees/clickhouse-upstream/tests/queries' directory Connecting to ClickHouse server... OK Connected to server 24.8.1.1 @ bef896ce143ea4e0464c9829de6277ba06cc1a53 mt/rename-without-lock-v2 Running 3 stateless tests (MainProcess). 00001_select_1: [ FAIL ] Reason: return code: 62 Code: 62. DB::Exception: Syntax error: failed at position 8 (''1; '): '1; . Single quoted string is not closed: ''1; '. (SYNTAX_ERROR) , result: stdout: Database: test_hz2zwymr Child processes of 13041: 13042 python3 ./tests/clickhouse-test 0001_select --test-runs 3 --max-failures-chain 1 --no-random-settings --no-random-merge-tree-settings Killing process group 13040 Processes in process group 13040: 13040 -bash 13042 python3 ./tests/clickhouse-test 0001_select --test-runs 3 --max-failures-chain 1 --no-random-settings --no-random-merge-tree-settings [2]+ Stopped /tmp/test.sh [1]$ Process group 13040 should be killed Max failures chain [2]+ Killed /tmp/test.sh After: $ /tmp/test.sh Using queries from '/src/ch/worktrees/clickhouse-upstream/tests/queries' directory Connecting to ClickHouse server... OK Connected to server 24.8.1.1 @ bef896ce143ea4e0464c9829de6277ba06cc1a53 mt/rename-without-lock-v2 Running 3 stateless tests (MainProcess). 00001_select_1: [ FAIL ] Reason: return code: 62 Code: 62. DB::Exception: Syntax error: failed at position 8 (''1; '): '1; . Single quoted string is not closed: ''1; '. (SYNTAX_ERROR) , result: stdout: Database: test_urz6rk5z Child processes of 9782: 9785 python3 ./tests/clickhouse-test 0001_select --test-runs 3 --max-failures-chain 1 --no-random-settings --no-random-merge-tree-settings Max failures chain Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 239375d7fec..dfcef86cf7e 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -3460,16 +3460,17 @@ def signal_handler(signal, frame): if __name__ == "__main__": + # Move to a new process group and kill it at exit so that we don't have any + # infinite tests processes left + # (new process group is required to avoid killing some parent processes) + os.setpgid(0, 0) + stop_time = None exit_code = multiprocessing.Value("i", 0) server_died = multiprocessing.Event() multiprocessing_manager = multiprocessing.Manager() restarted_tests = multiprocessing_manager.list() - # Move to a new process group and kill it at exit so that we don't have any - # infinite tests processes left - # (new process group is required to avoid killing some parent processes) - os.setpgid(0, 0) signal.signal(signal.SIGTERM, signal_handler) signal.signal(signal.SIGINT, signal_handler) signal.signal(signal.SIGHUP, signal_handler) From 8ce23ff1136c2de03348af92a595620eee703d9a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 6 Aug 2024 16:28:17 +0200 Subject: [PATCH 321/644] tests: increase delay co capture client stacktraces for sanitizers build 5 seconds is too small and not enough to print even few frames. [1]: https://s3.amazonaws.com/clickhouse-test-reports/67737/9658be5eea8351655dd3ea77b8c1d4717bac7999/stress_test__ubsan_.html Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index dfcef86cf7e..46d0f9e007e 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -374,7 +374,7 @@ def kill_process_group(pgid): if RELEASE_BUILD: sleep(0.5) else: - sleep(5) + sleep(10) # NOTE: this still may leave some processes, that had been # created by timeout(1), since it also creates new process # group. But this should not be a problem with default From 7c366a040fad26f7380adffb38c990f05f629c6d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 6 Aug 2024 16:33:44 +0200 Subject: [PATCH 322/644] ci: use bash arrays to pass opts to clickhouse-test for stateless/stateful Signed-off-by: Azat Khuzhin --- docker/test/stateful/run.sh | 24 +++++++++++++++++------- docker/test/stateless/run.sh | 20 +++++++++++++++----- 2 files changed, 32 insertions(+), 12 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 8e2f1890f89..1ad1f73395e 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -232,15 +232,25 @@ function run_tests() set +e + TEST_ARGS=( + -j 2 + --testname + --shard + --zookeeper + --check-zookeeper-session + --no-stateless + --hung-check + --print-time + "${ADDITIONAL_OPTIONS[@]}" + "$SKIP_TESTS_OPTION" + ) if [[ -n "$USE_PARALLEL_REPLICAS" ]] && [[ "$USE_PARALLEL_REPLICAS" -eq 1 ]]; then - clickhouse-test --client="clickhouse-client --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 \ - --max_parallel_replicas=100 --cluster_for_parallel_replicas='parallel_replicas'" \ - -j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --no-parallel-replicas --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ - "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt - else - clickhouse-test -j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ - "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt + TEST_ARGS+=( + --client="clickhouse-client --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 --max_parallel_replicas=100 --cluster_for_parallel_replicas='parallel_replicas'" + --no-parallel-replicas + ) fi + clickhouse-test "${TEST_ARGS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt set -e } diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ea32df23af0..bcfc2020696 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -264,11 +264,21 @@ function run_tests() TIMEOUT=$((MAX_RUN_TIME - 800 > 8400 ? 8400 : MAX_RUN_TIME - 800)) START_TIME=${SECONDS} set +e - timeout --preserve-status --signal TERM --kill-after 60m ${TIMEOUT}s \ - clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ - --no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ - | ts '%Y-%m-%d %H:%M:%S' \ - | tee -a test_output/test_result.txt + + TEST_ARGS=( + --testname + --shard + --zookeeper + --check-zookeeper-session + --hung-check + --print-time + --no-drop-if-fail + --test-runs "$NUM_TRIES" + "${ADDITIONAL_OPTIONS[@]}" + ) + timeout --preserve-status --signal TERM --kill-after 60m ${TIMEOUT}s clickhouse-test "${TEST_ARGS[@]}" 2>&1 \ + | ts '%Y-%m-%d %H:%M:%S' \ + | tee -a test_output/test_result.txt set -e DURATION=$((SECONDS - START_TIME)) From 72bd43a309f8e327b7e252a9866dabd2496c26af Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 6 Aug 2024 16:36:05 +0200 Subject: [PATCH 323/644] tests: do not capture client stacktraces in stress tests They are too uncontrollable, and likely will leave some clients [1]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/67737/9658be5eea8351655dd3ea77b8c1d4717bac7999/stress_test__ubsan_.html Signed-off-by: Azat Khuzhin --- docker/test/stateful/run.sh | 1 + docker/test/stateless/run.sh | 1 + tests/clickhouse-test | 28 +++++++++++++++++++++------- 3 files changed, 23 insertions(+), 7 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 1ad1f73395e..3a4f0d97993 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -241,6 +241,7 @@ function run_tests() --no-stateless --hung-check --print-time + --capture-client-stacktrace "${ADDITIONAL_OPTIONS[@]}" "$SKIP_TESTS_OPTION" ) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index bcfc2020696..063195181a8 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -273,6 +273,7 @@ function run_tests() --hung-check --print-time --no-drop-if-fail + --capture-client-stacktrace --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" ) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 46d0f9e007e..88ff6753a8f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -358,7 +358,13 @@ def clickhouse_execute_json( return rows +# Should we capture client's stacktraces via SIGTSTP +CAPTURE_CLIENT_STACKTRACE = False + + def kill_process_group(pgid): + global CAPTURE_CLIENT_STACKTRACE + print(f"Killing process group {pgid}") print(f"Processes in process group {pgid}:") print( @@ -368,13 +374,14 @@ def kill_process_group(pgid): end="", ) try: - # Let's try to dump stacktrace in client (useful to catch issues there) - os.killpg(pgid, signal.SIGTSTP) - # Wait some time for clickhouse utilities to gather stacktrace - if RELEASE_BUILD: - sleep(0.5) - else: - sleep(10) + if CAPTURE_CLIENT_STACKTRACE: + # Let's try to dump stacktrace in client (useful to catch issues there) + os.killpg(pgid, signal.SIGTSTP) + # Wait some time for clickhouse utilities to gather stacktrace + if RELEASE_BUILD: + sleep(0.5) + else: + sleep(10) # NOTE: this still may leave some processes, that had been # created by timeout(1), since it also creates new process # group. But this should not be a problem with default @@ -3446,6 +3453,11 @@ def parse_args(): default="./client.fatal.log", help="Path to file for fatal logs from client", ) + parser.add_argument( + "--capture-client-stacktrace", + action="store_true", + help="Capture stacktraces from clickhouse-client/local on errors", + ) return parser.parse_args() @@ -3488,6 +3500,8 @@ if __name__ == "__main__": ) sys.exit(1) + CAPTURE_CLIENT_STACKTRACE = args.capture_client_stacktrace + # Autodetect the directory with queries if not specified if args.queries is None: args.queries = "queries" From 2074485083e8860aafc36ac7886a54a75e144468 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 6 Aug 2024 14:25:02 +0000 Subject: [PATCH 324/644] Fix partial filtering in `filterBlockWithPredicate` --- src/Storages/MergeTree/MergeTreeData.cpp | 3 +- src/Storages/VirtualColumnUtils.cpp | 86 +++++++++---------- src/Storages/VirtualColumnUtils.h | 16 ++-- ...03217_read_rows_in_system_tables.reference | 10 +++ .../03217_read_rows_in_system_tables.sql | 34 ++++++++ 5 files changed, 97 insertions(+), 52 deletions(-) create mode 100644 tests/queries/0_stateless/03217_read_rows_in_system_tables.reference create mode 100644 tests/queries/0_stateless/03217_read_rows_in_system_tables.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ce27ad24e10..b24d7968b61 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6914,7 +6914,8 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( const auto * predicate = filter_dag->getOutputs().at(0); // Generate valid expressions for filtering - VirtualColumnUtils::filterBlockWithPredicate(predicate, virtual_columns_block, query_context); + VirtualColumnUtils::filterBlockWithPredicate( + predicate, virtual_columns_block, query_context, /*allow_filtering_with_partial_predicate =*/true); rows = virtual_columns_block.rows(); part_name_column = virtual_columns_block.getByName("_part").column; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 90c2c7f93c1..b40378250bb 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -1,51 +1,46 @@ -#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 -#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 "Functions/FunctionsLogical.h" -#include "Functions/IFunction.h" -#include "Functions/IFunctionAdaptors.h" -#include "Functions/indexHint.h" -#include -#include -#include -#include namespace DB @@ -273,10 +268,7 @@ bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node) } static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( - const ActionsDAG::Node * node, - const Block * allowed_inputs, - ActionsDAG::Nodes & additional_nodes, - bool allow_non_deterministic_functions) + const ActionsDAG::Node * node, const Block * allowed_inputs, ActionsDAG::Nodes & additional_nodes, bool allow_partial_result) { if (node->type == ActionsDAG::ActionType::FUNCTION) { @@ -285,13 +277,14 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( auto & node_copy = additional_nodes.emplace_back(*node); node_copy.children.clear(); for (const auto * child : node->children) - if (const auto * child_copy = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, allow_non_deterministic_functions)) + if (const auto * child_copy + = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, allow_partial_result)) node_copy.children.push_back(child_copy); - /// Expression like (now_allowed AND allowed) is not allowed if allow_non_deterministic_functions = true. This is important for + /// Expression like (now_allowed AND allowed) is not allowed if allow_partial_result = true. This is important for /// trivial count optimization, otherwise we can get incorrect results. For example, if the query is /// SELECT count() FROM table WHERE _partition_id = '0' AND rowNumberInBlock() = 1, we cannot apply /// trivial count. - else if (!allow_non_deterministic_functions) + else if (!allow_partial_result) return nullptr; if (node_copy.children.empty()) @@ -300,7 +293,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( if (node_copy.children.size() == 1) { const ActionsDAG::Node * res = node_copy.children.front(); - /// Expression like (not_allowed AND 256) can't be resuced to (and(256)) because AND requires + /// Expression like (not_allowed AND 256) can't be reduced to (and(256)) because AND requires /// at least two arguments; also it can't be reduced to (256) because result type is different. if (!res->result_type->equals(*node->result_type)) { @@ -318,7 +311,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( { auto & node_copy = additional_nodes.emplace_back(*node); for (auto & child : node_copy.children) - if (child = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, allow_non_deterministic_functions); !child) + if (child = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, allow_partial_result); !child) return nullptr; return &node_copy; @@ -332,7 +325,8 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( auto index_hint_dag = index_hint->getActions().clone(); ActionsDAG::NodeRawConstPtrs atoms; for (const auto & output : index_hint_dag.getOutputs()) - if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes, allow_non_deterministic_functions)) + if (const auto * child_copy + = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes, allow_partial_result)) atoms.push_back(child_copy); if (!atoms.empty()) @@ -366,22 +360,24 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( return node; } -std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool allow_non_deterministic_functions) +std::optional +splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool allow_partial_result) { if (!predicate) return {}; ActionsDAG::Nodes additional_nodes; - const auto * res = splitFilterNodeForAllowedInputs(predicate, allowed_inputs, additional_nodes, allow_non_deterministic_functions); + const auto * res = splitFilterNodeForAllowedInputs(predicate, allowed_inputs, additional_nodes, allow_partial_result); if (!res) return {}; return ActionsDAG::cloneSubDAG({res}, true); } -void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context) +void filterBlockWithPredicate( + const ActionsDAG::Node * predicate, Block & block, ContextPtr context, bool allow_filtering_with_partial_predicate) { - auto dag = splitFilterDagForAllowedInputs(predicate, &block, /*allow_non_deterministic_functions=*/ false); + auto dag = splitFilterDagForAllowedInputs(predicate, &block, /*allow_partial_result=*/allow_filtering_with_partial_predicate); if (dag) filterBlockWithExpression(buildFilterExpression(std::move(*dag), context), block); } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index abf46dc23a4..f76cf2cad76 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -26,9 +26,13 @@ namespace VirtualColumnUtils /// /// Otherwise calling filter*() outside applyFilters() will throw "Not-ready Set is passed" /// if there are subqueries. +/// +/// Similar to filterBlockWithExpression(buildFilterExpression(splitFilterDagForAllowedInputs(...)))./// Similar to filterBlockWithQuery, but uses ActionsDAG as a predicate. +/// Basically it is filterBlockWithDAG(splitFilterDagForAllowedInputs). +/// If allow_filtering_with_partial_predicate is true, then the filtering will be done even if some part of the predicate +/// cannot be evaluated using the columns from the block. +void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context, bool allow_filtering_with_partial_predicate = true); -/// Similar to filterBlockWithExpression(buildFilterExpression(splitFilterDagForAllowedInputs(...))). -void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context); /// Just filters block. Block should contain all the required columns. ExpressionActionsPtr buildFilterExpression(ActionsDAG dag, ContextPtr context); @@ -41,15 +45,15 @@ void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context); bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node); /// Extract a part of predicate that can be evaluated using only columns from input_names. -/// When allow_non_deterministic_functions is true then even if the predicate contains non-deterministic -/// functions, we still allow to extract a part of the predicate, otherwise we return nullptr. -/// allow_non_deterministic_functions must be false when we are going to use the result to filter parts in +/// When allow_partial_result is false, then the result will be empty if any part of if cannot be evaluated deterministically +/// on the given inputs. +/// allow_partial_result must be false when we are going to use the result to filter parts in /// MergeTreeData::totalRowsByPartitionPredicateImp. For example, if the query is /// `SELECT count() FROM table WHERE _partition_id = '0' AND rowNumberInBlock() = 1` /// The predicate will be `_partition_id = '0' AND rowNumberInBlock() = 1`, and `rowNumberInBlock()` is /// non-deterministic. If we still extract the part `_partition_id = '0'` for filtering parts, then trivial /// count optimization will be mistakenly applied to the query. -std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool allow_non_deterministic_functions = true); +std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool allow_partial_result = true); /// Extract from the input stream a set of `name` column values template diff --git a/tests/queries/0_stateless/03217_read_rows_in_system_tables.reference b/tests/queries/0_stateless/03217_read_rows_in_system_tables.reference new file mode 100644 index 00000000000..b21ead49b1e --- /dev/null +++ b/tests/queries/0_stateless/03217_read_rows_in_system_tables.reference @@ -0,0 +1,10 @@ +information_schema tables +default test_replica_1 r1 +Expression ((Project names + Projection)) + Aggregating + Expression (Before GROUP BY) + ReadFromMerge + Filter (( + ( + ))) + ReadFromMergeTree (default.test_replica_1) +1 1 +1 1 diff --git a/tests/queries/0_stateless/03217_read_rows_in_system_tables.sql b/tests/queries/0_stateless/03217_read_rows_in_system_tables.sql new file mode 100644 index 00000000000..3bea04ccccf --- /dev/null +++ b/tests/queries/0_stateless/03217_read_rows_in_system_tables.sql @@ -0,0 +1,34 @@ +SELECT database, table FROM system.tables WHERE database = 'information_schema' AND table = 'tables'; + +-- To verify StorageSystemReplicas applies the filter properly +CREATE TABLE test_replica_1(x UInt32) + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217/test_replica', 'r1') + ORDER BY x; +CREATE TABLE test_replica_2(x UInt32) + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217/test_replica', 'r2') + ORDER BY x; + +SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = 'test_replica_1' AND replica_name = 'r1'; + + +-- To verify StorageMerge +CREATE TABLE all_replicas (x UInt32) + ENGINE = Merge(currentDatabase(), 'test_replica_*'); + +INSERT INTO test_replica_1 SELECT number AS x FROM numbers(10); +SYSTEM SYNC REPLICA test_replica_2; +-- If the filter not applied, then the plan will show both replicas +EXPLAIN SELECT _table, count() FROM all_replicas WHERE _table = 'test_replica_1' AND x >= 0 GROUP BY _table; + +SYSTEM FLUSH LOGS; +-- argMin-argMax make the test repeatable + +-- StorageSystemTables +SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 + AND query LIKE '%SELECT database, table FROM system.tables WHERE database = \'information_schema\' AND table = \'tables\';' + AND type = 'QueryFinish'; + +-- StorageSystemReplicas +SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 + AND query LIKE '%SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = \'test_replica_1\' AND replica_name = \'r1\';' + AND type = 'QueryFinish'; From f20cfdb54ea9ee577f9747e1e2d99af2c0e9e250 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 14:49:20 +0000 Subject: [PATCH 325/644] Cosmetics III --- src/Storages/Statistics/Statistics.cpp | 2 +- src/Storages/Statistics/Statistics.h | 6 +++--- src/Storages/Statistics/StatisticsCountMinSketch.cpp | 12 ++++++------ src/Storages/Statistics/StatisticsCountMinSketch.h | 6 +++--- src/Storages/Statistics/StatisticsTDigest.cpp | 10 +++++----- src/Storages/Statistics/StatisticsTDigest.h | 4 ++-- src/Storages/Statistics/StatisticsUniq.cpp | 10 +++++----- src/Storages/Statistics/StatisticsUniq.h | 4 ++-- 8 files changed, 27 insertions(+), 27 deletions(-) diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 771304405a6..e3f9fcc8192 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -224,7 +224,7 @@ MergeTreeStatisticsFactory & MergeTreeStatisticsFactory::instance() return instance; } -void MergeTreeStatisticsFactory::validate(const ColumnStatisticsDescription & stats, DataTypePtr data_type) const +void MergeTreeStatisticsFactory::validate(const ColumnStatisticsDescription & stats, const DataTypePtr & data_type) const { for (const auto & [type, desc] : stats.types_to_desc) { diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 16f0c67eabd..c6a45e68aa6 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -87,10 +87,10 @@ class MergeTreeStatisticsFactory : private boost::noncopyable public: static MergeTreeStatisticsFactory & instance(); - void validate(const ColumnStatisticsDescription & stats, DataTypePtr data_type) const; + void validate(const ColumnStatisticsDescription & stats, const DataTypePtr & data_type) const; - using Validator = std::function; - using Creator = std::function; + using Validator = std::function; + using Creator = std::function; ColumnStatisticsPtr get(const ColumnStatisticsDescription & stats) const; ColumnsStatistics getMany(const ColumnsDescription & columns) const; diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index dce5b39ae56..0dc01f5fcf0 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -25,7 +25,7 @@ extern const int ILLEGAL_STATISTICS; static constexpr auto num_hashes = 7uz; static constexpr auto num_buckets = 2718uz; -StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & description, DataTypePtr data_type_) +StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & description, const DataTypePtr & data_type_) : IStatistics(description) , sketch(num_hashes, num_buckets) , data_type(data_type_) @@ -84,15 +84,15 @@ void StatisticsCountMinSketch::deserialize(ReadBuffer & buf) } -void countMinSketchStatisticsValidator(const SingleStatisticsDescription & /*description*/, DataTypePtr data_type) +void countMinSketchStatisticsValidator(const SingleStatisticsDescription & /*description*/, const DataTypePtr & data_type) { - data_type = removeNullable(data_type); - data_type = removeLowCardinalityAndNullable(data_type); - if (!data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type)) + DataTypePtr inner_data_type = removeNullable(data_type); + inner_data_type = removeLowCardinalityAndNullable(inner_data_type); + if (!inner_data_type->isValueRepresentedByNumber() && !isStringOrFixedString(inner_data_type)) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName()); } -StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & description, DataTypePtr data_type) +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type) { return std::make_shared(description, data_type); } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.h b/src/Storages/Statistics/StatisticsCountMinSketch.h index af01408f2a3..d1de1a3aea5 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.h +++ b/src/Storages/Statistics/StatisticsCountMinSketch.h @@ -14,7 +14,7 @@ namespace DB class StatisticsCountMinSketch : public IStatistics { public: - StatisticsCountMinSketch(const SingleStatisticsDescription & description, DataTypePtr data_type_); + StatisticsCountMinSketch(const SingleStatisticsDescription & description, const DataTypePtr & data_type_); Float64 estimateEqual(const Field & val) const override; @@ -31,8 +31,8 @@ private: }; -void countMinSketchStatisticsValidator(const SingleStatisticsDescription & description, DataTypePtr data_type); -StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & description, DataTypePtr data_type); +void countMinSketchStatisticsValidator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); } diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index 73ab6c84b4e..1cf92fea24b 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -57,15 +57,15 @@ Float64 StatisticsTDigest::estimateEqual(const Field & val) const throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName()); } -void tdigestStatisticsValidator(const SingleStatisticsDescription & /*description*/, DataTypePtr data_type) +void tdigestStatisticsValidator(const SingleStatisticsDescription & /*description*/, const DataTypePtr & data_type) { - data_type = removeNullable(data_type); - data_type = removeLowCardinalityAndNullable(data_type); - if (!data_type->isValueRepresentedByNumber()) + DataTypePtr inner_data_type = removeNullable(data_type); + inner_data_type = removeLowCardinalityAndNullable(inner_data_type); + if (!inner_data_type->isValueRepresentedByNumber()) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName()); } -StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, DataTypePtr /*data_type*/) +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & /*data_type*/) { return std::make_shared(description); } diff --git a/src/Storages/Statistics/StatisticsTDigest.h b/src/Storages/Statistics/StatisticsTDigest.h index 47d6c93f64c..2b37799d07b 100644 --- a/src/Storages/Statistics/StatisticsTDigest.h +++ b/src/Storages/Statistics/StatisticsTDigest.h @@ -23,7 +23,7 @@ private: QuantileTDigest t_digest; }; -void tdigestStatisticsValidator(const SingleStatisticsDescription & description, DataTypePtr data_type); -StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, DataTypePtr data_type); +void tdigestStatisticsValidator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); } diff --git a/src/Storages/Statistics/StatisticsUniq.cpp b/src/Storages/Statistics/StatisticsUniq.cpp index e737f9987a5..07311b5b86d 100644 --- a/src/Storages/Statistics/StatisticsUniq.cpp +++ b/src/Storages/Statistics/StatisticsUniq.cpp @@ -52,15 +52,15 @@ UInt64 StatisticsUniq::estimateCardinality() const return column->getUInt(0); } -void uniqStatisticsValidator(const SingleStatisticsDescription & /*description*/, DataTypePtr data_type) +void uniqStatisticsValidator(const SingleStatisticsDescription & /*description*/, const DataTypePtr & data_type) { - data_type = removeNullable(data_type); - data_type = removeLowCardinalityAndNullable(data_type); - if (!data_type->isValueRepresentedByNumber()) + DataTypePtr inner_data_type = removeNullable(data_type); + inner_data_type = removeLowCardinalityAndNullable(inner_data_type); + if (!inner_data_type->isValueRepresentedByNumber()) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName()); } -StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & description, DataTypePtr data_type) +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type) { return std::make_shared(description, data_type); } diff --git a/src/Storages/Statistics/StatisticsUniq.h b/src/Storages/Statistics/StatisticsUniq.h index 6b511d4f496..1fdcab8bd89 100644 --- a/src/Storages/Statistics/StatisticsUniq.h +++ b/src/Storages/Statistics/StatisticsUniq.h @@ -27,7 +27,7 @@ private: }; -void uniqStatisticsValidator(const SingleStatisticsDescription & description, DataTypePtr data_type); -StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & description, DataTypePtr data_type); +void uniqStatisticsValidator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); } From 72ead6e8432daa1e643a5b0cc8559a4ff4d9efd0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 6 Aug 2024 14:56:42 +0000 Subject: [PATCH 326/644] Cleanup. --- src/Storages/IStorage.h | 6 ++-- src/Storages/MergeTree/MutateTask.cpp | 34 +++++++++---------- ...61_lightweight_delete_projection.reference | 4 +-- .../03161_lightweight_delete_projection.sql | 4 +-- 4 files changed, 23 insertions(+), 25 deletions(-) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index d2cdc5af34f..0477a08b0d2 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -259,12 +259,12 @@ public: /// Return true if there is at least one part containing lightweight deleted mask. virtual bool hasLightweightDeletedMask() const { return false; } - /// Return true if storage has any projection. - virtual bool hasProjection() const { return false; } - /// Return true if storage can execute lightweight delete mutations. virtual bool supportsLightweightDelete() const { return false; } + /// Return true if storage has any projection. + virtual bool hasProjection() const { return false; } + /// Return true if storage can execute 'DELETE FROM' mutations. This is different from lightweight delete /// because those are internally translated into 'ALTER UDPATE' mutations. virtual bool supportsDelete() const { return false; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 8b5829eb058..3d9f49c9a7a 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -659,10 +659,8 @@ static NameSet collectFilesToSkip( const Block & updated_header, const std::set & indices_to_recalc, const String & mrk_extension, - const std::set & projections_to_recalc, - const std::set & stats_to_recalc, - const StorageMetadataPtr & metadata_snapshot, - bool skip_all_projections) + const std::set & projections_to_skip, + const std::set & stats_to_recalc) { NameSet files_to_skip = source_part->getFileNamesWithoutChecksums(); @@ -686,16 +684,8 @@ static NameSet collectFilesToSkip( } } - if (skip_all_projections) - { - for (const auto & projection : metadata_snapshot->getProjections()) - files_to_skip.insert(projection.getDirectoryName()); - } - else - { - for (const auto & projection : projections_to_recalc) - files_to_skip.insert(projection->getDirectoryName()); - } + for (const auto & projection : projections_to_skip) + files_to_skip.insert(projection->getDirectoryName()); for (const auto & stat : stats_to_recalc) files_to_skip.insert(stat->getFileName() + STATS_FILE_SUFFIX); @@ -2325,6 +2315,9 @@ bool MutateTask::prepare() lightweight_mutation_projection_mode == LightweightMutationProjectionMode::DROP || lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW; + std::set projections_to_skip_container; + auto * projections_to_skip = &projections_to_skip_container; + bool should_create_projections = !(lightweight_delete_mode && lightweight_delete_drops_projections); /// Under lightweight delete mode, if option is drop, projections_to_recalc should be empty. if (should_create_projections) @@ -2333,6 +2326,13 @@ bool MutateTask::prepare() ctx->source_part, ctx->metadata_snapshot, ctx->materialized_projections); + + projections_to_skip = &ctx->projections_to_recalc; + } + else + { + for (const auto & projection : ctx->metadata_snapshot->getProjections()) + projections_to_skip->insert(&projection); } ctx->stats_to_recalc = MutationHelpers::getStatisticsToRecalculate(ctx->metadata_snapshot, ctx->materialized_statistics); @@ -2343,10 +2343,8 @@ bool MutateTask::prepare() ctx->updated_header, ctx->indices_to_recalc, ctx->mrk_extension, - ctx->projections_to_recalc, - ctx->stats_to_recalc, - ctx->metadata_snapshot, - !should_create_projections); + *projections_to_skip, + ctx->stats_to_recalc); ctx->files_to_rename = MutationHelpers::collectFilesForRenames( ctx->source_part, diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index eef0c5a41b5..8edf541c2a0 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -40,7 +40,7 @@ all_3_3_0_4 SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1) AND parent_name like 'all_3_3%'; p1 all_3_3_0_4 p2 all_3_3_0_4 wide part @@ -85,6 +85,6 @@ all_3_3_0_4 SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1) AND parent_name like 'all_3_3%'; p1 all_3_3_0_4 p2 all_3_3_0_4 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 618f3ac0cb8..0b05326e2c1 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -67,7 +67,7 @@ WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1) AND parent_name like 'all_3_3%'; -- { echoOff } @@ -136,7 +136,7 @@ WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1) AND parent_name like 'all_3_3%'; -- { echoOff } From df2675fad0d1bcb79c8a2d7edd0c08b1da49a945 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Thu, 18 Jul 2024 12:32:16 +0200 Subject: [PATCH 327/644] [resubmit] add replication lag and recovery time metrics --- src/Databases/DatabaseReplicated.cpp | 45 ++++++++++---- src/Databases/DatabaseReplicated.h | 12 +++- src/Databases/DatabaseReplicatedWorker.cpp | 21 +++++++ src/Databases/DatabaseReplicatedWorker.h | 5 ++ src/Storages/System/StorageSystemClusters.cpp | 49 +++++++++++++-- src/Storages/System/StorageSystemClusters.h | 2 +- .../test_recovery_time_metric/__init__.py | 0 .../configs/config.xml | 41 +++++++++++++ .../test_recovery_time_metric/test.py | 61 +++++++++++++++++++ .../02117_show_create_table_system.reference | 2 + .../03206_replication_lag_metric.reference | 4 ++ .../03206_replication_lag_metric.sql | 11 ++++ 12 files changed, 234 insertions(+), 19 deletions(-) create mode 100644 tests/integration/test_recovery_time_metric/__init__.py create mode 100644 tests/integration/test_recovery_time_metric/configs/config.xml create mode 100644 tests/integration/test_recovery_time_metric/test.py create mode 100644 tests/queries/0_stateless/03206_replication_lag_metric.reference create mode 100644 tests/queries/0_stateless/03206_replication_lag_metric.sql diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index f127ccbc224..213c94d4d94 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -338,9 +339,12 @@ ClusterPtr DatabaseReplicated::getClusterImpl(bool all_groups) const return std::make_shared(getContext()->getSettingsRef(), shards, params); } -std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr & cluster_) const +ReplicasInfo DatabaseReplicated::tryGetReplicasInfo(const ClusterPtr & cluster_) const { Strings paths; + + paths.emplace_back(fs::path(zookeeper_path) / "max_log_ptr"); + const auto & addresses_with_failover = cluster_->getShardsAddresses(); const auto & shards_info = cluster_->getShardsInfo(); for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) @@ -349,31 +353,50 @@ std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr { String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); + paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "log_ptr"); } } try { auto current_zookeeper = getZooKeeper(); - auto res = current_zookeeper->exists(paths); + auto zk_res = current_zookeeper->tryGet(paths); - std::vector statuses; - statuses.resize(paths.size()); + auto max_log_ptr_zk = zk_res[0]; + if (max_log_ptr_zk.error != Coordination::Error::ZOK) + throw Coordination::Exception(max_log_ptr_zk.error); - for (size_t i = 0; i < res.size(); ++i) - if (res[i].error == Coordination::Error::ZOK) - statuses[i] = 1; + UInt32 max_log_ptr = parse(max_log_ptr_zk.data); - return statuses; - } - catch (...) + ReplicasInfo replicas_info; + replicas_info.resize((zk_res.size() - 1) / 2); + + size_t global_replica_index = 0; + for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) + { + for (const auto & replica : addresses_with_failover[shard_index]) + { + auto replica_active = zk_res[2 * global_replica_index + 1]; + auto replica_log_ptr = zk_res[2 * global_replica_index + 2]; + + replicas_info[global_replica_index] = ReplicaInfo{ + .is_active = replica_active.error == Coordination::Error::ZOK, + .replication_lag = replica_log_ptr.error != Coordination::Error::ZNONODE ? std::optional(max_log_ptr - parse(replica_log_ptr.data)) : std::nullopt, + .recovery_time = replica.is_local && ddl_worker ? ddl_worker->getCurrentInitializationDurationMs() : 0, + }; + + ++global_replica_index; + } + } + + return replicas_info; + } catch (...) { tryLogCurrentException(log); return {}; } } - void DatabaseReplicated::fillClusterAuthInfo(String collection_name, const Poco::Util::AbstractConfiguration & config_ref) { const auto & config_prefix = fmt::format("named_collections.{}", collection_name); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 27ab262d1f1..db683be8f36 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include #include @@ -17,6 +19,14 @@ using ZooKeeperPtr = std::shared_ptr; class Cluster; using ClusterPtr = std::shared_ptr; +struct ReplicaInfo +{ + bool is_active; + std::optional replication_lag; + UInt64 recovery_time; +}; +using ReplicasInfo = std::vector; + class DatabaseReplicated : public DatabaseAtomic { public: @@ -84,7 +94,7 @@ public: static void dropReplica(DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica, bool throw_if_noop); - std::vector tryGetAreReplicasActive(const ClusterPtr & cluster_) const; + ReplicasInfo tryGetReplicasInfo(const ClusterPtr & cluster_) const; void renameDatabase(ContextPtr query_context, const String & new_name) override; diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 1ef88dc03bc..4e7408aa96e 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -32,6 +32,12 @@ DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db bool DatabaseReplicatedDDLWorker::initializeMainThread() { + { + std::lock_guard lock(initialization_duration_timer_mutex); + initialization_duration_timer.emplace(); + initialization_duration_timer->start(); + } + while (!stop_flag) { try @@ -69,6 +75,10 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() initializeReplication(); initialized = true; + { + std::lock_guard lock(initialization_duration_timer_mutex); + initialization_duration_timer.reset(); + } return true; } catch (...) @@ -78,6 +88,11 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() } } + { + std::lock_guard lock(initialization_duration_timer_mutex); + initialization_duration_timer.reset(); + } + return false; } @@ -459,4 +474,10 @@ UInt32 DatabaseReplicatedDDLWorker::getLogPointer() const return max_id.load(); } +UInt64 DatabaseReplicatedDDLWorker::getCurrentInitializationDurationMs() const +{ + std::lock_guard lock(initialization_duration_timer_mutex); + return initialization_duration_timer ? initialization_duration_timer->elapsedMilliseconds() : 0; +} + } diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index 41edf2221b8..2309c831839 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -36,6 +36,8 @@ public: DatabaseReplicated * const database, bool committed = false); /// NOLINT UInt32 getLogPointer() const; + + UInt64 getCurrentInitializationDurationMs() const; private: bool initializeMainThread() override; void initializeReplication(); @@ -56,6 +58,9 @@ private: ZooKeeperPtr active_node_holder_zookeeper; /// It will remove "active" node when database is detached zkutil::EphemeralNodeHolderPtr active_node_holder; + + std::optional initialization_duration_timer; + mutable std::mutex initialization_duration_timer_mutex; }; } diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 9c5c07ae49f..db1955c2e99 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -31,6 +32,8 @@ ColumnsDescription StorageSystemClusters::getColumnsDescription() {"database_shard_name", std::make_shared(), "The name of the `Replicated` database shard (for clusters that belong to a `Replicated` database)."}, {"database_replica_name", std::make_shared(), "The name of the `Replicated` database replica (for clusters that belong to a `Replicated` database)."}, {"is_active", std::make_shared(std::make_shared()), "The status of the Replicated database replica (for clusters that belong to a Replicated database): 1 means 'replica is online', 0 means 'replica is offline', NULL means 'unknown'."}, + {"replication_lag", std::make_shared(std::make_shared()), "The replication lag of the `Replicated` database replica (for clusters that belong to a Replicated database)."}, + {"recovery_time", std::make_shared(std::make_shared()), "The recovery time of the `Replicated` database replica (for clusters that belong to a Replicated database), in milliseconds."}, }; description.setAliases({ @@ -67,6 +70,10 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const std const auto & shards_info = cluster->getShardsInfo(); const auto & addresses_with_failover = cluster->getShardsAddresses(); + ReplicasInfo replicas_info; + if (replicated) + replicas_info = replicated->tryGetReplicasInfo(name_and_cluster.second); + size_t replica_idx = 0; for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) { @@ -114,17 +121,47 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const std res_columns[res_index++]->insert(address.database_shard_name); if (columns_mask[src_index++]) res_columns[res_index++]->insert(address.database_replica_name); + if (columns_mask[src_index++]) { - std::vector is_active; - if (replicated) - is_active = replicated->tryGetAreReplicasActive(name_and_cluster.second); - - if (is_active.empty()) + if (replicas_info.empty()) res_columns[res_index++]->insertDefault(); else - res_columns[res_index++]->insert(is_active[replica_idx++]); + { + const auto & replica_info = replicas_info[replica_idx]; + res_columns[res_index++]->insert(replica_info.is_active); + } } + + if (columns_mask[src_index++]) + { + if (replicas_info.empty()) + res_columns[res_index++]->insertDefault(); + else + { + const auto & replica_info = replicas_info[replica_idx]; + if (replica_info.replication_lag != std::nullopt) + res_columns[res_index++]->insert(*replica_info.replication_lag); + else + res_columns[res_index++]->insertDefault(); + } + } + + if (columns_mask[src_index++]) + { + if (replicas_info.empty()) + res_columns[res_index++]->insertDefault(); + else + { + const auto & replica_info = replicas_info[replica_idx]; + if (replica_info.recovery_time != 0) + res_columns[res_index++]->insert(replica_info.recovery_time); + else + res_columns[res_index++]->insertDefault(); + } + } + + ++replica_idx; } } } diff --git a/src/Storages/System/StorageSystemClusters.h b/src/Storages/System/StorageSystemClusters.h index f6adb902f43..a5f6d551ca1 100644 --- a/src/Storages/System/StorageSystemClusters.h +++ b/src/Storages/System/StorageSystemClusters.h @@ -1,10 +1,10 @@ #pragma once +#include #include #include #include - namespace DB { diff --git a/tests/integration/test_recovery_time_metric/__init__.py b/tests/integration/test_recovery_time_metric/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_recovery_time_metric/configs/config.xml b/tests/integration/test_recovery_time_metric/configs/config.xml new file mode 100644 index 00000000000..bad9b1fa9ea --- /dev/null +++ b/tests/integration/test_recovery_time_metric/configs/config.xml @@ -0,0 +1,41 @@ + + 9000 + + + + + + + + + default + + + + + + 2181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + 20000 + + + + 1 + localhost + 9444 + + + + + + + localhost + 2181 + + 20000 + + + diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py new file mode 100644 index 00000000000..6fcf2fad423 --- /dev/null +++ b/tests/integration/test_recovery_time_metric/test.py @@ -0,0 +1,61 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=["configs/config.xml"], + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_recovery_time_metric(start_cluster): + node.query( + """ + DROP DATABASE IF EXISTS rdb; + CREATE DATABASE rdb + ENGINE = Replicated('/test/test_recovery_time_metric', 'shard1', 'replica1') + """ + ) + + node.query( + """ + DROP TABLE IF EXISTS rdb.t; + CREATE TABLE rdb.t + ( + `x` UInt32 + ) + ENGINE = MergeTree + ORDER BY x + """ + ) + + node.exec_in_container(["bash", "-c", "rm /var/lib/clickhouse/metadata/rdb/t.sql"]) + + node.restart_clickhouse() + + ret = int( + node.query( + """ + SELECT recovery_time + FROM system.clusters + WHERE cluster = 'rdb' + """ + ).strip() + ) + assert ret > 0 + + node.query( + """ + DROP DATABASE rdb + """ + ) 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 cfae4fee6c2..32e8b2f4312 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -52,6 +52,8 @@ CREATE TABLE system.clusters `database_shard_name` String, `database_replica_name` String, `is_active` Nullable(UInt8), + `replication_lag` Nullable(UInt32), + `recovery_time` Nullable(UInt64), `name` String ALIAS cluster ) ENGINE = SystemClusters diff --git a/tests/queries/0_stateless/03206_replication_lag_metric.reference b/tests/queries/0_stateless/03206_replication_lag_metric.reference new file mode 100644 index 00000000000..02f4a7264b1 --- /dev/null +++ b/tests/queries/0_stateless/03206_replication_lag_metric.reference @@ -0,0 +1,4 @@ +0 +2 +0 +2 diff --git a/tests/queries/0_stateless/03206_replication_lag_metric.sql b/tests/queries/0_stateless/03206_replication_lag_metric.sql new file mode 100644 index 00000000000..998c332a11c --- /dev/null +++ b/tests/queries/0_stateless/03206_replication_lag_metric.sql @@ -0,0 +1,11 @@ +-- Tags: no-parallel + +CREATE DATABASE rdb1 ENGINE = Replicated('/test/test_replication_lag_metric', 'shard1', 'replica1'); +CREATE DATABASE rdb2 ENGINE = Replicated('/test/test_replication_lag_metric', 'shard1', 'replica2'); + +SET distributed_ddl_task_timeout = 0; +CREATE TABLE rdb1.t (id UInt32) ENGINE = ReplicatedMergeTree ORDER BY id; +SELECT replication_lag FROM system.clusters WHERE cluster IN ('rdb1', 'rdb2') ORDER BY cluster ASC, replica_num ASC; + +DROP DATABASE rdb1; +DROP DATABASE rdb2; From 86e3b35f24a449b63172015a0f768434f9f203c6 Mon Sep 17 00:00:00 2001 From: Jacob Reckhard Date: Tue, 6 Aug 2024 09:07:17 -0600 Subject: [PATCH 328/644] spelling fixes --- src/DataTypes/DataTypeCustomGeo.cpp | 2 +- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp index d72787647c3..f90788ec403 100644 --- a/src/DataTypes/DataTypeCustomGeo.cpp +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -24,7 +24,7 @@ void registerDataTypeDomainGeo(DataTypeFactory & factory) std::make_unique(std::make_unique())); }); - // Custom type for mulitple lines stored as Array(LineString) + // Custom type for multiple lines stored as Array(LineString) factory.registerSimpleDataTypeCustom("MultiLineString", [] { return std::make_pair(DataTypeFactory::instance().get("Array(LineString)"), diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 3d7e77f213d..71f5efca893 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2086,6 +2086,7 @@ multiSearchFirstPositionUTF multibyte multidirectory multiline +multilinestring multiplyDecimal multipolygon multisearchany From 5390d1b108956907bf4b038a56fdb2ed8e584308 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 6 Aug 2024 16:12:44 +0200 Subject: [PATCH 329/644] docker: use self-compiled gdb 15 from cctools Signed-off-by: Azat Khuzhin --- docker/packager/binary-builder/Dockerfile | 5 ----- docker/test/fasttest/Dockerfile | 2 +- docker/test/integration/base/Dockerfile | 4 +++- docker/test/integration/runner/Dockerfile | 3 ++- docker/test/performance-comparison/Dockerfile | 4 +++- docker/test/util/Dockerfile | 4 +++- 6 files changed, 12 insertions(+), 10 deletions(-) diff --git a/docker/packager/binary-builder/Dockerfile b/docker/packager/binary-builder/Dockerfile index 647ab8758a5..7d6acdcd856 100644 --- a/docker/packager/binary-builder/Dockerfile +++ b/docker/packager/binary-builder/Dockerfile @@ -6,11 +6,6 @@ ENV CXX=clang++-${LLVM_VERSION} # If the cctools is updated, then first build it in the CI, then update here in a different commit COPY --from=clickhouse/cctools:d9e3596e706b /cctools /cctools -# TODO: same for gdb and in other places as well -# -# NOTE: here it will add circular dependency but it will be fixed after [1] -# -# [1]: https://github.com/ClickHouse/ClickHouse/issues/66493 # Rust toolchain and libraries ENV RUSTUP_HOME=/rust/rustup diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 4cac2ee6135..5d311c673a4 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -83,7 +83,7 @@ RUN arch=${TARGETARCH:-amd64} \ # Give suid to gdb to grant it attach permissions # chmod 777 to make the container user independent -RUN chmod u+s /usr/bin/gdb \ +RUN chmod u+s /opt/gdb/bin/gdb \ && mkdir -p /var/lib/clickhouse \ && chmod 777 /var/lib/clickhouse diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 469251f648c..dc4d470a262 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -11,7 +11,6 @@ RUN apt-get update \ curl \ default-jre \ g++ \ - gdb \ iproute2 \ krb5-user \ libicu-dev \ @@ -73,3 +72,6 @@ maxClientCnxns=80' > /opt/zookeeper/conf/zoo.cfg && \ ENV TZ=Etc/UTC RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone + +COPY --from=clickhouse/cctools:0d6b90a7a490 /opt/gdb /opt/gdb +ENV PATH="/opt/gdb/bin:${PATH}" diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index d250b746e7d..d62009f1be3 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -30,7 +30,6 @@ RUN apt-get update \ luajit \ libssl-dev \ libcurl4-openssl-dev \ - gdb \ default-jdk \ software-properties-common \ libkrb5-dev \ @@ -87,6 +86,8 @@ COPY modprobe.sh /usr/local/bin/modprobe COPY dockerd-entrypoint.sh /usr/local/bin/ COPY misc/ /misc/ +COPY --from=clickhouse/cctools:0d6b90a7a490 /opt/gdb /opt/gdb +ENV PATH="/opt/gdb/bin:${PATH}" # Same options as in test/base/Dockerfile # (in case you need to override them in tests) diff --git a/docker/test/performance-comparison/Dockerfile b/docker/test/performance-comparison/Dockerfile index c68a39f6f70..f7139275282 100644 --- a/docker/test/performance-comparison/Dockerfile +++ b/docker/test/performance-comparison/Dockerfile @@ -9,7 +9,6 @@ RUN apt-get update \ curl \ dmidecode \ g++ \ - gdb \ git \ gnuplot \ imagemagick \ @@ -42,6 +41,9 @@ RUN pip3 --no-cache-dir install -r requirements.txt COPY run.sh / +COPY --from=clickhouse/cctools:0d6b90a7a490 /opt/gdb /opt/gdb +ENV PATH="/opt/gdb/bin:${PATH}" + CMD ["bash", "/run.sh"] # docker run --network=host --volume :/workspace --volume=:/output -e PR_TO_TEST=<> -e SHA_TO_TEST=<> clickhouse/performance-comparison diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index dc928ba7195..8b949ed95db 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -44,7 +44,6 @@ RUN apt-get update \ bash \ bsdmainutils \ build-essential \ - gdb \ git \ gperf \ moreutils \ @@ -58,3 +57,6 @@ RUN apt-get update \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* COPY process_functional_tests_result.py / + +COPY --from=clickhouse/cctools:0d6b90a7a490 /opt/gdb /opt/gdb +ENV PATH="/opt/gdb/bin:${PATH}" From 54ba7703b1ccd116dceefe8b0e9c6aca5c24d212 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 15:16:43 +0000 Subject: [PATCH 330/644] Fix #67742 --- .../ConditionSelectivityEstimator.cpp | 38 ++++++---------- .../ConditionSelectivityEstimator.h | 10 ++--- src/Storages/Statistics/Statistics.cpp | 40 ++--------------- src/Storages/Statistics/Statistics.h | 7 --- src/Storages/Statistics/StatisticsTDigest.cpp | 43 ++++++++++--------- src/Storages/Statistics/StatisticsTDigest.h | 3 +- .../02864_statistics_bugs.reference | 1 + .../0_stateless/02864_statistics_bugs.sql | 9 ++++ 8 files changed, 54 insertions(+), 97 deletions(-) create mode 100644 tests/queries/0_stateless/02864_statistics_bugs.reference create mode 100644 tests/queries/0_stateless/02864_statistics_bugs.sql diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp index 57dff958b9a..432659f51f8 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp @@ -19,7 +19,7 @@ void ConditionSelectivityEstimator::ColumnSelectivityEstimator::merge(String par Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateLess(const Field & val, Float64 rows) const { if (part_statistics.empty()) - return default_normal_cond_factor * rows; + return default_cond_range_factor * rows; Float64 result = 0; Float64 part_rows = 0; for (const auto & [key, estimator] : part_statistics) @@ -39,13 +39,7 @@ Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateEqual { if (part_statistics.empty()) { - auto float_val = StatisticsUtils::tryConvertToFloat64(val); - if (!float_val) - return default_unknown_cond_factor * rows; - else if (float_val.value() < - threshold || float_val.value() > threshold) - return default_normal_cond_factor * rows; - else - return default_good_cond_factor * rows; + return default_cond_equal_factor * rows; } Float64 result = 0; Float64 partial_cnt = 0; @@ -149,30 +143,22 @@ Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode auto [op, val] = extractBinaryOp(node, col); + if (dummy) + { + if (op == "equals") + return default_cond_equal_factor * total_rows; + else if (op == "less" || op == "lessOrEquals" || op == "greater" || op == "greaterOrEquals") + return default_cond_range_factor * total_rows; + else + return default_unknown_cond_factor * total_rows; + } + if (op == "equals") - { - if (dummy) - { - auto float_val = StatisticsUtils::tryConvertToFloat64(val); - if (!float_val || (float_val < - threshold || float_val > threshold)) - return default_normal_cond_factor * total_rows; - else - return default_good_cond_factor * total_rows; - } return estimator.estimateEqual(val, total_rows); - } else if (op == "less" || op == "lessOrEquals") - { - if (dummy) - return default_normal_cond_factor * total_rows; return estimator.estimateLess(val, total_rows); - } else if (op == "greater" || op == "greaterOrEquals") - { - if (dummy) - return default_normal_cond_factor * total_rows; return estimator.estimateGreater(val, total_rows); - } else return default_unknown_cond_factor * total_rows; } diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.h b/src/Storages/Statistics/ConditionSelectivityEstimator.h index ce7fdd12e92..269ee9ac6cb 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.h +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.h @@ -38,12 +38,10 @@ private: std::pair extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name) const; - static constexpr auto default_good_cond_factor = 0.1; - static constexpr auto default_normal_cond_factor = 0.5; - static constexpr auto default_unknown_cond_factor = 1.0; - /// Conditions like "x = N" are considered good if abs(N) > threshold. - /// This is used to assume that condition is likely to have good selectivity. - static constexpr auto threshold = 2; + /// Used to estimate the selectivity of a condition when there is no statistics. + static constexpr auto default_cond_range_factor = 0.5; + static constexpr auto default_cond_equal_factor = 0.01; + static constexpr auto default_unknown_cond_factor = 1; UInt64 total_rows = 0; std::map column_estimators; diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index e3f9fcc8192..52eec437ac2 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -27,36 +27,6 @@ enum StatisticsFileVersion : UInt16 V0 = 0, }; -std::optional StatisticsUtils::tryConvertToFloat64(const Field & field) -{ - switch (field.getType()) - { - case Field::Types::Int64: - return field.get(); - case Field::Types::UInt64: - return field.get(); - case Field::Types::Float64: - return field.get(); - case Field::Types::Int128: - return field.get(); - case Field::Types::UInt128: - return field.get(); - case Field::Types::Int256: - return field.get(); - case Field::Types::UInt256: - return field.get(); - default: - return {}; - } -} - -std::optional StatisticsUtils::tryConvertToString(const DB::Field & field) -{ - if (field.getType() == Field::Types::String) - return field.get(); - return {}; -} - IStatistics::IStatistics(const SingleStatisticsDescription & stat_) : stat(stat_) { @@ -105,7 +75,7 @@ Float64 ColumnStatistics::estimateLess(const Field & val) const { if (stats.contains(StatisticsType::TDigest)) return stats.at(StatisticsType::TDigest)->estimateLess(val); - return rows * ConditionSelectivityEstimator::default_normal_cond_factor; + return rows * ConditionSelectivityEstimator::default_cond_range_factor; } Float64 ColumnStatistics::estimateGreater(const Field & val) const @@ -115,8 +85,7 @@ Float64 ColumnStatistics::estimateGreater(const Field & val) const Float64 ColumnStatistics::estimateEqual(const Field & val) const { - auto float_val = StatisticsUtils::tryConvertToFloat64(val); - if (float_val.has_value() && stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::TDigest)) + if (stats_desc.data_type->isValueRepresentedByNumber() && stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::TDigest)) { /// 2048 is the default number of buckets in TDigest. In this case, TDigest stores exactly one value (with many rows) for every bucket. if (stats.at(StatisticsType::Uniq)->estimateCardinality() < 2048) @@ -126,10 +95,7 @@ Float64 ColumnStatistics::estimateEqual(const Field & val) const if (stats.contains(StatisticsType::CountMinSketch)) return stats.at(StatisticsType::CountMinSketch)->estimateEqual(val); #endif - if (!float_val.has_value() && (float_val < - ConditionSelectivityEstimator::threshold || float_val > ConditionSelectivityEstimator::threshold)) - return rows * ConditionSelectivityEstimator::default_normal_cond_factor; - else - return rows * ConditionSelectivityEstimator::default_good_cond_factor; + return rows * ConditionSelectivityEstimator::default_cond_equal_factor; } /// ------------------------------------- diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index c6a45e68aa6..593ac20edb5 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -15,13 +15,6 @@ constexpr auto STATS_FILE_PREFIX = "statistics_"; constexpr auto STATS_FILE_SUFFIX = ".stats"; -struct StatisticsUtils -{ - /// Returns std::nullopt if input Field cannot be converted to a concrete value - static std::optional tryConvertToFloat64(const Field & field); - static std::optional tryConvertToString(const Field & field); -}; - /// Statistics describe properties of the values in the column, /// e.g. how many unique values exist, /// what are the N most frequent values, diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index 1cf92fea24b..b0c4bfda27d 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -1,6 +1,8 @@ #include -#include +#include #include +#include +#include namespace DB { @@ -10,24 +12,21 @@ extern const int ILLEGAL_STATISTICS; extern const int LOGICAL_ERROR; } -StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & description) +StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & description, const DataTypePtr & data_type_) : IStatistics(description) + , data_type(data_type_) { } void StatisticsTDigest::update(const ColumnPtr & column) { - size_t rows = column->size(); - for (size_t row = 0; row < rows; ++row) + for (size_t row = 0; row < column->size(); ++row) { - Field field; - column->get(row, field); - - if (field.isNull()) + if (column->isNullAt(row)) continue; - if (auto field_as_float = StatisticsUtils::tryConvertToFloat64(field)) - t_digest.add(*field_as_float, 1); + auto data = column->getFloat64(row); + t_digest.add(data, 1); } } @@ -43,18 +42,22 @@ void StatisticsTDigest::deserialize(ReadBuffer & buf) Float64 StatisticsTDigest::estimateLess(const Field & val) const { - auto val_as_float = StatisticsUtils::tryConvertToFloat64(val); - if (val_as_float) - return t_digest.getCountLessThan(*val_as_float); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName()); + Field val_converted = convertFieldToType(val, *data_type); + if (val_converted.isNull()) + return 0; + + auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); + return t_digest.getCountLessThan(val_as_float); } Float64 StatisticsTDigest::estimateEqual(const Field & val) const { - auto val_as_float = StatisticsUtils::tryConvertToFloat64(val); - if (val_as_float) - return t_digest.getCountEqual(*val_as_float); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName()); + Field val_converted = convertFieldToType(val, *data_type); + if (val_converted.isNull()) + return 0; + + auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); + return t_digest.getCountEqual(val_as_float); } void tdigestStatisticsValidator(const SingleStatisticsDescription & /*description*/, const DataTypePtr & data_type) @@ -65,9 +68,9 @@ void tdigestStatisticsValidator(const SingleStatisticsDescription & /*descriptio throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName()); } -StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & /*data_type*/) +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type) { - return std::make_shared(description); + return std::make_shared(description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsTDigest.h b/src/Storages/Statistics/StatisticsTDigest.h index 2b37799d07b..5e744fee2ce 100644 --- a/src/Storages/Statistics/StatisticsTDigest.h +++ b/src/Storages/Statistics/StatisticsTDigest.h @@ -9,7 +9,7 @@ namespace DB class StatisticsTDigest : public IStatistics { public: - explicit StatisticsTDigest(const SingleStatisticsDescription & description); + explicit StatisticsTDigest(const SingleStatisticsDescription & description, const DataTypePtr & data_type_); void update(const ColumnPtr & column) override; @@ -21,6 +21,7 @@ public: private: QuantileTDigest t_digest; + DataTypePtr data_type; }; void tdigestStatisticsValidator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); diff --git a/tests/queries/0_stateless/02864_statistics_bugs.reference b/tests/queries/0_stateless/02864_statistics_bugs.reference new file mode 100644 index 00000000000..f599e28b8ab --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_bugs.reference @@ -0,0 +1 @@ +10 diff --git a/tests/queries/0_stateless/02864_statistics_bugs.sql b/tests/queries/0_stateless/02864_statistics_bugs.sql new file mode 100644 index 00000000000..ef1735550e6 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_bugs.sql @@ -0,0 +1,9 @@ +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET mutations_sync = 1; + +DROP TABLE IF EXISTS bug_67742; +CREATE TABLE bug_67742 (a Float64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); +INSERT INTO bug_67742 SELECT number FROM system.numbers LIMIT 10000; +SELECT count(*) FROM bug_67742 WHERE a < '10'; +DROP TABLE bug_67742; From 5c4f4f85036838a7f145139f36949592720a1289 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 6 Aug 2024 18:52:29 +0200 Subject: [PATCH 331/644] do not add to custom disk names --- src/Disks/DiskFomAST.cpp | 88 +++++++++++------------------------- src/Disks/DiskSelector.h | 2 +- src/Disks/StoragePolicy.h | 1 - src/Interpreters/Context.cpp | 5 +- 4 files changed, 29 insertions(+), 67 deletions(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index c8a4f88547f..2a5e7368de9 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -18,7 +18,6 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; extern const int UNKNOWN_DISK; } @@ -27,8 +26,6 @@ std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string { Poco::Util::AbstractConfiguration::Keys disk_settings_keys; config->keys(disk_settings_keys); - - // Check that no settings are defined when disk from the config is referred. if (disk_settings_keys.empty()) throw Exception( @@ -36,74 +33,48 @@ std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string "Disk function has no arguments. Invalid disk description."); if (disk_settings_keys.size() == 1 && disk_settings_keys.front() == "name" && !attach) - { throw Exception( ErrorCodes::BAD_ARGUMENTS, "Disk function `{}` has to have the other arguments which describe the disk. Invalid disk description.", serialization); - } + + auto disk_settings_hash = sipHash128(serialization.data(), serialization.size()); std::string disk_name; if (config->has("name")) { disk_name = config->getString("name"); } - - if (!disk_name.empty()) - { - if (disk_name.starts_with(DiskSelector::CUSTOM_DISK_PREFIX)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Disk name `{}` could not start with `{}`", - disk_name, DiskSelector::CUSTOM_DISK_PREFIX); - - if (auto disk = context->tryGetDisk(disk_name)) - { - /// the disk is defined by config - if (disk->isCustomDisk()) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Disk with name `{}` already exist as a custom disk but the name does not start with `{}`", - disk_name, - DiskSelector::CUSTOM_DISK_PREFIX); - - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk `{}` is already exist. It is impossible to redefine it.", disk_name); - } - } - - auto disk_settings_hash = sipHash128(serialization.data(), serialization.size()); - - std::string custom_disk_name; - if (disk_name.empty()) + else { /// We need a unique name for a created custom disk, but it needs to be the same /// after table is reattached or server is restarted, so take a hash of the disk /// configuration serialized ast as a disk name suffix. - custom_disk_name = toString(DiskSelector::CUSTOM_DISK_PREFIX) + "noname_" + toString(disk_settings_hash); - } - else - { - custom_disk_name = toString(DiskSelector::CUSTOM_DISK_PREFIX) + disk_name; + disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX + toString(disk_settings_hash); } - auto result_disk = context->getOrCreateDisk(custom_disk_name, [&](const DisksMap & disks_map) -> DiskPtr { - auto disk = DiskFactory::instance().create( + + auto disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { + auto result = DiskFactory::instance().create( disk_name, *config, /* config_path */"", context, disks_map, /* attach */attach, /* custom_disk */true); /// Mark that disk can be used without storage policy. - disk->markDiskAsCustom(disk_settings_hash); - return disk; + result->markDiskAsCustom(disk_settings_hash); + return result; }); - if (!result_disk->isCustomDisk()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Disk with name `{}` expected to be custom disk", disk_name); + if (!disk->isCustomDisk()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "The disk `{}` is already exist and described by the config." + " It is impossible to redefine it.", + disk_name); - if (result_disk->getCustomDiskSettings() != disk_settings_hash && !attach) + if (disk->getCustomDiskSettings() != disk_settings_hash && !attach) throw Exception( ErrorCodes::BAD_ARGUMENTS, "The disk `{}` is already configured as a custom disk in another table. It can't be redefined with different settings.", disk_name); - if (!attach && !result_disk->isRemote()) + if (!attach && !disk->isRemote()) { static constexpr auto custom_local_disks_base_dir_in_config = "custom_local_disks_base_directory"; auto disk_path_expected_prefix = context->getConfigRef().getString(custom_local_disks_base_dir_in_config, ""); @@ -114,14 +85,14 @@ std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string "Base path for custom local disks must be defined in config file by `{}`", custom_local_disks_base_dir_in_config); - if (!pathStartsWith(result_disk->getPath(), disk_path_expected_prefix)) + if (!pathStartsWith(disk->getPath(), disk_path_expected_prefix)) throw Exception( ErrorCodes::BAD_ARGUMENTS, "Path of the custom local disk must be inside `{}` directory", disk_path_expected_prefix); } - return custom_disk_name; + return disk_name; } class DiskConfigurationFlattener @@ -168,22 +139,17 @@ std::string DiskFomAST::createCustomDisk(const ASTPtr & disk_function_ast, Conte std::string DiskFomAST::getConfigDefinedDisk(const std::string &disk_name, ContextPtr context) { - if (disk_name.starts_with(DiskSelector::CUSTOM_DISK_PREFIX)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Disk name `{}` could not start with `{}`", - disk_name, DiskSelector::CUSTOM_DISK_PREFIX); - if (auto result = context->tryGetDisk(disk_name)) - return disk_name; + { + if (result->isCustomDisk()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Disk name `{}` is a custom disk that is used in other table." + "That disk could not be used by a reference by other tables. The custom disk should be fully specified with a disk function.", + disk_name); - std::string custom_disk_name = DiskSelector::CUSTOM_DISK_PREFIX + disk_name; - if (auto result = context->tryGetDisk(custom_disk_name)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Disk name `{}` is a custom disk that is used in other table." - "That disk could not be used by a reference. The custom disk should be fully specified with a disk function.", - disk_name); + return disk_name; + } throw Exception(ErrorCodes::UNKNOWN_DISK, "Unknown disk {}", disk_name); } diff --git a/src/Disks/DiskSelector.h b/src/Disks/DiskSelector.h index 0f7424460a2..e6e2c257911 100644 --- a/src/Disks/DiskSelector.h +++ b/src/Disks/DiskSelector.h @@ -20,7 +20,7 @@ using DiskSelectorPtr = std::shared_ptr; class DiskSelector { public: - static constexpr auto CUSTOM_DISK_PREFIX = "__"; + static constexpr auto TMP_INTERNAL_DISK_PREFIX = "__tmp_internal_"; explicit DiskSelector(std::unordered_set skip_types_ = {}) : skip_types(skip_types_) { } DiskSelector(const DiskSelector & from) = default; diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index e23598214b3..ccf2e2071b2 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -120,7 +120,6 @@ class StoragePolicySelector { public: static constexpr auto TMP_STORAGE_POLICY_PREFIX = "__"; - static_assert(std::string_view(DiskSelector::CUSTOM_DISK_PREFIX) == std::string_view(TMP_STORAGE_POLICY_PREFIX)); StoragePolicySelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 0acbef26805..30f77f799e9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4,7 +4,6 @@ #include #include #include -#include "Common/Logger.h" #include #include #include @@ -4432,11 +4431,9 @@ StoragePolicyPtr Context::getStoragePolicy(const String & name) const StoragePolicyPtr Context::getStoragePolicyFromDisk(const String & disk_name) const { - LOG_DEBUG(getLogger("StoragePolicy"), "getStoragePolicyFromDisk disk_name {}", disk_name); - std::lock_guard lock(shared->storage_policies_mutex); - const std::string storage_policy_name = disk_name.starts_with(DiskSelector::CUSTOM_DISK_PREFIX) ? disk_name : StoragePolicySelector::TMP_STORAGE_POLICY_PREFIX + disk_name; + const std::string storage_policy_name = StoragePolicySelector::TMP_STORAGE_POLICY_PREFIX + disk_name; auto storage_policy_selector = getStoragePolicySelector(lock); StoragePolicyPtr storage_policy = storage_policy_selector->tryGet(storage_policy_name); From bf33aabec412aa2729bfd58f3e717c5b8285acb8 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 17 Jun 2024 10:39:10 +0200 Subject: [PATCH 332/644] Add documentation. (cherry picked from commit 083fff6ed6ccff44b678ae3ea6af75501d9359fb) --- docs/en/engines/table-engines/index.md | 1 + .../table-engines/integrations/time-series.md | 295 ++++++++++++++++++ docs/en/interfaces/prometheus.md | 160 ++++++++++ .../settings.md | 42 --- docs/en/operations/settings/settings.md | 11 + .../table-functions/timeSeriesData.md | 28 ++ .../table-functions/timeSeriesMetrics.md | 28 ++ .../table-functions/timeSeriesTags.md | 28 ++ .../aspell-ignore/en/aspell-dict.txt | 6 + 9 files changed, 557 insertions(+), 42 deletions(-) create mode 100644 docs/en/engines/table-engines/integrations/time-series.md create mode 100644 docs/en/interfaces/prometheus.md create mode 100644 docs/en/sql-reference/table-functions/timeSeriesData.md create mode 100644 docs/en/sql-reference/table-functions/timeSeriesMetrics.md create mode 100644 docs/en/sql-reference/table-functions/timeSeriesTags.md diff --git a/docs/en/engines/table-engines/index.md b/docs/en/engines/table-engines/index.md index 5e81eacc937..20c7c511aa9 100644 --- a/docs/en/engines/table-engines/index.md +++ b/docs/en/engines/table-engines/index.md @@ -61,6 +61,7 @@ Engines in the family: - [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md) - [PostgreSQL](../../engines/table-engines/integrations/postgresql.md) - [S3Queue](../../engines/table-engines/integrations/s3queue.md) +- [TimeSeries](../../engines/table-engines/integrations/time-series.md) ### Special Engines {#special-engines} diff --git a/docs/en/engines/table-engines/integrations/time-series.md b/docs/en/engines/table-engines/integrations/time-series.md new file mode 100644 index 00000000000..4830fd61d27 --- /dev/null +++ b/docs/en/engines/table-engines/integrations/time-series.md @@ -0,0 +1,295 @@ +--- +slug: /en/engines/table-engines/special/time_series +sidebar_position: 60 +sidebar_label: TimeSeries +--- + +# TimeSeries Engine [Experimental] + +A table engine storing time series, i.e. a set of values associated with timestamps and tags (or labels): + +``` +metric_name1[tag1=value1, tag2=value2, ...] = {timestamp1: value1, timestamp2: value2, ...} +metric_name2[...] = ... +``` + +:::info +This is an experimental feature that may change in backwards-incompatible ways in the future releases. +Enable usage of the TimeSeries table engine +with [allow_experimental_time_series_table](../../../operations/settings/settings.md#allow-experimental-time-series-table) setting. +Input the command `set allow_experimental_time_series_table = 1`. +::: + +## Syntax {#syntax} + +``` sql +CREATE TABLE name [(columns)] ENGINE=TimeSeries +[SETTINGS var1=value1, ...] +[DATA db.data_table_name | DATA ENGINE data_table_engine(arguments)] +[TAGS db.tags_table_name | TAGS ENGINE tags_table_engine(arguments)] +[METRICS db.metrics_table_name | METRICS ENGINE metrics_table_engine(arguments)] +``` + +## Usage {#usage} + +It's easier to start with everything set by default (it's allowed to create a `TimeSeries` table without specifying a list of columns): + +``` sql +CREATE TABLE my_table ENGINE=TimeSeries +``` + +Then this table can be used with the following protocols (a port must be assigned in the server configuration): +- [prometheus remote-write](../../../interfaces/prometheus.md#remote-write) +- [prometheus remote-read](../../../interfaces/prometheus.md#remote-read) + +## Target tables {#target-tables} + +A `TimeSeries` table doesn't have its own data, everything is stored in its target tables. +This is similar to how a [materialized view](../../../sql-reference/statements/create/view#materialized-view) works, +with the difference that a materialized view has one target table +whereas a `TimeSeries` table has three target tables named [data]{#data-table}, [tags]{#tags-table], and [metrics]{#metrics-table}. + +The target tables can be either specified explicitly in the `CREATE TABLE` query +or the `TimeSeries` table engine can generate inner target tables automatically. + +The target tables are the following: +1. The _data_ table {#data-table} contains time series associated with some identifier. +The _data_ table must have columns: + +| Name | Mandatory? | Default type | Possible types | Description | +|---|---|---|---|---| +| `id` | [x] | `UUID` | any | Identifies a combination of a metric names and tags | +| `timestamp` | [x] | `DateTime64(3)` | `DateTime64(X)` | A time point | +| `value` | [x] | `Float64` | `Float32` or `Float64` | A value associated with the `timestamp` | + +2. The _tags_ table {#tags-table} contains identifiers calculated for each combination of a metric name and tags. +The _tags_ table must have columns: + +| Name | Mandatory? | Default type | Possible types | Description | +|---|---|---|---|---| +| `id` | [x] | `UUID` | any (must match the type of `id` in the [data]{#data-table} table) | An `id` identifies a combination of a metric name and tags. The DEFAULT expression specifies how to calculate such an identifier | +| `metric_name` | [x] | `LowCardinality(String)` | `String` or `LowCardinality(String)` | The name of a metric | +| `` | [ ] | `String` | `String` or `LowCardinality(String)` or `LowCardinality(Nullable(String))` | The value of a specific tag, the tag's name and the name of a corresponding column are specified in the [tags_to_columns](#settings) setting | +| `tags` | [x] | `Map(LowCardinality(String), String)` | `Map(String, String)` or `Map(LowCardinality(String), String)` or `Map(LowCardinality(String), LowCardinality(String))` | Map of tags excluding the tag `__name__` containing the name of a metric and excluding tags with names enumerated in the [tags_to_columns](#settings) setting | +| `all_tags` | [ ] | `Map(String, String)` | `Map(String, String)` or `Map(LowCardinality(String), String)` or `Map(LowCardinality(String), LowCardinality(String))` | Ephemeral column, each row is a map of all the tags excluding only the tag `__name__` containing the name of a metric. The only purpose of that column is to be used while calculating `id` | +| `min_time` | [ ] | `Nullable(DateTime64(3))` | `DateTime64(X)` or `Nullable(DateTime64(X))` | Minimum timestamp of time series with that `id`. The column is created if [store_min_time_and_max_time](#settings) is `true` | +| `max_time` | [ ] | `Nullable(DateTime64(3))` | `DateTime64(X)` or `Nullable(DateTime64(X))` | Maximum timestamp of time series with that `id`. The column is created if [store_min_time_and_max_time](#settings) is `true` | + +3. The _metrics_ table {#metrics-table} contains some information about metrics been collected, the types of those metrics and their descriptions. +The _metrics_ table must have columns: + +| Name | Mandatory? | Default type | Possible types | Description | +|---|---|---|---|---| +| `metric_family_name` | [x] | `String` | `String` or `LowCardinality(String)` | The name of a metric family | +| `type` | [x] | `String` | `String` or `LowCardinality(String)` | The type of a metric family, one of "counter", "gauge", "summary", "stateset", "histogram", "gaugehistogram" | +| `unit` | [x] | `String` | `String` or `LowCardinality(String)` | The unit used in a metric | +| `help` | [x] | `String` | `String` or `LowCardinality(String)` | The description of a metric | + +Any row inserted into a `TimeSeries` table will be in fact stored in those three target tables. +A `TimeSeries` table contains all those columns from the [data]{#data-table}, [tags]{#tags-table}, [metrics]{#metrics-table} tables. + +## Creation {#creation} + +There are multiple ways to create a table with the `TimeSeries` table engine. +The simplest statement + +``` sql +CREATE TABLE my_table ENGINE=TimeSeries +``` + +will actually create the following table (you can see that by executing `SHOW CREATE TABLE my_table`): + +``` sql +CREATE TABLE my_table +( + `id` UUID DEFAULT reinterpretAsUUID(sipHash128(metric_name, all_tags)), + `timestamp` DateTime64(3), + `value` Float64, + `metric_name` LowCardinality(String), + `tags` Map(LowCardinality(String), String), + `all_tags` Map(String, String), + `min_time` Nullable(DateTime64(3)), + `max_time` Nullable(DateTime64(3)), + `metric_family_name` String, + `type` String, + `unit` String, + `help` String +) +ENGINE = TimeSeries +DATA ENGINE = MergeTree ORDER BY (id, timestamp) +DATA INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +TAGS ENGINE = AggregatingMergeTree PRIMARY KEY metric_name ORDER BY (metric_name, id) +TAGS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +METRICS ENGINE = ReplacingMergeTree ORDER BY metric_family_name +METRICS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +``` + +So the columns were generated automatically and also there are three inner UUIDs in this statement - +one per each inner target table that was created. +(Inner UUIDs are not shown normally until setting +[show_table_uuid_in_table_create_query_if_not_nil](../../../operations/settings/settings#show_table_uuid_in_table_create_query_if_not_nil) +is set.) + +Inner target tables have names like `.inner_id.data.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`, +`.inner_id.tags.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`, `.inner_id.metrics.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` +and each target table has columns which is a subset of the columns of the main `TimeSeries` table: + +``` sql +CREATE TABLE default.`.inner_id.data.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` +( + `id` UUID, + `timestamp` DateTime64(3), + `value` Float64 +) +ENGINE = MergeTree +ORDER BY (id, timestamp) +``` + +``` sql +CREATE TABLE default.`.inner_id.tags.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` +( + `id` UUID DEFAULT reinterpretAsUUID(sipHash128(metric_name, all_tags)), + `metric_name` LowCardinality(String), + `tags` Map(LowCardinality(String), String), + `all_tags` Map(String, String) EPHEMERAL, + `min_time` SimpleAggregateFunction(min, Nullable(DateTime64(3))), + `max_time` SimpleAggregateFunction(max, Nullable(DateTime64(3))) +) +ENGINE = AggregatingMergeTree +PRIMARY KEY metric_name +ORDER BY (metric_name, id) +``` + +``` sql +CREATE TABLE default.`.inner_id.metrics.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` +( + `metric_family_name` String, + `type` String, + `unit` String, + `help` String +) +ENGINE = ReplacingMergeTree +ORDER BY metric_family_name +``` + +## Adjusting types of columns {#adjusting-column-types} + +You can adjust the types of almost any column of the inner target tables by specifying them explicitly +while defining the main table. For example, + +``` sql +CREATE TABLE my_table +( + timestamp DateTime64(6) +) ENGINE=TimeSeries +``` + +will make the inner [data]{#data-table} table store timestamp in microseconds instead of milliseconds: + +``` sql +CREATE TABLE default.`.inner_id.data.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` +( + `id` UUID, + `timestamp` DateTime64(6), + `value` Float64 +) +ENGINE = MergeTree +ORDER BY (id, timestamp) +``` + +## The `id` column {#id-column} + +The `id` column contains identifiers, every identifier is calculated for a combination of a metric name and tags. +The DEFAULT expression for the `id` column is an expression which will be used to calculate such identifiers. +Both the type of the `id` column and that expression can be adjusted by specifying them explicitly: + +``` sql +CREATE TABLE my_table +( + id UInt64 DEFAULT sipHash64(metric_name, all_tags) +) ENGINE=TimeSeries +``` + +## The `tags` and `all_tags` columns {#tags-and-all-tags} + +There are two columns containing maps of tags - `tags` and `all_tags`. In this example they mean the same, however they can be different +if setting `tags_to_columns` is used. This setting allows to specify that a specific tag should be stored in a separate column instead of storing +in a map inside the `tags` column: + +``` sql +CREATE TABLE my_table ENGINE=TimeSeries SETTINGS = {'instance': 'instance', 'job': 'job'} +``` + +This statement will add columns +``` + `instance` String, + `job` String +``` +to the definition of both `my_table` and its inner [tags]{#tags-table} target table. In this case the `tags` column will not contain tags `instance` and `job`, +but the `all_tags` column will contain them. The `all_tags` column is ephemeral and its only purpose to be used in the DEFAULT expression +for the `id` column. + +The types of columns can be adjusted by specifying them explicitly: + +``` sql +CREATE TABLE my_table (instance LowCardinality(String), job LowCardinality(Nullable(String))) +ENGINE=TimeSeries SETTINGS = {'instance': 'instance', 'job': 'job'} +``` + +## Table engines of inner target tables {#inner-table-engines} + +By default inner target tables use the following table engines: +- the [data]{#data-table} table uses [MergeTree](../mergetree-family/mergetree); +- the [tags]{#tags-table} table uses [AggregatingMergeTree](../mergetree-family/aggregatingmergetree) because the same data is often inserted multiple times to this table so we need a way +to remove duplicates, and also because it's required to do aggregation for columns `min_time` and `max_time`; +- the [metrics]{#metrics-table} table uses [ReplacingMergeTree](../mergetree-family/replacingmergetree) because the same data is often inserted multiple times to this table so we need a way +to remove duplicates. + +Other table engines also can be used for inner target tables if it's specified so: + +``` sql +CREATE TABLE my_table ENGINE=TimeSeries +DATA ENGINE=ReplicatedMergeTree +TAGS ENGINE=ReplicatedAggregatingMergeTree +METRICS ENGINE=ReplicatedReplacingMergeTree +``` + +## External target tables {#external-target-tables} + +It's possible to make a `TimeSeries` table use a manually created table: + +``` sql +CREATE TABLE data_for_my_table +( + `id` UUID, + `timestamp` DateTime64(3), + `value` Float64 +) +ENGINE = MergeTree +ORDER BY (id, timestamp); + +CREATE TABLE tags_for_my_table ... + +CREATE TABLE metrics_for_my_table ... + +CREATE TABLE my_table ENGINE=TimeSeries DATA data_for_my_table TAGS tags_for_my_table METRICS metrics_for_my_table; +``` + +## Settings {#settings} + +Here is a list of settings which can be specified while defining a `TimeSeries` table: + +| Name | Type | Default | Description | +|---|---|---|---| +| `tags_to_columns` | Map | {} | Map specifying which tags should be put to separate columns in the [tags]{#tags-table} table. Syntax: `{'tag1': 'column1', 'tag2' : column2, ...}` | +| `use_all_tags_column_to_generate_id` | Bool | true | When generating an expression to calculate an identifier of a time series, this flag enables using the `all_tags` column in that calculation | +| `store_min_time_and_max_time` | Bool | true | If set to true then the table will store `min_time` and `max_time` for each time series | +| `aggregate_min_time_and_max_time` | Bool | true | When creating an inner target `tags` table, this flag enables using `SimpleAggregateFunction(min, Nullable(DateTime64(3)))` instead of just `Nullable(DateTime64(3))` as the type of the `min_time` column, and the same for the `max_time` column | +| `filter_by_min_time_and_max_time` | Bool | true | If set to true then the table will use the `min_time` and `max_time` columns for filtering time series | + +# Functions {#functions} + +Here is a list of functions supporting a `TimeSeries` table as an argument: +- [timeSeriesData](../../../sql-reference/table-functions/timeSeriesData.md) +- [timeSeriesTags](../../../sql-reference/table-functions/timeSeriesTags.md) +- [timeSeriesMetrics](../../../sql-reference/table-functions/timeSeriesMetrics.md) diff --git a/docs/en/interfaces/prometheus.md b/docs/en/interfaces/prometheus.md new file mode 100644 index 00000000000..75a68c59219 --- /dev/null +++ b/docs/en/interfaces/prometheus.md @@ -0,0 +1,160 @@ +--- +slug: /en/interfaces/prometheus +sidebar_position: 19 +sidebar_label: Prometheus protocols +--- + +# Prometheus protocols + +## Exposing metrics {#expose} + +:::note +ClickHouse Cloud does not currently support connecting to Prometheus. To be notified when this feature is supported, please contact support@clickhouse.com. +::: + +ClickHouse can expose its own metrics for scraping from Prometheus: + +```xml + + 9363 + /metrics + true + true + true + true + + +Section `` can be used to make more extended handlers. +This section is similar to [](http.md) but works for prometheus protocols: + +```xml + + 9363 + + + /metrics + + expose_metrics + true + true + true + true + + + + +``` + +Settings: + +| Name | Default | Description | +|---|---|---|---| +| `port` | none | Port for serving the exposing metrics protocol. | +| `endpoint` | `/metrics` | HTTP endpoint for scraping metrics by prometheus server. Starts with `/`. Should not be used with the `` section. | +| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](http.md) section. | +| `metrics` | true | Expose metrics from the [system.metrics](../operations/system-tables/metrics.md) table. | +| `asynchronous_metrics` | true | Expose current metrics values from the [system.asynchronous_metrics](../operations/system-tables/asynchronous_metrics.md) table. | +| `events` | true | Expose metrics from the [system.events](../operations/system-tables/events.md) table. | +| `errors` | true | Expose the number of errors by error codes occurred since the last server restart. This information could be obtained from the [system.errors](../operations/system-tables/errors.md) as well. | + +Check (replace `127.0.0.1` with the IP addr or hostname of your ClickHouse server): +```bash +curl 127.0.0.1:9363/metrics +``` + +## Remote-write protocol {#remote-write} + +ClickHouse supports the [remote-write](https://prometheus.io/docs/specs/remote_write_spec/) protocol. +Data are received by this protocol and written to a [TimeSeries](../engines/table-engines/integrations/time-series.md) table +(which should be created beforehand). + +```xml + + 9363 + + + /write + + remote_writedb_name + time_series_table
+
+
+
+
+``` + +Settings: + +| Name | Default | Description | +|---|---|---|---| +| `port` | none | Port for serving the `remote-write` protocol. | +| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](http.md) section. | +| `table` | none | The name of a [TimeSeries](../engines/table-engines/integrations/time-series.md) table to write data received by the `remote-write` protocol. This name can optionally contain the name of a database too. | +| `database` | none | The name of a database where the table specified in the `table` setting is located if it's not specified in the `table` setting. | + +## Remote-read protocol {#remote-read} + +ClickHouse supports the [remote-read](https://prometheus.io/docs/prometheus/latest/querying/remote_read_api/) protocol. +Data are read from a [TimeSeries](../engines/table-engines/integrations/time-series.md) table and sent via this protocol. + +```xml + + 9363 + + + /read + + remote_readdb_name + time_series_table
+
+
+
+
+``` + +Settings: + +| Name | Default | Description | +|---|---|---|---| +| `port` | none | Port for serving the `remote-read` protocol. | +| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](http.md) section. | +| `table` | none | The name of a [TimeSeries](../engines/table-engines/integrations/time-series.md) table to read data to send by the `remote-read` protocol. This name can optionally contain the name of a database too. | +| `database` | none | The name of a database where the table specified in the `table` setting is located if it's not specified in the `table` setting. | + +## Configuration for multiple protocols {#multiple-protocols} + +Multiple protocols can be specified together in one place: + +```xml + + 9363 + + + /metrics + + expose_metrics + true + true + true + true + + + + /write + + remote_writedb_name.time_series_table + + + + /read + + remote_readdb_name.time_series_table + + + + +``` diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index a1e3c292b04..68f61650e00 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2112,48 +2112,6 @@ The trailing slash is mandatory. /var/lib/clickhouse/ ``` -## Prometheus {#prometheus} - -:::note -ClickHouse Cloud does not currently support connecting to Prometheus. To be notified when this feature is supported, please contact support@clickhouse.com. -::: - -Exposing metrics data for scraping from [Prometheus](https://prometheus.io). - -Settings: - -- `endpoint` – HTTP endpoint for scraping metrics by prometheus server. Start from ‘/’. -- `port` – Port for `endpoint`. -- `metrics` – Expose metrics from the [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) table. -- `events` – Expose metrics from the [system.events](../../operations/system-tables/events.md#system_tables-events) table. -- `asynchronous_metrics` – Expose current metrics values from the [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) table. -- `errors` - Expose the number of errors by error codes occurred since the last server restart. This information could be obtained from the [system.errors](../../operations/system-tables/asynchronous_metrics.md#system_tables-errors) as well. - -**Example** - -``` xml - - 0.0.0.0 - 8123 - 9000 - - - /metrics - 9363 - true - true - true - true - - - -``` - -Check (replace `127.0.0.1` with the IP addr or hostname of your ClickHouse server): -```bash -curl 127.0.0.1:9363/metrics -``` - ## query_log {#query-log} Setting for logging queries received with the [log_queries=1](../../operations/settings/settings.md) setting. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 35547c3a9a6..feac12f9c99 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5614,3 +5614,14 @@ Default value: `1GiB`. Disable all insert and mutations (alter table update / alter table delete / alter table drop partition). Set to true, can make this node focus on reading queries. Default value: `false`. + +## allow_experimental_time_series_table {#allow-experimental-time-series-table} + +Allows creation of tables with the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine. + +Possible values: + +- 0 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is disabled. +- 1 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is enabled. + +Default value: `0`. diff --git a/docs/en/sql-reference/table-functions/timeSeriesData.md b/docs/en/sql-reference/table-functions/timeSeriesData.md new file mode 100644 index 00000000000..aa7a9d30c2a --- /dev/null +++ b/docs/en/sql-reference/table-functions/timeSeriesData.md @@ -0,0 +1,28 @@ +--- +slug: /en/sql-reference/table-functions/timeSeriesData +sidebar_position: 145 +sidebar_label: timeSeriesData +--- + +# timeSeriesData + +`timeSeriesData(db_name.time_series_table)` - Returns the [data](../../engines/table-engines/integrations/time-series.md#data-table) table +used by table `db_name.time_series_table` which table engine is [TimeSeries](../../engines/table-engines/integrations/time-series.md): + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries DATA data_table +``` + +The function also works if the _data_ table is inner: + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries DATA INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +``` + +The following queries are equivalent: + +``` sql +SELECT * FROM timeSeriesData(db_name.time_series_table); +SELECT * FROM timeSeriesData('db_name.time_series_table'); +SELECT * FROM timeSeriesData('db_name', 'time_series_table'); +``` diff --git a/docs/en/sql-reference/table-functions/timeSeriesMetrics.md b/docs/en/sql-reference/table-functions/timeSeriesMetrics.md new file mode 100644 index 00000000000..913f1185bca --- /dev/null +++ b/docs/en/sql-reference/table-functions/timeSeriesMetrics.md @@ -0,0 +1,28 @@ +--- +slug: /en/sql-reference/table-functions/timeSeriesMetrics +sidebar_position: 145 +sidebar_label: timeSeriesMetrics +--- + +# timeSeriesMetrics + +`timeSeriesMetrics(db_name.time_series_table)` - Returns the [metrics](../../engines/table-engines/integrations/time-series.md#metrics-table) table +used by table `db_name.time_series_table` which table engine is [TimeSeries](../../engines/table-engines/integrations/time-series.md): + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries METRICS metrics_table +``` + +The function also works if the _metrics_ table is inner: + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries METRICS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +``` + +The following queries are equivalent: + +``` sql +SELECT * FROM timeSeriesMetrics(db_name.time_series_table); +SELECT * FROM timeSeriesMetrics('db_name.time_series_table'); +SELECT * FROM timeSeriesMetrics('db_name', 'time_series_table'); +``` diff --git a/docs/en/sql-reference/table-functions/timeSeriesTags.md b/docs/en/sql-reference/table-functions/timeSeriesTags.md new file mode 100644 index 00000000000..663a7dc6ac8 --- /dev/null +++ b/docs/en/sql-reference/table-functions/timeSeriesTags.md @@ -0,0 +1,28 @@ +--- +slug: /en/sql-reference/table-functions/timeSeriesTags +sidebar_position: 145 +sidebar_label: timeSeriesTags +--- + +# timeSeriesTags + +`timeSeriesTags(db_name.time_series_table)` - Returns the [tags](../../engines/table-engines/integrations/time-series.md#tags-table) table +used by table `db_name.time_series_table` which table engine is [TimeSeries](../../engines/table-engines/integrations/time-series.md): + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries TAGS tags_table +``` + +The function also works if the _tags_ table is inner: + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries TAGS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +``` + +The following queries are equivalent: + +``` sql +SELECT * FROM timeSeriesTags(db_name.time_series_table); +SELECT * FROM timeSeriesTags('db_name.time_series_table'); +SELECT * FROM timeSeriesTags('db_name', 'time_series_table'); +``` diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 8a9a8d2e76c..382e64f343c 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1672,6 +1672,7 @@ fuzzQuery fuzzer fuzzers gRPC +gaugehistogram gccMurmurHash gcem generateRandom @@ -2556,6 +2557,7 @@ startsWithUTF startswith statbox stateful +stateset stddev stddevPop stddevPopStable @@ -2687,6 +2689,10 @@ themself threadpool throwIf timeDiff +TimeSeries +timeSeriesData +timeSeriesMetrics +timeSeriesTags timeSlot timeSlots timeZone From 60175f80a9bb3e4fea46a2f8dc8d15595d987b85 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 6 Aug 2024 20:19:15 +0200 Subject: [PATCH 333/644] Revert "Add documentation." This reverts commit 083fff6ed6ccff44b678ae3ea6af75501d9359fb. --- docs/en/engines/table-engines/index.md | 1 - .../table-engines/integrations/time-series.md | 299 ------------------ docs/en/interfaces/prometheus.md | 161 ---------- .../settings.md | 42 +++ docs/en/operations/settings/settings.md | 11 - .../table-functions/timeSeriesData.md | 28 -- .../table-functions/timeSeriesMetrics.md | 28 -- .../table-functions/timeSeriesTags.md | 28 -- .../aspell-ignore/en/aspell-dict.txt | 6 - 9 files changed, 42 insertions(+), 562 deletions(-) delete mode 100644 docs/en/engines/table-engines/integrations/time-series.md delete mode 100644 docs/en/interfaces/prometheus.md delete mode 100644 docs/en/sql-reference/table-functions/timeSeriesData.md delete mode 100644 docs/en/sql-reference/table-functions/timeSeriesMetrics.md delete mode 100644 docs/en/sql-reference/table-functions/timeSeriesTags.md diff --git a/docs/en/engines/table-engines/index.md b/docs/en/engines/table-engines/index.md index 20c7c511aa9..5e81eacc937 100644 --- a/docs/en/engines/table-engines/index.md +++ b/docs/en/engines/table-engines/index.md @@ -61,7 +61,6 @@ Engines in the family: - [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md) - [PostgreSQL](../../engines/table-engines/integrations/postgresql.md) - [S3Queue](../../engines/table-engines/integrations/s3queue.md) -- [TimeSeries](../../engines/table-engines/integrations/time-series.md) ### Special Engines {#special-engines} diff --git a/docs/en/engines/table-engines/integrations/time-series.md b/docs/en/engines/table-engines/integrations/time-series.md deleted file mode 100644 index 2914bf4bf37..00000000000 --- a/docs/en/engines/table-engines/integrations/time-series.md +++ /dev/null @@ -1,299 +0,0 @@ ---- -slug: /en/engines/table-engines/special/time_series -sidebar_position: 60 -sidebar_label: TimeSeries ---- - -# TimeSeries Engine [Experimental] - -A table engine storing time series, i.e. a set of values associated with timestamps and tags (or labels): - -```text -metric_name1[tag1=value1, tag2=value2, ...] = {timestamp1: value1, timestamp2: value2, ...} -metric_name2[...] = ... -``` - -:::info -This is an experimental feature that may change in backwards-incompatible ways in the future releases. -Enable usage of the TimeSeries table engine -with [allow_experimental_time_series_table](../../../operations/settings/settings.md#allow-experimental-time-series-table) setting. -Input the command `set allow_experimental_time_series_table = 1`. -::: - -## Syntax {#syntax} - -``` sql -CREATE TABLE name [(columns)] ENGINE=TimeSeries -[SETTINGS var1=value1, ...] -[DATA db.data_table_name | DATA ENGINE data_table_engine(arguments)] -[TAGS db.tags_table_name | TAGS ENGINE tags_table_engine(arguments)] -[METRICS db.metrics_table_name | METRICS ENGINE metrics_table_engine(arguments)] -``` - -## Usage {#usage} - -It's easier to start with everything set by default (it's allowed to create a `TimeSeries` table without specifying a list of columns): - -``` sql -CREATE TABLE my_table ENGINE=TimeSeries -``` - -Then this table can be used with the following protocols (a port must be assigned in the server configuration): - -- [prometheus remote-write](../../../interfaces/prometheus.md#remote-write) -- [prometheus remote-read](../../../interfaces/prometheus.md#remote-read) - -## Target tables {#target-tables} - -A `TimeSeries` table doesn't have its own data, everything is stored in its target tables. -This is similar to how a [materialized view](../../../sql-reference/statements/create/view#materialized-view) works, -with the difference that a materialized view has one target table -whereas a `TimeSeries` table has three target tables named [data]{#data-table}, [tags]{#tags-table], and [metrics]{#metrics-table}. - -The target tables can be either specified explicitly in the `CREATE TABLE` query -or the `TimeSeries` table engine can generate inner target tables automatically. - -The target tables are the following: - -1. The _data_ table {#data-table} contains time series associated with some identifier. -The _data_ table must have columns: - -| Name | Mandatory? | Default type | Possible types | Description | -|---|---|---|---|---| -| `id` | [x] | `UUID` | any | Identifies a combination of a metric names and tags | -| `timestamp` | [x] | `DateTime64(3)` | `DateTime64(X)` | A time point | -| `value` | [x] | `Float64` | `Float32` or `Float64` | A value associated with the `timestamp` | - -2. The _tags_ table {#tags-table} contains identifiers calculated for each combination of a metric name and tags. -The _tags_ table must have columns: - -| Name | Mandatory? | Default type | Possible types | Description | -|---|---|---|---|---| -| `id` | [x] | `UUID` | any (must match the type of `id` in the [data]{#data-table} table) | An `id` identifies a combination of a metric name and tags. The DEFAULT expression specifies how to calculate such an identifier | -| `metric_name` | [x] | `LowCardinality(String)` | `String` or `LowCardinality(String)` | The name of a metric | -| `` | [ ] | `String` | `String` or `LowCardinality(String)` or `LowCardinality(Nullable(String))` | The value of a specific tag, the tag's name and the name of a corresponding column are specified in the [tags_to_columns](#settings) setting | -| `tags` | [x] | `Map(LowCardinality(String), String)` | `Map(String, String)` or `Map(LowCardinality(String), String)` or `Map(LowCardinality(String), LowCardinality(String))` | Map of tags excluding the tag `__name__` containing the name of a metric and excluding tags with names enumerated in the [tags_to_columns](#settings) setting | -| `all_tags` | [ ] | `Map(String, LowCardinality(String))` | `Map(String, String)` or `Map(LowCardinality(String), String)` or `Map(LowCardinality(String), LowCardinality(String))` | Ephemeral column, each row is a map of all the tags excluding only the tag `__name__` containing the name of a metric. The only purpose of that column is to be used while calculating `id` | -| `min_time` | [ ] | `Nullable(DateTime64(3))` | `DateTime64(X)` or `Nullable(DateTime64(X))` | Minimum timestamp of time series with that `id`. The column is created if [store_min_time_and_max_time](#settings) is `true` | -| `max_time` | [ ] | `Nullable(DateTime64(3))` | `DateTime64(X)` or `Nullable(DateTime64(X))` | Maximum timestamp of time series with that `id`. The column is created if [store_min_time_and_max_time](#settings) is `true` | - -3. The _metrics_ table {#metrics-table} contains some information about metrics been collected, the types of those metrics and their descriptions. -The _metrics_ table must have columns: - -| Name | Mandatory? | Default type | Possible types | Description | -|---|---|---|---|---| -| `metric_family_name` | [x] | `String` | `String` or `LowCardinality(String)` | The name of a metric family | -| `type` | [x] | `String` | `String` or `LowCardinality(String)` | The type of a metric family, one of "counter", "gauge", "summary", "stateset", "histogram", "gaugehistogram" | -| `unit` | [x] | `String` | `String` or `LowCardinality(String)` | The unit used in a metric | -| `help` | [x] | `String` | `String` or `LowCardinality(String)` | The description of a metric | - -Any row inserted into a `TimeSeries` table will be in fact stored in those three target tables. -A `TimeSeries` table contains all those columns from the [data]{#data-table}, [tags]{#tags-table}, [metrics]{#metrics-table} tables. - -## Creation {#creation} - -There are multiple ways to create a table with the `TimeSeries` table engine. -The simplest statement - -``` sql -CREATE TABLE my_table ENGINE=TimeSeries -``` - -will actually create the following table (you can see that by executing `SHOW CREATE TABLE my_table`): - -``` sql -CREATE TABLE my_table -( - `id` UUID DEFAULT reinterpretAsUUID(sipHash128(metric_name, all_tags)), - `timestamp` DateTime64(3), - `value` Float64, - `metric_name` LowCardinality(String), - `tags` Map(LowCardinality(String), String), - `all_tags` Map(String, String), - `min_time` Nullable(DateTime64(3)), - `max_time` Nullable(DateTime64(3)), - `metric_family_name` String, - `type` String, - `unit` String, - `help` String -) -ENGINE = TimeSeries -DATA ENGINE = MergeTree ORDER BY (id, timestamp) -DATA INNER UUID '01234567-89ab-cdef-0123-456789abcdef' -TAGS ENGINE = AggregatingMergeTree PRIMARY KEY metric_name ORDER BY (metric_name, id) -TAGS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' -METRICS ENGINE = ReplacingMergeTree ORDER BY metric_family_name -METRICS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' -``` - -So the columns were generated automatically and also there are three inner UUIDs in this statement - -one per each inner target table that was created. -(Inner UUIDs are not shown normally until setting -[show_table_uuid_in_table_create_query_if_not_nil](../../../operations/settings/settings#show_table_uuid_in_table_create_query_if_not_nil) -is set.) - -Inner target tables have names like `.inner_id.data.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`, -`.inner_id.tags.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`, `.inner_id.metrics.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` -and each target table has columns which is a subset of the columns of the main `TimeSeries` table: - -``` sql -CREATE TABLE default.`.inner_id.data.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` -( - `id` UUID, - `timestamp` DateTime64(3), - `value` Float64 -) -ENGINE = MergeTree -ORDER BY (id, timestamp) -``` - -``` sql -CREATE TABLE default.`.inner_id.tags.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` -( - `id` UUID DEFAULT reinterpretAsUUID(sipHash128(metric_name, all_tags)), - `metric_name` LowCardinality(String), - `tags` Map(LowCardinality(String), String), - `all_tags` Map(String, String) EPHEMERAL, - `min_time` SimpleAggregateFunction(min, Nullable(DateTime64(3))), - `max_time` SimpleAggregateFunction(max, Nullable(DateTime64(3))) -) -ENGINE = AggregatingMergeTree -PRIMARY KEY metric_name -ORDER BY (metric_name, id) -``` - -``` sql -CREATE TABLE default.`.inner_id.metrics.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` -( - `metric_family_name` String, - `type` String, - `unit` String, - `help` String -) -ENGINE = ReplacingMergeTree -ORDER BY metric_family_name -``` - -## Adjusting types of columns {#adjusting-column-types} - -You can adjust the types of almost any column of the inner target tables by specifying them explicitly -while defining the main table. For example, - -``` sql -CREATE TABLE my_table -( - timestamp DateTime64(6) -) ENGINE=TimeSeries -``` - -will make the inner [data]{#data-table} table store timestamp in microseconds instead of milliseconds: - -``` sql -CREATE TABLE default.`.inner_id.data.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` -( - `id` UUID, - `timestamp` DateTime64(6), - `value` Float64 -) -ENGINE = MergeTree -ORDER BY (id, timestamp) -``` - -## The `id` column {#id-column} - -The `id` column contains identifiers, every identifier is calculated for a combination of a metric name and tags. -The DEFAULT expression for the `id` column is an expression which will be used to calculate such identifiers. -Both the type of the `id` column and that expression can be adjusted by specifying them explicitly: - -``` sql -CREATE TABLE my_table -( - id UInt64 DEFAULT sipHash64(metric_name, all_tags) -) ENGINE=TimeSeries -``` - -## The `tags` and `all_tags` columns {#tags-and-all-tags} - -There are two columns containing maps of tags - `tags` and `all_tags`. In this example they mean the same, however they can be different -if setting `tags_to_columns` is used. This setting allows you to specify that a specific tag should be stored in a separate column instead of storing -it in a map inside the `tags` column: - -``` sql -CREATE TABLE my_table ENGINE=TimeSeries SETTINGS = {'instance': 'instance', 'job': 'job'} -``` - -This statement will add columns -``` - `instance` String, - `job` String -``` -to the definition of both `my_table` and its inner [tags]{#tags-table} target table. In this case the `tags` column will not contain tags `instance` and `job`, -but the `all_tags` column will contain them. The `all_tags` column is ephemeral and its only purpose to be used in the DEFAULT expression -for the `id` column. - -The types of columns can be adjusted by specifying them explicitly: - -``` sql -CREATE TABLE my_table (instance LowCardinality(String), job LowCardinality(Nullable(String))) -ENGINE=TimeSeries SETTINGS = {'instance': 'instance', 'job': 'job'} -``` - -## Table engines of inner target tables {#inner-table-engines} - -By default inner target tables use the following table engines: - -- the [data]{#data-table} table uses [MergeTree](../mergetree-family/mergetree); -- the [tags]{#tags-table} table uses [AggregatingMergeTree](../mergetree-family/aggregatingmergetree) because the same data is often inserted multiple times to this table so we need a way -to remove duplicates, and also because it's required to do aggregation for columns `min_time` and `max_time`; -- the [metrics]{#metrics-table} table uses [ReplacingMergeTree](../mergetree-family/replacingmergetree) because the same data is often inserted multiple times to this table so we need a way -to remove duplicates. - -Other table engines also can be used for inner target tables if it's specified so: - -``` sql -CREATE TABLE my_table ENGINE=TimeSeries -DATA ENGINE=ReplicatedMergeTree -TAGS ENGINE=ReplicatedAggregatingMergeTree -METRICS ENGINE=ReplicatedReplacingMergeTree -``` - -## External target tables {#external-target-tables} - -It's possible to make a `TimeSeries` table use a manually created table: - -``` sql -CREATE TABLE data_for_my_table -( - `id` UUID, - `timestamp` DateTime64(3), - `value` Float64 -) -ENGINE = MergeTree -ORDER BY (id, timestamp); - -CREATE TABLE tags_for_my_table ... - -CREATE TABLE metrics_for_my_table ... - -CREATE TABLE my_table ENGINE=TimeSeries DATA data_for_my_table TAGS tags_for_my_table METRICS metrics_for_my_table; -``` - -## Settings {#settings} - -Here is a list of settings which can be specified while defining a `TimeSeries` table: - -| Name | Type | Default | Description | -|---|---|---|---| -| `tags_to_columns` | Map | {} | Map specifying which tags should be put to separate columns in the [tags]{#tags-table} table. Syntax: `{'tag1': 'column1', 'tag2' : column2, ...}` | -| `use_all_tags_column_to_generate_id` | Bool | true | When generating an expression to calculate an identifier of a time series, this flag enables using the `all_tags` column in that calculation | -| `store_min_time_and_max_time` | Bool | true | If set to true then the table will store `min_time` and `max_time` for each time series | -| `aggregate_min_time_and_max_time` | Bool | true | When creating an inner target `tags` table, this flag enables using `SimpleAggregateFunction(min, Nullable(DateTime64(3)))` instead of just `Nullable(DateTime64(3))` as the type of the `min_time` column, and the same for the `max_time` column | -| `filter_by_min_time_and_max_time` | Bool | true | If set to true then the table will use the `min_time` and `max_time` columns for filtering time series | - -## Functions {#functions} - -Here is a list of functions supporting a `TimeSeries` table as an argument: - -- [timeSeriesData](../../../sql-reference/table-functions/timeSeriesData.md) -- [timeSeriesTags](../../../sql-reference/table-functions/timeSeriesTags.md) -- [timeSeriesMetrics](../../../sql-reference/table-functions/timeSeriesMetrics.md) diff --git a/docs/en/interfaces/prometheus.md b/docs/en/interfaces/prometheus.md deleted file mode 100644 index 5eac99f685e..00000000000 --- a/docs/en/interfaces/prometheus.md +++ /dev/null @@ -1,161 +0,0 @@ ---- -slug: /en/interfaces/prometheus -sidebar_position: 19 -sidebar_label: Prometheus protocols ---- - -# Prometheus protocols - -## Exposing metrics {#expose} - -:::note -ClickHouse Cloud does not currently support connecting to Prometheus. To be notified when this feature is supported, please contact support@clickhouse.com. -::: - -ClickHouse can expose its own metrics for scraping from Prometheus: - -```xml - - 9363 - /metrics - true - true - true - true - - -Section `` can be used to make more extended handlers. -This section is similar to [](http.md) but works for prometheus protocols: - -```xml - - 9363 - - - /metrics - - expose_metrics - true - true - true - true - - - - -``` - -Settings: - -| Name | Default | Description | -|---|---|---|---| -| `port` | none | Port for serving the exposing metrics protocol. | -| `endpoint` | `/metrics` | HTTP endpoint for scraping metrics by prometheus server. Starts with `/`. Should not be used with the `` section. | -| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](http.md) section. | -| `metrics` | true | Expose metrics from the [system.metrics](../operations/system-tables/metrics.md) table. | -| `asynchronous_metrics` | true | Expose current metrics values from the [system.asynchronous_metrics](../operations/system-tables/asynchronous_metrics.md) table. | -| `events` | true | Expose metrics from the [system.events](../operations/system-tables/events.md) table. | -| `errors` | true | Expose the number of errors by error codes occurred since the last server restart. This information could be obtained from the [system.errors](../operations/system-tables/errors.md) as well. | - -Check (replace `127.0.0.1` with the IP addr or hostname of your ClickHouse server): - -```bash -curl 127.0.0.1:9363/metrics -``` - -## Remote-write protocol {#remote-write} - -ClickHouse supports the [remote-write](https://prometheus.io/docs/specs/remote_write_spec/) protocol. -Data are received by this protocol and written to a [TimeSeries](../engines/table-engines/integrations/time-series.md) table -(which should be created beforehand). - -```xml - - 9363 - - - /write - - remote_writedb_name - time_series_table
-
-
-
-
-``` - -Settings: - -| Name | Default | Description | -|---|---|---|---| -| `port` | none | Port for serving the `remote-write` protocol. | -| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](http.md) section. | -| `table` | none | The name of a [TimeSeries](../engines/table-engines/integrations/time-series.md) table to write data received by the `remote-write` protocol. This name can optionally contain the name of a database too. | -| `database` | none | The name of a database where the table specified in the `table` setting is located if it's not specified in the `table` setting. | - -## Remote-read protocol {#remote-read} - -ClickHouse supports the [remote-read](https://prometheus.io/docs/prometheus/latest/querying/remote_read_api/) protocol. -Data are read from a [TimeSeries](../engines/table-engines/integrations/time-series.md) table and sent via this protocol. - -```xml - - 9363 - - - /read - - remote_readdb_name - time_series_table
-
-
-
-
-``` - -Settings: - -| Name | Default | Description | -|---|---|---|---| -| `port` | none | Port for serving the `remote-read` protocol. | -| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](http.md) section. | -| `table` | none | The name of a [TimeSeries](../engines/table-engines/integrations/time-series.md) table to read data to send by the `remote-read` protocol. This name can optionally contain the name of a database too. | -| `database` | none | The name of a database where the table specified in the `table` setting is located if it's not specified in the `table` setting. | - -## Configuration for multiple protocols {#multiple-protocols} - -Multiple protocols can be specified together in one place: - -```xml - - 9363 - - - /metrics - - expose_metrics - true - true - true - true - - - - /write - - remote_writedb_name.time_series_table - - - - /read - - remote_readdb_name.time_series_table - - - - -``` diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 68f61650e00..a1e3c292b04 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2112,6 +2112,48 @@ The trailing slash is mandatory. /var/lib/clickhouse/ ``` +## Prometheus {#prometheus} + +:::note +ClickHouse Cloud does not currently support connecting to Prometheus. To be notified when this feature is supported, please contact support@clickhouse.com. +::: + +Exposing metrics data for scraping from [Prometheus](https://prometheus.io). + +Settings: + +- `endpoint` – HTTP endpoint for scraping metrics by prometheus server. Start from ‘/’. +- `port` – Port for `endpoint`. +- `metrics` – Expose metrics from the [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) table. +- `events` – Expose metrics from the [system.events](../../operations/system-tables/events.md#system_tables-events) table. +- `asynchronous_metrics` – Expose current metrics values from the [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) table. +- `errors` - Expose the number of errors by error codes occurred since the last server restart. This information could be obtained from the [system.errors](../../operations/system-tables/asynchronous_metrics.md#system_tables-errors) as well. + +**Example** + +``` xml + + 0.0.0.0 + 8123 + 9000 + + + /metrics + 9363 + true + true + true + true + + + +``` + +Check (replace `127.0.0.1` with the IP addr or hostname of your ClickHouse server): +```bash +curl 127.0.0.1:9363/metrics +``` + ## query_log {#query-log} Setting for logging queries received with the [log_queries=1](../../operations/settings/settings.md) setting. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index feac12f9c99..35547c3a9a6 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5614,14 +5614,3 @@ Default value: `1GiB`. Disable all insert and mutations (alter table update / alter table delete / alter table drop partition). Set to true, can make this node focus on reading queries. Default value: `false`. - -## allow_experimental_time_series_table {#allow-experimental-time-series-table} - -Allows creation of tables with the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine. - -Possible values: - -- 0 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is disabled. -- 1 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is enabled. - -Default value: `0`. diff --git a/docs/en/sql-reference/table-functions/timeSeriesData.md b/docs/en/sql-reference/table-functions/timeSeriesData.md deleted file mode 100644 index aa7a9d30c2a..00000000000 --- a/docs/en/sql-reference/table-functions/timeSeriesData.md +++ /dev/null @@ -1,28 +0,0 @@ ---- -slug: /en/sql-reference/table-functions/timeSeriesData -sidebar_position: 145 -sidebar_label: timeSeriesData ---- - -# timeSeriesData - -`timeSeriesData(db_name.time_series_table)` - Returns the [data](../../engines/table-engines/integrations/time-series.md#data-table) table -used by table `db_name.time_series_table` which table engine is [TimeSeries](../../engines/table-engines/integrations/time-series.md): - -``` sql -CREATE TABLE db_name.time_series_table ENGINE=TimeSeries DATA data_table -``` - -The function also works if the _data_ table is inner: - -``` sql -CREATE TABLE db_name.time_series_table ENGINE=TimeSeries DATA INNER UUID '01234567-89ab-cdef-0123-456789abcdef' -``` - -The following queries are equivalent: - -``` sql -SELECT * FROM timeSeriesData(db_name.time_series_table); -SELECT * FROM timeSeriesData('db_name.time_series_table'); -SELECT * FROM timeSeriesData('db_name', 'time_series_table'); -``` diff --git a/docs/en/sql-reference/table-functions/timeSeriesMetrics.md b/docs/en/sql-reference/table-functions/timeSeriesMetrics.md deleted file mode 100644 index 913f1185bca..00000000000 --- a/docs/en/sql-reference/table-functions/timeSeriesMetrics.md +++ /dev/null @@ -1,28 +0,0 @@ ---- -slug: /en/sql-reference/table-functions/timeSeriesMetrics -sidebar_position: 145 -sidebar_label: timeSeriesMetrics ---- - -# timeSeriesMetrics - -`timeSeriesMetrics(db_name.time_series_table)` - Returns the [metrics](../../engines/table-engines/integrations/time-series.md#metrics-table) table -used by table `db_name.time_series_table` which table engine is [TimeSeries](../../engines/table-engines/integrations/time-series.md): - -``` sql -CREATE TABLE db_name.time_series_table ENGINE=TimeSeries METRICS metrics_table -``` - -The function also works if the _metrics_ table is inner: - -``` sql -CREATE TABLE db_name.time_series_table ENGINE=TimeSeries METRICS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' -``` - -The following queries are equivalent: - -``` sql -SELECT * FROM timeSeriesMetrics(db_name.time_series_table); -SELECT * FROM timeSeriesMetrics('db_name.time_series_table'); -SELECT * FROM timeSeriesMetrics('db_name', 'time_series_table'); -``` diff --git a/docs/en/sql-reference/table-functions/timeSeriesTags.md b/docs/en/sql-reference/table-functions/timeSeriesTags.md deleted file mode 100644 index 663a7dc6ac8..00000000000 --- a/docs/en/sql-reference/table-functions/timeSeriesTags.md +++ /dev/null @@ -1,28 +0,0 @@ ---- -slug: /en/sql-reference/table-functions/timeSeriesTags -sidebar_position: 145 -sidebar_label: timeSeriesTags ---- - -# timeSeriesTags - -`timeSeriesTags(db_name.time_series_table)` - Returns the [tags](../../engines/table-engines/integrations/time-series.md#tags-table) table -used by table `db_name.time_series_table` which table engine is [TimeSeries](../../engines/table-engines/integrations/time-series.md): - -``` sql -CREATE TABLE db_name.time_series_table ENGINE=TimeSeries TAGS tags_table -``` - -The function also works if the _tags_ table is inner: - -``` sql -CREATE TABLE db_name.time_series_table ENGINE=TimeSeries TAGS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' -``` - -The following queries are equivalent: - -``` sql -SELECT * FROM timeSeriesTags(db_name.time_series_table); -SELECT * FROM timeSeriesTags('db_name.time_series_table'); -SELECT * FROM timeSeriesTags('db_name', 'time_series_table'); -``` diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 382e64f343c..8a9a8d2e76c 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1672,7 +1672,6 @@ fuzzQuery fuzzer fuzzers gRPC -gaugehistogram gccMurmurHash gcem generateRandom @@ -2557,7 +2556,6 @@ startsWithUTF startswith statbox stateful -stateset stddev stddevPop stddevPopStable @@ -2689,10 +2687,6 @@ themself threadpool throwIf timeDiff -TimeSeries -timeSeriesData -timeSeriesMetrics -timeSeriesTags timeSlot timeSlots timeZone From d09eaac0dae0c937fa652a4973263f6c1dc18028 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 19:34:56 +0200 Subject: [PATCH 334/644] Make 00965_shard_unresolvable_addresses.sql faster and parallelizable --- .../0_stateless/00965_shard_unresolvable_addresses.sql | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql b/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql index 16b62c37d80..41bf4d261f6 100644 --- a/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql +++ b/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql @@ -1,9 +1,9 @@ --- Tags: shard, no-parallel +-- Tags: shard SET prefer_localhost_replica = 1; +SET connections_with_failover_max_tries=1; +SET connect_timeout_with_failover_ms=2000; +SET connect_timeout_with_failover_secure_ms=2000; SELECT count() FROM remote('127.0.0.1,localhos', system.one); -- { serverError ALL_CONNECTION_TRIES_FAILED } SELECT count() FROM remote('127.0.0.1|localhos', system.one); - --- Clear cache to avoid future errors in the logs -SYSTEM DROP DNS CACHE From 71d47d2d07c29e99193f2c5d454ba6967a43ec48 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 19:35:15 +0200 Subject: [PATCH 335/644] Disable 02434_cancel_insert_when_client_dies in fast tests --- .../0_stateless/02434_cancel_insert_when_client_dies.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh index dca8dae22c3..1548bef857f 100755 --- a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh +++ b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-asan, no-msan, no-tsan, no-debug +# Tags: no-random-settings, no-asan, no-msan, no-tsan, no-debug, no-fasttest +# no-fasttest: The test runs for 40 seconds # shellcheck disable=SC2009 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From ebc5b260abbfb0b87d1c551d4c8c67b010858c0f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 19:35:32 +0200 Subject: [PATCH 336/644] Disable 02447_drop_database_replica in slow tests --- tests/queries/0_stateless/02447_drop_database_replica.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02447_drop_database_replica.sh b/tests/queries/0_stateless/02447_drop_database_replica.sh index c6bf298f944..abe99398a56 100755 --- a/tests/queries/0_stateless/02447_drop_database_replica.sh +++ b/tests/queries/0_stateless/02447_drop_database_replica.sh @@ -1,8 +1,9 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest # no-parallel: This test is not parallel because when we execute system-wide SYSTEM DROP REPLICA, # other tests might shut down the storage in parallel and the test will fail. +# no-fasttest: It has several tests with timeouts for inactive replicas CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 64e33c510a7cc80bb771597a0ddcc11b06fa4ac0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 19:35:52 +0200 Subject: [PATCH 337/644] Disable 02998_primary_key_skip_columns in fast tests --- tests/queries/0_stateless/02998_primary_key_skip_columns.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02998_primary_key_skip_columns.sql b/tests/queries/0_stateless/02998_primary_key_skip_columns.sql index ee558996b52..1abe692a7a4 100644 --- a/tests/queries/0_stateless/02998_primary_key_skip_columns.sql +++ b/tests/queries/0_stateless/02998_primary_key_skip_columns.sql @@ -1,4 +1,5 @@ --- Tags: no-asan, no-tsan, no-msan, no-ubsan, no-random-settings, no-debug +-- Tags: no-asan, no-tsan, no-msan, no-ubsan, no-random-settings, no-debug, no-fasttest +-- no-fasttest: Low index granularity and too many parts makes the test slow DROP TABLE IF EXISTS test; From 401d4348b5ed5666a93dfb9cfe36d093bce9c908 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 19:49:06 +0200 Subject: [PATCH 338/644] Remove waiting for mutations from fast tests --- .../0_stateless/00834_kill_mutation_replicated_zookeeper.sh | 3 ++- .../0_stateless/01414_mutations_and_errors_zookeeper.sh | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh b/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh index 16ad08deeb2..3e6b339cb57 100755 --- a/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: replica, no-debug +# Tags: replica, no-debug, no-fasttest +# no-fasttest: Waiting for failed mutations is slow: https://github.com/ClickHouse/ClickHouse/issues/67936 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh b/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh index 31e2cc395aa..5cdd6057050 100755 --- a/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh +++ b/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-parallel +# Tags: zookeeper, no-parallel, no-fasttest +# no-fasttest: Waiting for failed mutations is slow: https://github.com/ClickHouse/ClickHouse/issues/67936 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 015691058073c4e7587e053efbc880dc88190586 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 20:14:09 +0200 Subject: [PATCH 339/644] Disable another slow tests in fasttest --- tests/queries/0_stateless/01030_storage_url_syntax.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/01030_storage_url_syntax.sql b/tests/queries/0_stateless/01030_storage_url_syntax.sql index 0eb89af8462..084486b61ee 100644 --- a/tests/queries/0_stateless/01030_storage_url_syntax.sql +++ b/tests/queries/0_stateless/01030_storage_url_syntax.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: Timeout for the first query (CANNOT_DETECT_FORMAT) is too slow: https://github.com/ClickHouse/ClickHouse/issues/67939 + drop table if exists test_table_url_syntax ; create table test_table_url_syntax (id UInt32) ENGINE = URL('') From 6061f01dc0fabd8d737a7d5e13b3828f2b9c0ab7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 20:21:36 +0200 Subject: [PATCH 340/644] Disable more slow tests --- .../0_stateless/02044_url_glob_parallel_connection_refused.sh | 3 ++- tests/queries/0_stateless/02435_rollback_cancelled_queries.sh | 3 ++- .../queries/0_stateless/02445_replicated_db_alter_partition.sh | 3 +++ .../02581_share_big_sets_between_mutation_tasks.sql | 3 ++- tests/queries/0_stateless/02703_max_local_read_bandwidth.sh | 3 ++- .../queries/0_stateless/02805_distributed_queries_timeouts.sql | 2 ++ .../0_stateless/02994_merge_tree_mutations_cleanup.sql.j2 | 2 ++ tests/queries/0_stateless/02995_forget_partition.sh | 3 ++- 8 files changed, 17 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh b/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh index b4b0ee8a023..d70845f52eb 100755 --- a/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh +++ b/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: distributed +# Tags: distributed, no-fasttest +# no-fasttest: Slow wait and retries CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index ba652013a57..1bc7d4cd1d3 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-ordinary-database +# Tags: no-random-settings, no-ordinary-database, no-fasttest +# no-fasttest: The test is slow (too many small blocks) # shellcheck disable=SC2009 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) diff --git a/tests/queries/0_stateless/02445_replicated_db_alter_partition.sh b/tests/queries/0_stateless/02445_replicated_db_alter_partition.sh index 4d9048354a1..f716a6aa779 100755 --- a/tests/queries/0_stateless/02445_replicated_db_alter_partition.sh +++ b/tests/queries/0_stateless/02445_replicated_db_alter_partition.sh @@ -1,5 +1,8 @@ #!/usr/bin/env bash +# Tags: no-fasttest +# no-fasttest: Slow timeouts + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.sql b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.sql index ea1452fc372..ad8bef7fbb7 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.sql +++ b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.sql @@ -1,4 +1,5 @@ --- Tags: no-tsan, no-asan, no-ubsan, no-msan +-- Tags: no-tsan, no-asan, no-ubsan, no-msan, no-fasttest +-- no-fasttest: Slow test -- no sanitizers: too slow sometimes DROP TABLE IF EXISTS 02581_trips; diff --git a/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh b/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh index 6f43c1ae869..03e0f363d71 100755 --- a/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh +++ b/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-object-storage, no-random-settings, no-random-merge-tree-settings +# Tags: no-object-storage, no-random-settings, no-random-merge-tree-settings, no-fasttest +# no-fasttest: The test is slow CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql index f6bccc99977..98aeac36243 100644 --- a/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql +++ b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest +-- no-fasttest: Timeouts are slow create table dist as system.one engine=Distributed(test_shard_localhost, system, one); select sleep(8) from dist settings function_sleep_max_microseconds_per_block=8e9, prefer_localhost_replica=0, receive_timeout=7, async_socket_for_remote=0, use_hedged_requests=1 format Null; select sleep(8) from dist settings function_sleep_max_microseconds_per_block=8e9, prefer_localhost_replica=0, receive_timeout=7, async_socket_for_remote=1, use_hedged_requests=0 format Null; diff --git a/tests/queries/0_stateless/02994_merge_tree_mutations_cleanup.sql.j2 b/tests/queries/0_stateless/02994_merge_tree_mutations_cleanup.sql.j2 index 1b9be79dbe4..65601cd39be 100644 --- a/tests/queries/0_stateless/02994_merge_tree_mutations_cleanup.sql.j2 +++ b/tests/queries/0_stateless/02994_merge_tree_mutations_cleanup.sql.j2 @@ -1,3 +1,5 @@ +-- Tags: no-fasttest +-- no-fasttest: Slow wait drop table if exists data_rmt; drop table if exists data_mt; diff --git a/tests/queries/0_stateless/02995_forget_partition.sh b/tests/queries/0_stateless/02995_forget_partition.sh index 6fa0b96e90d..e9d4590cce6 100755 --- a/tests/queries/0_stateless/02995_forget_partition.sh +++ b/tests/queries/0_stateless/02995_forget_partition.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-replicated-database +# Tags: zookeeper, no-replicated-database, no-fasttest +# no-fasttest: Slow wait CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From c4fda6cd4c5b7cfa40792c742eab98aa1857fd7d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 18:26:22 +0000 Subject: [PATCH 341/644] Fix style --- src/Storages/Statistics/StatisticsTDigest.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index b0c4bfda27d..fd9b922ffc8 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -9,7 +9,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_STATISTICS; -extern const int LOGICAL_ERROR; } StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & description, const DataTypePtr & data_type_) From 45b55c4d6ed5cb5f023855ac78f4097dafba7fec Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Tue, 6 Aug 2024 22:25:08 +0200 Subject: [PATCH 342/644] Update comment --- CMakeLists.txt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 0d862b23e3a..2e4be09f5d3 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -430,7 +430,10 @@ endif() if (NOT OS_ANDROID AND OS_LINUX AND NOT ARCH_S390X AND NOT SANITIZE) # Slightly more efficient code can be generated - # Disabled for Android, because otherwise ClickHouse cannot run on Android. + # Using '-no-pie' builds executables with fixed addresses, resulting in slightly more efficient code + # and keeping binary addresses constant even with ASLR enabled. + # Disabled on Android as it requires PIE: https://source.android.com/docs/security/enhancements#android-5 + # Disabled on IBM S390X due to build issues with 'no-pie' set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -fno-pie") set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -fno-pie") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -no-pie -Wl,-no-pie") From db2d732b2b6ee7e61a2fe5c644db7ccb718e9f0c Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Tue, 6 Aug 2024 22:25:47 +0200 Subject: [PATCH 343/644] Update comment2 --- CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 2e4be09f5d3..afab666a733 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -429,7 +429,6 @@ if (NOT SANITIZE) endif() if (NOT OS_ANDROID AND OS_LINUX AND NOT ARCH_S390X AND NOT SANITIZE) - # Slightly more efficient code can be generated # Using '-no-pie' builds executables with fixed addresses, resulting in slightly more efficient code # and keeping binary addresses constant even with ASLR enabled. # Disabled on Android as it requires PIE: https://source.android.com/docs/security/enhancements#android-5 From 340a2bcd2582c563c0f2eaeda0da1f32269b5253 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Tue, 6 Aug 2024 22:36:16 +0200 Subject: [PATCH 344/644] Update comment3 --- CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index afab666a733..7b4e0484ab1 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -433,6 +433,7 @@ if (NOT OS_ANDROID AND OS_LINUX AND NOT ARCH_S390X AND NOT SANITIZE) # and keeping binary addresses constant even with ASLR enabled. # Disabled on Android as it requires PIE: https://source.android.com/docs/security/enhancements#android-5 # Disabled on IBM S390X due to build issues with 'no-pie' + # Disabled with sanitizers to avoid issues with maximum relocation size: https://github.com/ClickHouse/ClickHouse/pull/49145 set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -fno-pie") set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -fno-pie") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -no-pie -Wl,-no-pie") From 04438784e2178820537c65b66b5a8341f3d63b8d Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 6 Aug 2024 16:45:46 -0400 Subject: [PATCH 345/644] add a stateless test for `grant current grants` --- .../03215_grant_current_grants.reference | 2 ++ .../0_stateless/03215_grant_current_grants.sh | 26 +++++++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 tests/queries/0_stateless/03215_grant_current_grants.reference create mode 100755 tests/queries/0_stateless/03215_grant_current_grants.sh diff --git a/tests/queries/0_stateless/03215_grant_current_grants.reference b/tests/queries/0_stateless/03215_grant_current_grants.reference new file mode 100644 index 00000000000..e4f6850b806 --- /dev/null +++ b/tests/queries/0_stateless/03215_grant_current_grants.reference @@ -0,0 +1,2 @@ +GRANT SELECT, CREATE TABLE, CREATE VIEW ON default.* +GRANT SELECT ON default.* diff --git a/tests/queries/0_stateless/03215_grant_current_grants.sh b/tests/queries/0_stateless/03215_grant_current_grants.sh new file mode 100755 index 00000000000..68af4a62bba --- /dev/null +++ b/tests/queries/0_stateless/03215_grant_current_grants.sh @@ -0,0 +1,26 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +user1="user03215_1_${CLICKHOUSE_DATABASE}_$RANDOM" +user2="user03215_2_${CLICKHOUSE_DATABASE}_$RANDOM" +user3="user03215_3_${CLICKHOUSE_DATABASE}_$RANDOM" +db=${CLICKHOUSE_DATABASE} + + +${CLICKHOUSE_CLIENT} --query "CREATE USER $user1, $user2, $user3;"; +${CLICKHOUSE_CLIENT} --query "GRANT SELECT, CREATE TABLE, CREATE VIEW ON $db.* TO $user1 WITH GRANT OPTION;"; + +${CLICKHOUSE_CLIENT} --query "GRANT CURRENT GRANTS ON $db.* TO $user2" --user $user1; +${CLICKHOUSE_CLIENT} --query "GRANT CURRENT GRANTS ON $db.* TO $user3" --user $user2; + +${CLICKHOUSE_CLIENT} --query "SHOW GRANTS FOR $user2" | sed 's/ TO.*//'; +${CLICKHOUSE_CLIENT} --query "SHOW GRANTS FOR $user3" | sed 's/ TO.*//'; + +${CLICKHOUSE_CLIENT} --query "GRANT CURRENT GRANTS(SELECT ON $db.*) TO $user3" --user $user1; +${CLICKHOUSE_CLIENT} --query "SHOW GRANTS FOR $user3" | sed 's/ TO.*//'; + +${CLICKHOUSE_CLIENT} --query "DROP USER IF EXISTS $user1, $user2, $user3"; From 5ae5cd35b5b263d14bdd62aa5cbaa1e22219208a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 6 Aug 2024 21:50:31 +0100 Subject: [PATCH 346/644] update --- base/poco/Net/include/Poco/Net/HTTPServerSession.h | 4 ++-- src/Server/HTTP/sendExceptionToHTTPClient.cpp | 2 +- .../0_stateless/00408_http_keep_alive.reference | 6 +++--- tests/queries/0_stateless/00408_http_keep_alive.sh | 7 ++++--- tests/queries/0_stateless/00501_http_head.re | 12 ++++++++++++ tests/queries/0_stateless/00501_http_head.reference | 4 ++-- tests/queries/0_stateless/00501_http_head.sh | 5 +++-- 7 files changed, 27 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/00501_http_head.re diff --git a/base/poco/Net/include/Poco/Net/HTTPServerSession.h b/base/poco/Net/include/Poco/Net/HTTPServerSession.h index 93f31012336..54e7f2c8c50 100644 --- a/base/poco/Net/include/Poco/Net/HTTPServerSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPServerSession.h @@ -57,10 +57,10 @@ namespace Net /// Returns the server's address. void setKeepAliveTimeout(Poco::Timespan keepAliveTimeout); - + size_t getKeepAliveTimeout() const { return _keepAliveTimeout.totalSeconds(); } - size_t getMaxKeepAliveRequests() const { return _maxKeepAliveRequests; } + size_t getMaxKeepAliveRequests() const { return _maxKeepAliveRequests; } private: bool _firstRequest; diff --git a/src/Server/HTTP/sendExceptionToHTTPClient.cpp b/src/Server/HTTP/sendExceptionToHTTPClient.cpp index 022a763a9a2..658b7a4707a 100644 --- a/src/Server/HTTP/sendExceptionToHTTPClient.cpp +++ b/src/Server/HTTP/sendExceptionToHTTPClient.cpp @@ -29,7 +29,7 @@ void sendExceptionToHTTPClient( if (!out) { /// If nothing was sent yet. - WriteBufferFromHTTPServerResponse out_for_message{response, request.getMethod() == HTTPRequest::HTTP_HEAD, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT}; + WriteBufferFromHTTPServerResponse out_for_message{response, request.getMethod() == HTTPRequest::HTTP_HEAD}; out_for_message.writeln(exception_message); out_for_message.finalize(); diff --git a/tests/queries/0_stateless/00408_http_keep_alive.reference b/tests/queries/0_stateless/00408_http_keep_alive.reference index d5d7dacce9e..5402036bfd7 100644 --- a/tests/queries/0_stateless/00408_http_keep_alive.reference +++ b/tests/queries/0_stateless/00408_http_keep_alive.reference @@ -1,6 +1,6 @@ < Connection: Keep-Alive -< Keep-Alive: timeout=10, max=10000 +< Keep-Alive: timeout=10, max=? < Connection: Keep-Alive -< Keep-Alive: timeout=10, max=10000 +< Keep-Alive: timeout=10, max=? < Connection: Keep-Alive -< Keep-Alive: timeout=10, max=10000 +< Keep-Alive: timeout=10, max=? diff --git a/tests/queries/0_stateless/00408_http_keep_alive.sh b/tests/queries/0_stateless/00408_http_keep_alive.sh index 4bd0e494eb8..4a1cb4ed712 100755 --- a/tests/queries/0_stateless/00408_http_keep_alive.sh +++ b/tests/queries/0_stateless/00408_http_keep_alive.sh @@ -6,9 +6,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) URL="${CLICKHOUSE_PORT_HTTP_PROTO}://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTP}/" -${CLICKHOUSE_CURL} -vsS "${URL}" --data-binary @- <<< "SELECT 1" 2>&1 | perl -lnE 'print if /Keep-Alive/'; -${CLICKHOUSE_CURL} -vsS "${URL}" --data-binary @- <<< " error here " 2>&1 | perl -lnE 'print if /Keep-Alive/'; -${CLICKHOUSE_CURL} -vsS "${URL}"ping 2>&1 | perl -lnE 'print if /Keep-Alive/'; +# the sed command here replaces the real number of left requests with a question mark, because it can vary and we don't really have control over it +${CLICKHOUSE_CURL} -vsS "${URL}" --data-binary @- <<< "SELECT 1" 2>&1 | sed -r 's/(keep-alive: timeout=10, max=)[0-9]+/\1?/I' | grep -i 'keep-alive'; +${CLICKHOUSE_CURL} -vsS "${URL}" --data-binary @- <<< " error here " 2>&1 | sed -r 's/(keep-alive: timeout=10, max=)[0-9]+/\1?/I' | grep -i 'keep-alive'; +${CLICKHOUSE_CURL} -vsS "${URL}"ping 2>&1 | perl -lnE 'print if /Keep-Alive/' | sed -r 's/(keep-alive: timeout=10, max=)[0-9]+/\1?/I' | grep -i 'keep-alive'; # no keep-alive: ${CLICKHOUSE_CURL} -vsS "${URL}"404/not/found/ 2>&1 | perl -lnE 'print if /Keep-Alive/'; diff --git a/tests/queries/0_stateless/00501_http_head.re b/tests/queries/0_stateless/00501_http_head.re new file mode 100644 index 00000000000..807bcd4922e --- /dev/null +++ b/tests/queries/0_stateless/00501_http_head.re @@ -0,0 +1,12 @@ +HTTP/1.1 200 OK +Connection: Keep-Alive +Content-Type: text/tab-separated-values; charset=UTF-8 +Transfer-Encoding: chunked +Keep-Alive: timeout=10, max=? + +HTTP/1.1 200 OK +Connection: Keep-Alive +Content-Type: text/tab-separated-values; charset=UTF-8 +Transfer-Encoding: chunked +Keep-Alive: timeout=10, max=? + diff --git a/tests/queries/0_stateless/00501_http_head.reference b/tests/queries/0_stateless/00501_http_head.reference index db82132b145..807bcd4922e 100644 --- a/tests/queries/0_stateless/00501_http_head.reference +++ b/tests/queries/0_stateless/00501_http_head.reference @@ -2,11 +2,11 @@ HTTP/1.1 200 OK Connection: Keep-Alive Content-Type: text/tab-separated-values; charset=UTF-8 Transfer-Encoding: chunked -Keep-Alive: timeout=10, max=10000 +Keep-Alive: timeout=10, max=? HTTP/1.1 200 OK Connection: Keep-Alive Content-Type: text/tab-separated-values; charset=UTF-8 Transfer-Encoding: chunked -Keep-Alive: timeout=10, max=10000 +Keep-Alive: timeout=10, max=? diff --git a/tests/queries/0_stateless/00501_http_head.sh b/tests/queries/0_stateless/00501_http_head.sh index 60283f26833..30da64c31f0 100755 --- a/tests/queries/0_stateless/00501_http_head.sh +++ b/tests/queries/0_stateless/00501_http_head.sh @@ -4,8 +4,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -( ${CLICKHOUSE_CURL} -s --head "${CLICKHOUSE_URL}&query=SELECT%201"; - ${CLICKHOUSE_CURL} -s --head "${CLICKHOUSE_URL}&query=select+*+from+system.numbers+limit+1000000" ) | grep -v "Date:" | grep -v "X-ClickHouse-Server-Display-Name:" | grep -v "X-ClickHouse-Query-Id:" | grep -v "X-ClickHouse-Format:" | grep -v "X-ClickHouse-Timezone:" +# the sed command here replaces the real number of left requests with a question mark, because it can vary and we don't really have control over it +( ${CLICKHOUSE_CURL} -s --head "${CLICKHOUSE_URL}&query=SELECT%201" | sed -r 's/(keep-alive: timeout=10, max=)[0-9]+/\1?/I'; + ${CLICKHOUSE_CURL} -s --head "${CLICKHOUSE_URL}&query=select+*+from+system.numbers+limit+1000000" ) | sed -r 's/(keep-alive: timeout=10, max=)[0-9]+/\1?/I' | grep -v "Date:" | grep -v "X-ClickHouse-Server-Display-Name:" | grep -v "X-ClickHouse-Query-Id:" | grep -v "X-ClickHouse-Format:" | grep -v "X-ClickHouse-Timezone:" if [[ $(${CLICKHOUSE_CURL} -sS -X POST -I "${CLICKHOUSE_URL}&query=SELECT+1" | grep -c '411 Length Required') -ne 1 ]]; then echo FAIL From 1f5c4101b2d74d7ccf798621083fb536bf35de18 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 6 Aug 2024 21:54:15 +0100 Subject: [PATCH 347/644] rm redundant file --- tests/queries/0_stateless/00501_http_head.re | 12 ------------ 1 file changed, 12 deletions(-) delete mode 100644 tests/queries/0_stateless/00501_http_head.re diff --git a/tests/queries/0_stateless/00501_http_head.re b/tests/queries/0_stateless/00501_http_head.re deleted file mode 100644 index 807bcd4922e..00000000000 --- a/tests/queries/0_stateless/00501_http_head.re +++ /dev/null @@ -1,12 +0,0 @@ -HTTP/1.1 200 OK -Connection: Keep-Alive -Content-Type: text/tab-separated-values; charset=UTF-8 -Transfer-Encoding: chunked -Keep-Alive: timeout=10, max=? - -HTTP/1.1 200 OK -Connection: Keep-Alive -Content-Type: text/tab-separated-values; charset=UTF-8 -Transfer-Encoding: chunked -Keep-Alive: timeout=10, max=? - From 5b3692b4f02421d56692107365ad4cc7a3297418 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 7 Aug 2024 00:29:19 +0200 Subject: [PATCH 348/644] Added some useful debug logs --- src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp index 1e8164152a5..702d058ee79 100644 --- a/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp +++ b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -529,6 +530,8 @@ namespace ContextMutablePtr insert_context = Context::createCopy(context); insert_context->setCurrentQueryId(context->getCurrentQueryId() + ":" + String{toString(table_kind)}); + LOG_TEST(log, "{}: Executing query: {}", time_series_storage_id.getNameForLogs(), queryToString(insert_query)); + InterpreterInsertQuery interpreter( insert_query, insert_context, From 20c2d346a5db550d38954b5c5b1de2d1a09a884c Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 7 Aug 2024 00:35:25 +0000 Subject: [PATCH 349/644] just add test --- .../MergeTree/MergeTreeIndexBloomFilter.cpp | 26 +------------------ .../00945_bloom_filter_index.reference | 2 ++ .../0_stateless/00945_bloom_filter_index.sql | 11 ++++++++ ..._bloom_filter_not_supported_func.reference | 2 -- .../03215_bloom_filter_not_supported_func.sql | 14 ---------- 5 files changed, 14 insertions(+), 41 deletions(-) delete mode 100644 tests/queries/0_stateless/03215_bloom_filter_not_supported_func.reference delete mode 100644 tests/queries/0_stateless/03215_bloom_filter_not_supported_func.sql diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index 0a4eda3be69..dc314ce53d4 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -366,31 +366,7 @@ bool MergeTreeIndexConditionBloomFilter::extractAtomFromTree(const RPNBuilderTre } } - if (node.isFunction()) - { - /// Similar to the logic of KeyCondition, restrict the usage of bloom filter, in case of func like cast(c=1 or c=9999 as Bool). - const std::unordered_set atom_map - { - "equals", - "notEquals", - "has", - "mapContains", - "indexOf", - "hasAny", - "hasAll", - "in", - "notIn", - "globalIn", - "globalNotIn" - }; - - auto func_name = node.toFunctionNode().getFunctionName(); - if (atom_map.find(func_name) == std::end(atom_map)) - return false; - } - - bool res = traverseFunction(node, out, nullptr /*parent*/); - return res; + return traverseFunction(node, out, nullptr /*parent*/); } bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNode & node, RPNElement & out, const RPNBuilderTreeNode * parent) diff --git a/tests/queries/0_stateless/00945_bloom_filter_index.reference b/tests/queries/0_stateless/00945_bloom_filter_index.reference index e6751fe4762..9d9b49b29c9 100644 --- a/tests/queries/0_stateless/00945_bloom_filter_index.reference +++ b/tests/queries/0_stateless/00945_bloom_filter_index.reference @@ -227,3 +227,5 @@ 1 value1 1 value2 2 value3 +1 +1 diff --git a/tests/queries/0_stateless/00945_bloom_filter_index.sql b/tests/queries/0_stateless/00945_bloom_filter_index.sql index 2b7feacbd98..71109df79e7 100644 --- a/tests/queries/0_stateless/00945_bloom_filter_index.sql +++ b/tests/queries/0_stateless/00945_bloom_filter_index.sql @@ -374,3 +374,14 @@ SELECT id, ary[indexOf(ary, 'value2')] FROM test_bf_indexOf WHERE ary[indexOf(ar SELECT id, ary[indexOf(ary, 'value3')] FROM test_bf_indexOf WHERE ary[indexOf(ary, 'value3')] = 'value3' ORDER BY id FORMAT TSV; DROP TABLE IF EXISTS test_bf_indexOf; + +-- expecting cast function to be unknown +DROP TABLE IF EXISTS test_bf_cast; + +CREATE TABLE test_bf_cast (c Int32, INDEX x1 (c) type bloom_filter) ENGINE = MergeTree ORDER BY c as select 1; + +SELECT count() FROM test_bf_cast WHERE cast(c=1 or c=9999 as Bool) settings use_skip_indexes=0; + +SELECT count() FROM test_bf_cast WHERE cast(c=1 or c=9999 as Bool) settings use_skip_indexes=1; + +DROP TABLE test_bf_cast; \ No newline at end of file diff --git a/tests/queries/0_stateless/03215_bloom_filter_not_supported_func.reference b/tests/queries/0_stateless/03215_bloom_filter_not_supported_func.reference deleted file mode 100644 index 6ed281c757a..00000000000 --- a/tests/queries/0_stateless/03215_bloom_filter_not_supported_func.reference +++ /dev/null @@ -1,2 +0,0 @@ -1 -1 diff --git a/tests/queries/0_stateless/03215_bloom_filter_not_supported_func.sql b/tests/queries/0_stateless/03215_bloom_filter_not_supported_func.sql deleted file mode 100644 index 3d094244892..00000000000 --- a/tests/queries/0_stateless/03215_bloom_filter_not_supported_func.sql +++ /dev/null @@ -1,14 +0,0 @@ -drop table if exists t; - -create table t ( - c Int32, - index x1 (c) type bloom_filter -) engine=MergeTree order by c as select 1; - -SELECT count() FROM t WHERE cast(c=1 or c=9999 as Bool) -settings use_skip_indexes=0; - -SELECT count() FROM t WHERE cast(c=1 or c=9999 as Bool) -settings use_skip_indexes=1; - -drop table t; \ No newline at end of file From 25f557667a90f9805bc22796a8c799f3203019fa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 7 Aug 2024 02:47:53 +0200 Subject: [PATCH 350/644] Change log level in clickhouse-local --- src/Client/LocalConnection.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 072184e0a66..7595a29912b 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -365,7 +365,7 @@ bool LocalConnection::poll(size_t) { while (pollImpl()) { - LOG_DEBUG(&Poco::Logger::get("LocalConnection"), "Executor timeout encountered, will retry"); + LOG_TEST(&Poco::Logger::get("LocalConnection"), "Executor timeout encountered, will retry"); if (needSendProgressOrMetrics()) return true; From 2a5a8f15f4ab5d4322fb09d41ca1e8279197abe4 Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 6 Aug 2024 20:51:44 -0400 Subject: [PATCH 351/644] Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++++ .../02884_create_view_with_sql_security_option.reference | 1 + .../0_stateless/02884_create_view_with_sql_security_option.sh | 2 ++ 3 files changed, 7 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 49888596fbb..5d63f6c94d3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3348,6 +3348,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ALTER MODIFY REFRESH is not supported by MergeTree engines family"); + if (command.type == AlterCommand::MODIFY_SQL_SECURITY) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "ALTER MODIFY SQL SECURITY is not supported by MergeTree engines family"); + if (command.type == AlterCommand::MODIFY_ORDER_BY && !is_custom_partitioned) { throw Exception(ErrorCodes::BAD_ARGUMENTS, diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference index a03343c8cb3..39e7aad87e0 100644 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference @@ -27,6 +27,7 @@ OK OK 100 100 +OK ===== TestGrants ===== OK OK diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh index cc4e76a9ed9..fadbbff7f34 100755 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh @@ -199,6 +199,8 @@ ${CLICKHOUSE_CLIENT} --user $user2 --query "INSERT INTO source SELECT * FROM gen ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM destination1" ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM destination2" +(( $(${CLICKHOUSE_CLIENT} --query "ALTER TABLE test_table MODIFY SQL SECURITY INVOKER" 2>&1 | grep -c "is not supported") >= 1 )) && echo "OK" || echo "UNEXPECTED" + echo "===== TestGrants =====" ${CLICKHOUSE_CLIENT} --query "GRANT CREATE ON *.* TO $user1" ${CLICKHOUSE_CLIENT} --query "GRANT SELECT ON $db.test_table TO $user1, $user2" From 114284bdcea7e4a769f2b9c004c5092cbb323550 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 2 Aug 2024 15:20:49 +0800 Subject: [PATCH 352/644] fixed --- src/AggregateFunctions/WindowFunction.h | 117 ++ src/Planner/PlannerActionsVisitor.cpp | 75 +- src/Planner/PlannerActionsVisitor.h | 6 + src/Planner/PlannerWindowFunctions.cpp | 40 +- src/Processors/Transforms/WindowTransform.cpp | 1018 +++++++---------- src/Processors/Transforms/WindowTransform.h | 25 +- 6 files changed, 615 insertions(+), 666 deletions(-) create mode 100644 src/AggregateFunctions/WindowFunction.h diff --git a/src/AggregateFunctions/WindowFunction.h b/src/AggregateFunctions/WindowFunction.h new file mode 100644 index 00000000000..f7fbd7389ea --- /dev/null +++ b/src/AggregateFunctions/WindowFunction.h @@ -0,0 +1,117 @@ +#pragma once +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} +class WindowTransform; + + +// Interface for true window functions. It's not much of an interface, they just +// accept the guts of WindowTransform and do 'something'. Given a small number of +// true window functions, and the fact that the WindowTransform internals are +// pretty much well-defined in domain terms (e.g. frame boundaries), this is +// somewhat acceptable. +class IWindowFunction +{ +public: + virtual ~IWindowFunction() = default; + + // Must insert the result for current_row. + virtual void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const = 0; + + virtual std::optional getDefaultFrame() const { return {}; } + + virtual ColumnPtr castColumn(const Columns &, const std::vector &) { return nullptr; } + + /// Is the frame type supported by this function. + virtual bool checkWindowFrameType(const WindowTransform * /*transform*/) const { return true; } +}; + +// Runtime data for computing one window function. +struct WindowFunctionWorkspace +{ + AggregateFunctionPtr aggregate_function; + + // Cached value of aggregate function isState virtual method + bool is_aggregate_function_state = false; + + // This field is set for pure window functions. When set, we ignore the + // window_function.aggregate_function, and work through this interface + // instead. + IWindowFunction * window_function_impl = nullptr; + + std::vector argument_column_indices; + + // Will not be initialized for a pure window function. + mutable AlignedBuffer aggregate_function_state; + + // Argument columns. Be careful, this is a per-block cache. + std::vector argument_columns; + UInt64 cached_block_number = std::numeric_limits::max(); +}; + +// A basic implementation for a true window function. It pretends to be an +// aggregate function, but refuses to work as such. +struct WindowFunction : public IAggregateFunctionHelper, public IWindowFunction +{ + std::string name; + + WindowFunction( + const std::string & name_, const DataTypes & argument_types_, const Array & parameters_, const DataTypePtr & result_type_) + : IAggregateFunctionHelper(argument_types_, parameters_, result_type_), name(name_) + { + } + + bool isOnlyWindowFunction() const override { return true; } + + [[noreturn]] void fail() const + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "The function '{}' can only be used as a window function, not as an aggregate function", getName()); + } + + String getName() const override { return name; } + void create(AggregateDataPtr __restrict) const override { } + void destroy(AggregateDataPtr __restrict) const noexcept override { } + bool hasTrivialDestructor() const override { return true; } + size_t sizeOfData() const override { return 0; } + size_t alignOfData() const override { return 1; } + void add(AggregateDataPtr __restrict, const IColumn **, size_t, Arena *) const override { fail(); } + void merge(AggregateDataPtr __restrict, ConstAggregateDataPtr, Arena *) const override { fail(); } + void serialize(ConstAggregateDataPtr __restrict, WriteBuffer &, std::optional) const override { fail(); } + void deserialize(AggregateDataPtr __restrict, ReadBuffer &, std::optional, Arena *) const override { fail(); } + void insertResultInto(AggregateDataPtr __restrict, IColumn &, Arena *) const override { fail(); } +}; + +template +struct StatefulWindowFunction : public WindowFunction +{ + StatefulWindowFunction( + const std::string & name_, const DataTypes & argument_types_, const Array & parameters_, const DataTypePtr & result_type_) + : WindowFunction(name_, argument_types_, parameters_, result_type_) + { + } + + size_t sizeOfData() const override { return sizeof(State); } + size_t alignOfData() const override { return 1; } + + void create(AggregateDataPtr __restrict place) const override { new (place) State(); } + + void destroy(AggregateDataPtr __restrict place) const noexcept override { reinterpret_cast(place)->~State(); } + + bool hasTrivialDestructor() const override { return std::is_trivially_destructible_v; } + + State & getState(const WindowFunctionWorkspace & workspace) const + { + return *reinterpret_cast(workspace.aggregate_function_state.data()); + } +}; + +} diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 57457493844..99b9c3f7482 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -236,8 +236,16 @@ public: if (function_node.isWindowFunction()) { + auto get_window_frame = [&]() -> std::optional + { + auto & window_node = function_node.getWindowNode()->as(); + auto & window_frame = window_node.getWindowFrame(); + if (!window_frame.is_default) + return window_frame; + return {}; + }; buffer << " OVER ("; - buffer << calculateWindowNodeActionName(function_node.getWindowNode()); + buffer << calculateWindowNodeActionName(function_node.getWindowNode(), get_window_frame); buffer << ')'; } @@ -298,7 +306,7 @@ public: return calculateConstantActionNodeName(constant_literal, applyVisitor(FieldToDataType(), constant_literal)); } - String calculateWindowNodeActionName(const QueryTreeNodePtr & node) + String calculateWindowNodeActionName(const QueryTreeNodePtr & node, std::function()> get_window_frame) { auto & window_node = node->as(); WriteBufferFromOwnString buffer; @@ -364,44 +372,14 @@ public: } } - auto & window_frame = window_node.getWindowFrame(); - if (!window_frame.is_default) + auto window_frame_opt = get_window_frame(); + if (window_frame_opt) { + auto & window_frame = *window_frame_opt; if (window_node.hasPartitionBy() || window_node.hasOrderBy()) buffer << ' '; - buffer << window_frame.type << " BETWEEN "; - if (window_frame.begin_type == WindowFrame::BoundaryType::Current) - { - buffer << "CURRENT ROW"; - } - else if (window_frame.begin_type == WindowFrame::BoundaryType::Unbounded) - { - buffer << "UNBOUNDED"; - buffer << " " << (window_frame.begin_preceding ? "PRECEDING" : "FOLLOWING"); - } - else - { - buffer << calculateActionNodeName(window_node.getFrameBeginOffsetNode()); - buffer << " " << (window_frame.begin_preceding ? "PRECEDING" : "FOLLOWING"); - } - - buffer << " AND "; - - if (window_frame.end_type == WindowFrame::BoundaryType::Current) - { - buffer << "CURRENT ROW"; - } - else if (window_frame.end_type == WindowFrame::BoundaryType::Unbounded) - { - buffer << "UNBOUNDED"; - buffer << " " << (window_frame.end_preceding ? "PRECEDING" : "FOLLOWING"); - } - else - { - buffer << calculateActionNodeName(window_node.getFrameEndOffsetNode()); - buffer << " " << (window_frame.end_preceding ? "PRECEDING" : "FOLLOWING"); - } + window_frame.toString(buffer); } return buffer.str(); @@ -1062,14 +1040,35 @@ String calculateWindowNodeActionName(const QueryTreeNodePtr & node, bool use_column_identifier_as_action_node_name) { ActionNodeNameHelper helper(node_to_name, planner_context, use_column_identifier_as_action_node_name); - return helper.calculateWindowNodeActionName(node); + auto get_window_frame = [&]()-> std::optional{ + auto & window_node = node->as(); + auto & window_frame = window_node.getWindowFrame(); + if (!window_frame.is_default) + return window_frame; + return {}; + }; + return helper.calculateWindowNodeActionName(node, get_window_frame); } String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, bool use_column_identifier_as_action_node_name) { QueryTreeNodeToName empty_map; ActionNodeNameHelper helper(empty_map, planner_context, use_column_identifier_as_action_node_name); - return helper.calculateWindowNodeActionName(node); + auto get_window_frame = [&]()-> std::optional{ + auto & window_node = node->as(); + auto & window_frame = window_node.getWindowFrame(); + if (!window_frame.is_default) + return window_frame; + return {}; + }; + return helper.calculateWindowNodeActionName(node, get_window_frame); +} + +String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, std::function()> get_window_frame, bool use_column_identifier_as_action_node_name) +{ + QueryTreeNodeToName empty_map; + ActionNodeNameHelper helper(empty_map, planner_context, use_column_identifier_as_action_node_name); + return helper.calculateWindowNodeActionName(node, get_window_frame); } } diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 6bb32047327..78d7c69357a 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -8,6 +9,7 @@ #include #include +#include namespace DB { @@ -85,5 +87,9 @@ String calculateWindowNodeActionName(const QueryTreeNodePtr & node, String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, bool use_column_identifier_as_action_node_name = true); +String calculateWindowNodeActionName(const QueryTreeNodePtr & node, + const PlannerContext & planner_context, + std::function()> get_window_frame, + bool use_column_identifier_as_action_node_name = true); } diff --git a/src/Planner/PlannerWindowFunctions.cpp b/src/Planner/PlannerWindowFunctions.cpp index 225852de5a7..7d0fc3a85b3 100644 --- a/src/Planner/PlannerWindowFunctions.cpp +++ b/src/Planner/PlannerWindowFunctions.cpp @@ -1,5 +1,7 @@ +#include #include +#include #include #include #include @@ -8,41 +10,60 @@ #include -#include #include +#include namespace DB { namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; +extern const int NOT_IMPLEMENTED; } namespace { -WindowDescription extractWindowDescriptionFromWindowNode(const QueryTreeNodePtr & node, const PlannerContext & planner_context) +//WindowDescription extractWindowDescriptionFromWindowNode(const QueryTreeNodePtr & node, const PlannerContext & planner_context) +WindowDescription extractWindowDescriptionFromWindowNode(const FunctionNode & func_node, const PlannerContext & planner_context) { + auto node = func_node.getWindowNode(); auto & window_node = node->as(); + auto get_window_frame = [&]() -> std::optional + { + auto frame = window_node.getWindowFrame(); + if (!frame.is_default) + return frame; + auto aggregate_function = func_node.getAggregateFunction(); + if (const auto * win_func = dynamic_cast(aggregate_function.get())) + { + return win_func->getDefaultFrame(); + } + return {}; + }; + WindowDescription window_description; - window_description.window_name = calculateWindowNodeActionName(node, planner_context); + window_description.window_name = calculateWindowNodeActionName(node, planner_context, get_window_frame); for (const auto & partition_by_node : window_node.getPartitionBy().getNodes()) { auto partition_by_node_action_name = calculateActionNodeName(partition_by_node, planner_context); - auto partition_by_sort_column_description = SortColumnDescription(partition_by_node_action_name, 1 /* direction */, 1 /* nulls_direction */); + auto partition_by_sort_column_description + = SortColumnDescription(partition_by_node_action_name, 1 /* direction */, 1 /* nulls_direction */); window_description.partition_by.push_back(std::move(partition_by_sort_column_description)); } window_description.order_by = extractSortDescription(window_node.getOrderByNode(), planner_context); window_description.full_sort_description = window_description.partition_by; - window_description.full_sort_description.insert(window_description.full_sort_description.end(), window_description.order_by.begin(), window_description.order_by.end()); + window_description.full_sort_description.insert( + window_description.full_sort_description.end(), window_description.order_by.begin(), window_description.order_by.end()); /// WINDOW frame is validated during query analysis stage - window_description.frame = window_node.getWindowFrame(); + auto window_frame = get_window_frame(); + window_description.frame = window_frame ? *window_frame : window_node.getWindowFrame(); + auto node_frame = window_node.getWindowFrame(); const auto & query_context = planner_context.getQueryContext(); const auto & query_context_settings = query_context->getSettingsRef(); @@ -64,7 +85,8 @@ WindowDescription extractWindowDescriptionFromWindowNode(const QueryTreeNodePtr } -std::vector extractWindowDescriptions(const QueryTreeNodes & window_function_nodes, const PlannerContext & planner_context) +std::vector +extractWindowDescriptions(const QueryTreeNodes & window_function_nodes, const PlannerContext & planner_context) { std::unordered_map window_name_to_description; @@ -72,7 +94,7 @@ std::vector extractWindowDescriptions(const QueryTreeNodes & { auto & window_function_node_typed = window_function_node->as(); - auto function_window_description = extractWindowDescriptionFromWindowNode(window_function_node_typed.getWindowNode(), planner_context); + auto function_window_description = extractWindowDescriptionFromWindowNode(window_function_node_typed, planner_context); auto frame_type = function_window_description.frame.type; if (frame_type != WindowFrame::FrameType::ROWS && frame_type != WindowFrame::FrameType::RANGE) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 8de248a9c95..ae9a94bff90 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -6,9 +6,12 @@ #include #include #include +#include #include #include +#include #include +#include #include #include #include @@ -16,9 +19,6 @@ #include #include #include -#include -#include -#include #include #include @@ -57,52 +57,31 @@ struct Settings; namespace ErrorCodes { - extern const int BAD_ARGUMENTS; - extern const int NOT_IMPLEMENTED; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; - extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; +extern const int BAD_ARGUMENTS; +extern const int NOT_IMPLEMENTED; +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; +extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } -// Interface for true window functions. It's not much of an interface, they just -// accept the guts of WindowTransform and do 'something'. Given a small number of -// true window functions, and the fact that the WindowTransform internals are -// pretty much well-defined in domain terms (e.g. frame boundaries), this is -// somewhat acceptable. -class IWindowFunction -{ -public: - virtual ~IWindowFunction() = default; - - // Must insert the result for current_row. - virtual void windowInsertResultInto(const WindowTransform * transform, - size_t function_index) const = 0; - - virtual std::optional getDefaultFrame() const { return {}; } - - virtual ColumnPtr castColumn(const Columns &, const std::vector &) { return nullptr; } - - /// Is the frame type supported by this function. - virtual bool checkWindowFrameType(const WindowTransform * /*transform*/) const { return true; } -}; // Compares ORDER BY column values at given rows to find the boundaries of frame: // [compared] with [reference] +/- offset. Return value is -1/0/+1, like in // sorting predicates -- -1 means [compared] is less than [reference] +/- offset. template -static int compareValuesWithOffset(const IColumn * _compared_column, - size_t compared_row, const IColumn * _reference_column, +static int compareValuesWithOffset( + const IColumn * _compared_column, + size_t compared_row, + const IColumn * _reference_column, size_t reference_row, const Field & _offset, bool offset_is_preceding) { // Casting the columns to the known type here makes it faster, probably // because the getData call can be devirtualized. - const auto * compared_column = assert_cast( - _compared_column); - const auto * reference_column = assert_cast( - _reference_column); + const auto * compared_column = assert_cast(_compared_column); + const auto * reference_column = assert_cast(_reference_column); using ValueType = typename ColumnType::ValueType; // Note that the storage type of offset returned by get<> is different, so @@ -112,13 +91,11 @@ static int compareValuesWithOffset(const IColumn * _compared_column, const auto compared_value_data = compared_column->getDataAt(compared_row); assert(compared_value_data.size == sizeof(ValueType)); - auto compared_value = unalignedLoad( - compared_value_data.data); + auto compared_value = unalignedLoad(compared_value_data.data); const auto reference_value_data = reference_column->getDataAt(reference_row); assert(reference_value_data.size == sizeof(ValueType)); - auto reference_value = unalignedLoad( - reference_value_data.data); + auto reference_value = unalignedLoad(reference_value_data.data); bool is_overflow; if (offset_is_preceding) @@ -143,37 +120,34 @@ static int compareValuesWithOffset(const IColumn * _compared_column, else { // No overflow, compare normally. - return compared_value < reference_value ? -1 - : compared_value == reference_value ? 0 : 1; + return compared_value < reference_value ? -1 : compared_value == reference_value ? 0 : 1; } } // A specialization of compareValuesWithOffset for floats. template -static int compareValuesWithOffsetFloat(const IColumn * _compared_column, - size_t compared_row, const IColumn * _reference_column, +static int compareValuesWithOffsetFloat( + const IColumn * _compared_column, + size_t compared_row, + const IColumn * _reference_column, size_t reference_row, const Field & _offset, bool offset_is_preceding) { // Casting the columns to the known type here makes it faster, probably // because the getData call can be devirtualized. - const auto * compared_column = assert_cast( - _compared_column); - const auto * reference_column = assert_cast( - _reference_column); + const auto * compared_column = assert_cast(_compared_column); + const auto * reference_column = assert_cast(_reference_column); const auto offset = _offset.get(); chassert(offset >= 0); const auto compared_value_data = compared_column->getDataAt(compared_row); assert(compared_value_data.size == sizeof(typename ColumnType::ValueType)); - auto compared_value = unalignedLoad( - compared_value_data.data); + auto compared_value = unalignedLoad(compared_value_data.data); const auto reference_value_data = reference_column->getDataAt(reference_row); assert(reference_value_data.size == sizeof(typename ColumnType::ValueType)); - auto reference_value = unalignedLoad( - reference_value_data.data); + auto reference_value = unalignedLoad(reference_value_data.data); /// Floats overflow to Inf and the comparison will work normally, so we don't have to do anything. if (offset_is_preceding) @@ -181,58 +155,58 @@ static int compareValuesWithOffsetFloat(const IColumn * _compared_column, else reference_value += static_cast(offset); - const auto result = compared_value < reference_value ? -1 - : (compared_value == reference_value ? 0 : 1); + const auto result = compared_value < reference_value ? -1 : (compared_value == reference_value ? 0 : 1); return result; } // Helper macros to dispatch on type of the ORDER BY column #define APPLY_FOR_ONE_NEST_TYPE(FUNCTION, TYPE) \ -else if (typeid_cast(nest_compared_column.get())) \ -{ \ - /* clang-tidy you're dumb, I can't put FUNCTION in braces here. */ \ - nest_compare_function = FUNCTION; /* NOLINT */ \ -} + else if (typeid_cast(nest_compared_column.get())) \ + { \ + /* clang-tidy you're dumb, I can't put FUNCTION in braces here. */ \ + nest_compare_function = FUNCTION; /* NOLINT */ \ + } #define APPLY_FOR_NEST_TYPES(FUNCTION) \ -if (false) /* NOLINT */ \ -{ \ - /* Do nothing, a starter condition. */ \ -} \ -APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ -APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ -APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ -APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ + if (false) /* NOLINT */ \ + { \ + /* Do nothing, a starter condition. */ \ + } \ + APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ \ -APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ -APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ -APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ -APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ -APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ \ -APPLY_FOR_ONE_NEST_TYPE(FUNCTION##Float, ColumnVector) \ -APPLY_FOR_ONE_NEST_TYPE(FUNCTION##Float, ColumnVector) \ + APPLY_FOR_ONE_NEST_TYPE(FUNCTION##Float, ColumnVector) \ + APPLY_FOR_ONE_NEST_TYPE(FUNCTION##Float, ColumnVector) \ \ -else \ -{ \ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, \ - "The RANGE OFFSET frame for '{}' ORDER BY nest column is not implemented", \ - demangle(typeid(nest_compared_column).name())); \ -} + else \ + { \ + throw Exception( \ + ErrorCodes::NOT_IMPLEMENTED, \ + "The RANGE OFFSET frame for '{}' ORDER BY nest column is not implemented", \ + demangle(typeid(nest_compared_column).name())); \ + } // A specialization of compareValuesWithOffset for nullable. template -static int compareValuesWithOffsetNullable(const IColumn * _compared_column, - size_t compared_row, const IColumn * _reference_column, +static int compareValuesWithOffsetNullable( + const IColumn * _compared_column, + size_t compared_row, + const IColumn * _reference_column, size_t reference_row, const Field & _offset, bool offset_is_preceding) { - const auto * compared_column = assert_cast( - _compared_column); - const auto * reference_column = assert_cast( - _reference_column); + const auto * compared_column = assert_cast(_compared_column); + const auto * reference_column = assert_cast(_reference_column); if (compared_column->isNullAt(compared_row) && !reference_column->isNullAt(reference_row)) { @@ -251,54 +225,59 @@ static int compareValuesWithOffsetNullable(const IColumn * _compared_column, ColumnPtr nest_reference_column = reference_column->getNestedColumnPtr(); std::function nest_compare_function; + bool offset_is_preceding)> + nest_compare_function; APPLY_FOR_NEST_TYPES(compareValuesWithOffset) - return nest_compare_function(nest_compared_column.get(), compared_row, - nest_reference_column.get(), reference_row, _offset, offset_is_preceding); + return nest_compare_function( + nest_compared_column.get(), compared_row, nest_reference_column.get(), reference_row, _offset, offset_is_preceding); } // Helper macros to dispatch on type of the ORDER BY column #define APPLY_FOR_ONE_TYPE(FUNCTION, TYPE) \ -else if (typeid_cast(column)) \ -{ \ - /* clang-tidy you're dumb, I can't put FUNCTION in braces here. */ \ - compare_values_with_offset = FUNCTION; /* NOLINT */ \ -} + else if (typeid_cast(column)) \ + { \ + /* clang-tidy you're dumb, I can't put FUNCTION in braces here. */ \ + compare_values_with_offset = FUNCTION; /* NOLINT */ \ + } #define APPLY_FOR_TYPES(FUNCTION) \ -if (false) /* NOLINT */ \ -{ \ - /* Do nothing, a starter condition. */ \ -} \ -APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ -APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ -APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ -APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ + if (false) /* NOLINT */ \ + { \ + /* Do nothing, a starter condition. */ \ + } \ + APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ \ -APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ -APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ -APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ -APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ -APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ + APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ \ -APPLY_FOR_ONE_TYPE(FUNCTION##Float, ColumnVector) \ -APPLY_FOR_ONE_TYPE(FUNCTION##Float, ColumnVector) \ + APPLY_FOR_ONE_TYPE(FUNCTION##Float, ColumnVector) \ + APPLY_FOR_ONE_TYPE(FUNCTION##Float, ColumnVector) \ \ -APPLY_FOR_ONE_TYPE(FUNCTION##Nullable, ColumnNullable) \ -else \ -{ \ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, \ - "The RANGE OFFSET frame for '{}' ORDER BY column is not implemented", \ - demangle(typeid(*column).name())); \ -} + APPLY_FOR_ONE_TYPE(FUNCTION##Nullable, ColumnNullable) \ + else \ + { \ + throw Exception( \ + ErrorCodes::NOT_IMPLEMENTED, \ + "The RANGE OFFSET frame for '{}' ORDER BY column is not implemented", \ + demangle(typeid(*column).name())); \ + } -WindowTransform::WindowTransform(const Block & input_header_, - const Block & output_header_, - const WindowDescription & window_description_, - const std::vector & functions) +WindowTransform::WindowTransform( + const Block & input_header_, + const Block & output_header_, + const WindowDescription & window_description_, + const std::vector & functions) : IProcessor({input_header_}, {output_header_}) , input(inputs.front()) , output(outputs.front()) @@ -329,8 +308,7 @@ WindowTransform::WindowTransform(const Block & input_header_, workspace.argument_column_indices.reserve(f.argument_names.size()); for (const auto & argument_name : f.argument_names) { - workspace.argument_column_indices.push_back( - input_header.getPositionByName(argument_name)); + workspace.argument_column_indices.push_back(input_header.getPositionByName(argument_name)); } workspace.argument_columns.assign(f.argument_names.size(), nullptr); @@ -347,9 +325,7 @@ WindowTransform::WindowTransform(const Block & input_header_, } workspace.is_aggregate_function_state = workspace.aggregate_function->isState(); - workspace.aggregate_function_state.reset( - aggregate_function->sizeOfData(), - aggregate_function->alignOfData()); + workspace.aggregate_function_state.reset(aggregate_function->sizeOfData(), aggregate_function->alignOfData()); aggregate_function->create(workspace.aggregate_function_state.data()); workspaces.push_back(std::move(workspace)); @@ -358,24 +334,20 @@ WindowTransform::WindowTransform(const Block & input_header_, partition_by_indices.reserve(window_description.partition_by.size()); for (const auto & column : window_description.partition_by) { - partition_by_indices.push_back( - input_header.getPositionByName(column.column_name)); + partition_by_indices.push_back(input_header.getPositionByName(column.column_name)); } order_by_indices.reserve(window_description.order_by.size()); for (const auto & column : window_description.order_by) { - order_by_indices.push_back( - input_header.getPositionByName(column.column_name)); + order_by_indices.push_back(input_header.getPositionByName(column.column_name)); } // Choose a row comparison function for RANGE OFFSET frame based on the // type of the ORDER BY column. if (window_description.frame.type == WindowFrame::FrameType::RANGE - && (window_description.frame.begin_type - == WindowFrame::BoundaryType::Offset - || window_description.frame.end_type - == WindowFrame::BoundaryType::Offset)) + && (window_description.frame.begin_type == WindowFrame::BoundaryType::Offset + || window_description.frame.end_type == WindowFrame::BoundaryType::Offset)) { assert(order_by_indices.size() == 1); const auto & entry = input_header.getByPosition(order_by_indices[0]); @@ -385,32 +357,26 @@ WindowTransform::WindowTransform(const Block & input_header_, // Convert the offsets to the ORDER BY column type. We can't just check // that the type matches, because e.g. the int literals are always // (U)Int64, but the column might be Int8 and so on. - if (window_description.frame.begin_type - == WindowFrame::BoundaryType::Offset) + if (window_description.frame.begin_type == WindowFrame::BoundaryType::Offset) { - window_description.frame.begin_offset = convertFieldToTypeOrThrow( - window_description.frame.begin_offset, - *entry.type); + window_description.frame.begin_offset = convertFieldToTypeOrThrow(window_description.frame.begin_offset, *entry.type); - if (applyVisitor(FieldVisitorAccurateLess{}, - window_description.frame.begin_offset, Field(0))) + if (applyVisitor(FieldVisitorAccurateLess{}, window_description.frame.begin_offset, Field(0))) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Window frame start offset must be nonnegative, {} given", window_description.frame.begin_offset); } } - if (window_description.frame.end_type - == WindowFrame::BoundaryType::Offset) + if (window_description.frame.end_type == WindowFrame::BoundaryType::Offset) { - window_description.frame.end_offset = convertFieldToTypeOrThrow( - window_description.frame.end_offset, - *entry.type); + window_description.frame.end_offset = convertFieldToTypeOrThrow(window_description.frame.end_offset, *entry.type); - if (applyVisitor(FieldVisitorAccurateLess{}, - window_description.frame.end_offset, Field(0))) + if (applyVisitor(FieldVisitorAccurateLess{}, window_description.frame.end_offset, Field(0))) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Window frame start offset must be nonnegative, {} given", window_description.frame.end_offset); } @@ -423,11 +389,10 @@ WindowTransform::WindowTransform(const Block & input_header_, { if (!workspace.window_function_impl->checkWindowFrameType(this)) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported window frame type for function '{}'", - workspace.aggregate_function->getName()); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Unsupported window frame type for function '{}'", workspace.aggregate_function->getName()); } } - } } @@ -436,8 +401,7 @@ WindowTransform::~WindowTransform() // Some states may be not created yet if the creation failed. for (auto & ws : workspaces) { - ws.aggregate_function->destroy( - ws.aggregate_function_state.data()); + ws.aggregate_function->destroy(ws.aggregate_function_state.data()); } } @@ -511,14 +475,10 @@ void WindowTransform::advancePartitionEnd() size_t i = 0; for (; i < partition_by_columns; ++i) { - const auto * reference_column - = inputAt(prev_frame_start)[partition_by_indices[i]].get(); - const auto * compared_column - = inputAt(partition_end)[partition_by_indices[i]].get(); + const auto * reference_column = inputAt(prev_frame_start)[partition_by_indices[i]].get(); + const auto * compared_column = inputAt(partition_end)[partition_by_indices[i]].get(); - if (compared_column->compareAt(partition_end.row, - prev_frame_start.row, *reference_column, - 1 /* nan_direction_hint */) != 0) + if (compared_column->compareAt(partition_end.row, prev_frame_start.row, *reference_column, 1 /* nan_direction_hint */) != 0) { break; } @@ -630,9 +590,8 @@ auto WindowTransform::moveRowNumber(const RowNumber & original_row_number, Int64 void WindowTransform::advanceFrameStartRowsOffset() { // Just recalculate it each time by walking blocks. - const auto [moved_row, offset_left] = moveRowNumber(current_row, - window_description.frame.begin_offset.get() - * (window_description.frame.begin_preceding ? -1 : 1)); + const auto [moved_row, offset_left] = moveRowNumber( + current_row, window_description.frame.begin_offset.get() * (window_description.frame.begin_preceding ? -1 : 1)); frame_start = moved_row; @@ -669,21 +628,17 @@ void WindowTransform::advanceFrameStartRangeOffset() { // See the comment for advanceFrameEndRangeOffset(). const int direction = window_description.order_by[0].direction; - const bool preceding = window_description.frame.begin_preceding - == (direction > 0); - const auto * reference_column - = inputAt(current_row)[order_by_indices[0]].get(); + const bool preceding = window_description.frame.begin_preceding == (direction > 0); + const auto * reference_column = inputAt(current_row)[order_by_indices[0]].get(); for (; frame_start < partition_end; advanceRowNumber(frame_start)) { // The first frame value is [current_row] with offset, so we advance // while [frames_start] < [current_row] with offset. - const auto * compared_column - = inputAt(frame_start)[order_by_indices[0]].get(); - if (compare_values_with_offset(compared_column, frame_start.row, - reference_column, current_row.row, - window_description.frame.begin_offset, - preceding) - * direction >= 0) + const auto * compared_column = inputAt(frame_start)[order_by_indices[0]].get(); + if (compare_values_with_offset( + compared_column, frame_start.row, reference_column, current_row.row, window_description.frame.begin_offset, preceding) + * direction + >= 0) { frame_started = true; return; @@ -728,7 +683,8 @@ void WindowTransform::advanceFrameStart() advanceFrameStartRangeOffset(); break; default: - throw Exception(ErrorCodes::NOT_IMPLEMENTED, + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Frame start type '{}' for frame '{}' is not implemented", window_description.frame.begin_type, window_description.frame.type); @@ -788,8 +744,7 @@ bool WindowTransform::arePeers(const RowNumber & x, const RowNumber & y) const { const auto * column_x = inputAt(x)[order_by_indices[i]].get(); const auto * column_y = inputAt(y)[order_by_indices[i]].get(); - if (column_x->compareAt(x.row, y.row, *column_y, - 1 /* nan_direction_hint */) != 0) + if (column_x->compareAt(x.row, y.row, *column_y, 1 /* nan_direction_hint */) != 0) { return false; } @@ -806,8 +761,7 @@ void WindowTransform::advanceFrameEndCurrentRow() // (only loop over rows and not over blocks), that should hopefully be more // efficient. // partition_end is either in this new block or past-the-end. - assert(frame_end.block == partition_end.block - || frame_end.block + 1 == partition_end.block); + assert(frame_end.block == partition_end.block || frame_end.block + 1 == partition_end.block); if (frame_end == partition_end) { @@ -869,10 +823,8 @@ void WindowTransform::advanceFrameEndRowsOffset() { // Walk the specified offset from the current row. The "+1" is needed // because the frame_end is a past-the-end pointer. - const auto [moved_row, offset_left] = moveRowNumber(current_row, - window_description.frame.end_offset.get() - * (window_description.frame.end_preceding ? -1 : 1) - + 1); + const auto [moved_row, offset_left] = moveRowNumber( + current_row, window_description.frame.end_offset.get() * (window_description.frame.end_preceding ? -1 : 1) + 1); if (partition_end <= moved_row) { @@ -905,22 +857,18 @@ void WindowTransform::advanceFrameEndRangeOffset() // PRECEDING/FOLLOWING change direction for DESC order. // See CD 9075-2:201?(E) 7.14 p. 429. const int direction = window_description.order_by[0].direction; - const bool preceding = window_description.frame.end_preceding - == (direction > 0); - const auto * reference_column - = inputAt(current_row)[order_by_indices[0]].get(); + const bool preceding = window_description.frame.end_preceding == (direction > 0); + const auto * reference_column = inputAt(current_row)[order_by_indices[0]].get(); for (; frame_end < partition_end; advanceRowNumber(frame_end)) { // The last frame value is current_row with offset, and we need a // past-the-end pointer, so we advance while // [frame_end] <= [current_row] with offset. - const auto * compared_column - = inputAt(frame_end)[order_by_indices[0]].get(); - if (compare_values_with_offset(compared_column, frame_end.row, - reference_column, current_row.row, - window_description.frame.end_offset, - preceding) - * direction > 0) + const auto * compared_column = inputAt(frame_end)[order_by_indices[0]].get(); + if (compare_values_with_offset( + compared_column, frame_end.row, reference_column, current_row.row, window_description.frame.end_offset, preceding) + * direction + > 0) { frame_ended = true; return; @@ -955,9 +903,8 @@ void WindowTransform::advanceFrameEnd() advanceFrameEndRangeOffset(); break; default: - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "The frame end type '{}' is not implemented", - window_description.frame.end_type); + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "The frame end type '{}' is not implemented", window_description.frame.end_type); } break; } @@ -1028,13 +975,9 @@ void WindowTransform::updateAggregationState() // rows manually, instead of using advanceRowNumber(). // For this purpose, the past-the-end block can be different than the // block of the past-the-end row (it's usually the next block). - const auto past_the_end_block = rows_to_add_end.row == 0 - ? rows_to_add_end.block - : rows_to_add_end.block + 1; + const auto past_the_end_block = rows_to_add_end.row == 0 ? rows_to_add_end.block : rows_to_add_end.block + 1; - for (auto block_number = rows_to_add_start.block; - block_number < past_the_end_block; - ++block_number) + for (auto block_number = rows_to_add_start.block; block_number < past_the_end_block; ++block_number) { auto & block = blockAt(block_number); @@ -1042,18 +985,15 @@ void WindowTransform::updateAggregationState() { for (size_t i = 0; i < ws.argument_column_indices.size(); ++i) { - ws.argument_columns[i] = block.input_columns[ - ws.argument_column_indices[i]].get(); + ws.argument_columns[i] = block.input_columns[ws.argument_column_indices[i]].get(); } ws.cached_block_number = block_number; } // First and last blocks may be processed partially, and other blocks // are processed in full. - const auto first_row = block_number == rows_to_add_start.block - ? rows_to_add_start.row : 0; - const auto past_the_end_row = block_number == rows_to_add_end.block - ? rows_to_add_end.row : block.rows; + const auto first_row = block_number == rows_to_add_start.block ? rows_to_add_start.row : 0; + const auto past_the_end_row = block_number == rows_to_add_end.block ? rows_to_add_end.row : block.rows; // We should add an addBatch analog that can accept a starting offset. // For now, add the values one by one. @@ -1101,8 +1041,7 @@ void WindowTransform::writeOutCurrentRow() } } -static void assertSameColumns(const Columns & left_all, - const Columns & right_all) +static void assertSameColumns(const Columns & left_all, const Columns & right_all) { assert(left_all.size() == right_all.size()); @@ -1120,8 +1059,7 @@ static void assertSameColumns(const Columns & left_all, if (const auto * right_lc = typeid_cast(right_column)) right_column = right_lc->getDictionary().getNestedColumn().get(); - assert(typeid(*left_column).hash_code() - == typeid(*right_column).hash_code()); + assert(typeid(*left_column).hash_code() == typeid(*right_column).hash_code()); if (isColumnConst(*left_column)) { @@ -1182,8 +1120,7 @@ void WindowTransform::appendChunk(Chunk & chunk) if (ws.window_function_impl) block.casted_columns.push_back(ws.window_function_impl->castColumn(block.input_columns, ws.argument_column_indices)); - block.output_columns.push_back(ws.aggregate_function->getResultType() - ->createColumn()); + block.output_columns.push_back(ws.aggregate_function->getResultType()->createColumn()); block.output_columns.back()->reserve(block.rows); } @@ -1404,6 +1341,13 @@ IProcessor::Status WindowTransform::prepare() { // Output the ready block. const auto i = next_output_block_number - first_block_number; + LOG_ERROR( + getLogger("WindowTransform"), + "xxx {} output block: {}, next_output_block_number: {} first_not_ready_row.block: {}", + fmt::ptr(this), + i, + next_output_block_number, + first_not_ready_row.block); auto & block = blocks[i]; auto columns = block.original_input_columns; for (auto & res : block.output_columns) @@ -1507,12 +1451,10 @@ void WindowTransform::work() // that the frame start can be further than current row for some frame specs // (e.g. EXCLUDE CURRENT ROW), so we have to check both. assert(prev_frame_start <= frame_start); - const auto first_used_block = std::min(next_output_block_number, - std::min(prev_frame_start.block, current_row.block)); + const auto first_used_block = std::min(next_output_block_number, std::min(prev_frame_start.block, current_row.block)); if (first_block_number < first_used_block) { - blocks.erase(blocks.begin(), - blocks.begin() + (first_used_block - first_block_number)); + blocks.erase(blocks.begin(), blocks.begin() + (first_used_block - first_block_number)); first_block_number = first_used_block; assert(next_output_block_number >= first_block_number); @@ -1523,118 +1465,82 @@ void WindowTransform::work() } } -// A basic implementation for a true window function. It pretends to be an -// aggregate function, but refuses to work as such. -struct WindowFunction - : public IAggregateFunctionHelper - , public IWindowFunction -{ - std::string name; - - WindowFunction(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_, const DataTypePtr & result_type_) - : IAggregateFunctionHelper(argument_types_, parameters_, result_type_) - , name(name_) - {} - - bool isOnlyWindowFunction() const override { return true; } - - [[noreturn]] void fail() const - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "The function '{}' can only be used as a window function, not as an aggregate function", - getName()); - } - - String getName() const override { return name; } - void create(AggregateDataPtr __restrict) const override {} - void destroy(AggregateDataPtr __restrict) const noexcept override {} - bool hasTrivialDestructor() const override { return true; } - size_t sizeOfData() const override { return 0; } - size_t alignOfData() const override { return 1; } - void add(AggregateDataPtr __restrict, const IColumn **, size_t, Arena *) const override { fail(); } - void merge(AggregateDataPtr __restrict, ConstAggregateDataPtr, Arena *) const override { fail(); } - void serialize(ConstAggregateDataPtr __restrict, WriteBuffer &, std::optional) const override { fail(); } - void deserialize(AggregateDataPtr __restrict, ReadBuffer &, std::optional, Arena *) const override { fail(); } - void insertResultInto(AggregateDataPtr __restrict, IColumn &, Arena *) const override { fail(); } -}; struct WindowFunctionRank final : public WindowFunction { - WindowFunctionRank(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionRank(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) : WindowFunction(name_, argument_types_, parameters_, std::make_shared()) - {} + { + } bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, - size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override { - IColumn & to = *transform->blockAt(transform->current_row) - .output_columns[function_index]; - assert_cast(to).getData().push_back( - transform->peer_group_start_row_number); + IColumn & to = *transform->blockAt(transform->current_row).output_columns[function_index]; + assert_cast(to).getData().push_back(transform->peer_group_start_row_number); } }; struct WindowFunctionDenseRank final : public WindowFunction { - WindowFunctionDenseRank(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionDenseRank(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) : WindowFunction(name_, argument_types_, parameters_, std::make_shared()) - {} + { + } bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, - size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override { - IColumn & to = *transform->blockAt(transform->current_row) - .output_columns[function_index]; - assert_cast(to).getData().push_back( - transform->peer_group_number); + IColumn & to = *transform->blockAt(transform->current_row).output_columns[function_index]; + assert_cast(to).getData().push_back(transform->peer_group_number); } }; namespace recurrent_detail { - template T getValue(const WindowTransform * /*transform*/, size_t /*function_index*/, size_t /*column_index*/, RowNumber /*row*/) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "recurrent_detail::getValue() is not implemented for {} type", typeid(T).name()); - } +template +T getValue(const WindowTransform * /*transform*/, size_t /*function_index*/, size_t /*column_index*/, RowNumber /*row*/) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "recurrent_detail::getValue() is not implemented for {} type", typeid(T).name()); +} - template<> Float64 getValue(const WindowTransform * transform, size_t function_index, size_t column_index, RowNumber row) - { - const auto & workspace = transform->workspaces[function_index]; - const auto & column = transform->blockAt(row.block).input_columns[workspace.argument_column_indices[column_index]]; - return column->getFloat64(row.row); - } +template <> +Float64 getValue(const WindowTransform * transform, size_t function_index, size_t column_index, RowNumber row) +{ + const auto & workspace = transform->workspaces[function_index]; + const auto & column = transform->blockAt(row.block).input_columns[workspace.argument_column_indices[column_index]]; + return column->getFloat64(row.row); +} - template void setValueToOutputColumn(const WindowTransform * /*transform*/, size_t /*function_index*/, T /*value*/) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "recurrent_detail::setValueToOutputColumn() is not implemented for {} type", typeid(T).name()); - } +template +void setValueToOutputColumn(const WindowTransform * /*transform*/, size_t /*function_index*/, T /*value*/) +{ + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "recurrent_detail::setValueToOutputColumn() is not implemented for {} type", typeid(T).name()); +} - template<> void setValueToOutputColumn(const WindowTransform * transform, size_t function_index, Float64 value) - { - auto current_row = transform->current_row; - const auto & current_block = transform->blockAt(current_row); - IColumn & to = *current_block.output_columns[function_index]; +template <> +void setValueToOutputColumn(const WindowTransform * transform, size_t function_index, Float64 value) +{ + auto current_row = transform->current_row; + const auto & current_block = transform->blockAt(current_row); + IColumn & to = *current_block.output_columns[function_index]; - assert_cast(to).getData().push_back(value); - } + assert_cast(to).getData().push_back(value); +} } struct WindowFunctionHelpers { - template + template static T getValue(const WindowTransform * transform, size_t function_index, size_t column_index, RowNumber row) { return recurrent_detail::getValue(transform, function_index, column_index, row); } - template + template static void setValueToOutputColumn(const WindowTransform * transform, size_t function_index, T value) { recurrent_detail::setValueToOutputColumn(transform, function_index, value); @@ -1669,35 +1575,6 @@ struct WindowFunctionHelpers } }; -template -struct StatefulWindowFunction : public WindowFunction -{ - StatefulWindowFunction(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_, const DataTypePtr & result_type_) - : WindowFunction(name_, argument_types_, parameters_, result_type_) - { - } - - size_t sizeOfData() const override { return sizeof(State); } - size_t alignOfData() const override { return 1; } - - void create(AggregateDataPtr __restrict place) const override - { - new (place) State(); - } - - void destroy(AggregateDataPtr __restrict place) const noexcept override - { - reinterpret_cast(place)->~State(); - } - - bool hasTrivialDestructor() const override { return std::is_trivially_destructible_v; } - - State & getState(const WindowFunctionWorkspace & workspace) const - { - return *reinterpret_cast(workspace.aggregate_function_state.data()); - } -}; struct ExponentialTimeDecayedSumState { @@ -1721,34 +1598,34 @@ struct WindowFunctionExponentialTimeDecayedSum final : public StatefulWindowFunc { if (parameters_.size() != 1) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Function {} takes exactly one parameter", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one parameter", name_); } return applyVisitor(FieldVisitorConvertToNumber(), parameters_[0]); } - WindowFunctionExponentialTimeDecayedSum(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionExponentialTimeDecayedSum(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) : StatefulWindowFunction(name_, argument_types_, parameters_, std::make_shared()) , decay_length(getDecayLength(parameters_, name_)) { if (argument_types.size() != 2) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Function {} takes exactly two arguments", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly two arguments", name_); } if (!isNumber(argument_types[ARGUMENT_VALUE])) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Argument {} must be a number, '{}' given", ARGUMENT_VALUE, argument_types[ARGUMENT_VALUE]->getName()); } - if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) && !isDateTime64(argument_types[ARGUMENT_TIME])) + if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) + && !isDateTime64(argument_types[ARGUMENT_TIME])) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Argument {} must be DateTime, DateTime64 or a number, '{}' given", ARGUMENT_TIME, argument_types[ARGUMENT_TIME]->getName()); @@ -1757,8 +1634,7 @@ struct WindowFunctionExponentialTimeDecayedSum final : public StatefulWindowFunc bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, - size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override { const auto & workspace = transform->workspaces[function_index]; auto & state = getState(workspace); @@ -1770,8 +1646,7 @@ struct WindowFunctionExponentialTimeDecayedSum final : public StatefulWindowFunc RowNumber frame_back = transform->prevRowNumber(transform->frame_end); Float64 back_t = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_TIME, frame_back); - if (transform->prev_frame_start <= transform->frame_start - && transform->frame_start < transform->prev_frame_end + if (transform->prev_frame_start <= transform->frame_start && transform->frame_start < transform->prev_frame_end && transform->prev_frame_end <= transform->frame_end) { for (RowNumber i = transform->prev_frame_start; i < transform->frame_start; transform->advanceRowNumber(i)) @@ -1805,8 +1680,8 @@ struct WindowFunctionExponentialTimeDecayedSum final : public StatefulWindowFunc WindowFunctionHelpers::setValueToOutputColumn(transform, function_index, result); } - private: - const Float64 decay_length; +private: + const Float64 decay_length; }; struct WindowFunctionExponentialTimeDecayedMax final : public WindowFunction @@ -1818,34 +1693,34 @@ struct WindowFunctionExponentialTimeDecayedMax final : public WindowFunction { if (parameters_.size() != 1) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Function {} takes exactly one parameter", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one parameter", name_); } return applyVisitor(FieldVisitorConvertToNumber(), parameters_[0]); } - WindowFunctionExponentialTimeDecayedMax(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionExponentialTimeDecayedMax(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) : WindowFunction(name_, argument_types_, parameters_, std::make_shared()) , decay_length(getDecayLength(parameters_, name_)) { if (argument_types.size() != 2) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Function {} takes exactly two arguments", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly two arguments", name_); } if (!isNumber(argument_types[ARGUMENT_VALUE])) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Argument {} must be a number, '{}' given", ARGUMENT_VALUE, argument_types[ARGUMENT_VALUE]->getName()); } - if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) && !isDateTime64(argument_types[ARGUMENT_TIME])) + if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) + && !isDateTime64(argument_types[ARGUMENT_TIME])) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Argument {} must be DateTime, DateTime64 or a number, '{}' given", ARGUMENT_TIME, argument_types[ARGUMENT_TIME]->getName()); @@ -1854,8 +1729,7 @@ struct WindowFunctionExponentialTimeDecayedMax final : public WindowFunction bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, - size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override { Float64 result = std::numeric_limits::quiet_NaN(); @@ -1881,8 +1755,8 @@ struct WindowFunctionExponentialTimeDecayedMax final : public WindowFunction WindowFunctionHelpers::setValueToOutputColumn(transform, function_index, result); } - private: - const Float64 decay_length; +private: + const Float64 decay_length; }; struct WindowFunctionExponentialTimeDecayedCount final : public StatefulWindowFunction @@ -1893,26 +1767,25 @@ struct WindowFunctionExponentialTimeDecayedCount final : public StatefulWindowFu { if (parameters_.size() != 1) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Function {} takes exactly one parameter", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one parameter", name_); } return applyVisitor(FieldVisitorConvertToNumber(), parameters_[0]); } - WindowFunctionExponentialTimeDecayedCount(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionExponentialTimeDecayedCount(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) : StatefulWindowFunction(name_, argument_types_, parameters_, std::make_shared()) , decay_length(getDecayLength(parameters_, name_)) { if (argument_types.size() != 1) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Function {} takes exactly one argument", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one argument", name_); } - if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) && !isDateTime64(argument_types[ARGUMENT_TIME])) + if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) + && !isDateTime64(argument_types[ARGUMENT_TIME])) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Argument {} must be DateTime, DateTime64 or a number, '{}' given", ARGUMENT_TIME, argument_types[ARGUMENT_TIME]->getName()); @@ -1921,8 +1794,7 @@ struct WindowFunctionExponentialTimeDecayedCount final : public StatefulWindowFu bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, - size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override { const auto & workspace = transform->workspaces[function_index]; auto & state = getState(workspace); @@ -1934,8 +1806,7 @@ struct WindowFunctionExponentialTimeDecayedCount final : public StatefulWindowFu RowNumber frame_back = transform->prevRowNumber(transform->frame_end); Float64 back_t = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_TIME, frame_back); - if (transform->prev_frame_start <= transform->frame_start - && transform->frame_start < transform->prev_frame_end + if (transform->prev_frame_start <= transform->frame_start && transform->frame_start < transform->prev_frame_end && transform->prev_frame_end <= transform->frame_end) { for (RowNumber i = transform->prev_frame_start; i < transform->frame_start; transform->advanceRowNumber(i)) @@ -1966,8 +1837,8 @@ struct WindowFunctionExponentialTimeDecayedCount final : public StatefulWindowFu WindowFunctionHelpers::setValueToOutputColumn(transform, function_index, result); } - private: - const Float64 decay_length; +private: + const Float64 decay_length; }; struct WindowFunctionExponentialTimeDecayedAvg final : public StatefulWindowFunction @@ -1979,34 +1850,34 @@ struct WindowFunctionExponentialTimeDecayedAvg final : public StatefulWindowFunc { if (parameters_.size() != 1) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Function {} takes exactly one parameter", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one parameter", name_); } return applyVisitor(FieldVisitorConvertToNumber(), parameters_[0]); } - WindowFunctionExponentialTimeDecayedAvg(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionExponentialTimeDecayedAvg(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) : StatefulWindowFunction(name_, argument_types_, parameters_, std::make_shared()) , decay_length(getDecayLength(parameters_, name_)) { if (argument_types.size() != 2) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Function {} takes exactly two arguments", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly two arguments", name_); } if (!isNumber(argument_types[ARGUMENT_VALUE])) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Argument {} must be a number, '{}' given", ARGUMENT_VALUE, argument_types[ARGUMENT_VALUE]->getName()); } - if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) && !isDateTime64(argument_types[ARGUMENT_TIME])) + if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) + && !isDateTime64(argument_types[ARGUMENT_TIME])) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Argument {} must be DateTime, DateTime64 or a number, '{}' given", ARGUMENT_TIME, argument_types[ARGUMENT_TIME]->getName()); @@ -2015,8 +1886,7 @@ struct WindowFunctionExponentialTimeDecayedAvg final : public StatefulWindowFunc bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, - size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override { const auto & workspace = transform->workspaces[function_index]; auto & state = getState(workspace); @@ -2030,8 +1900,7 @@ struct WindowFunctionExponentialTimeDecayedAvg final : public StatefulWindowFunc RowNumber frame_back = transform->prevRowNumber(transform->frame_end); Float64 back_t = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_TIME, frame_back); - if (transform->prev_frame_start <= transform->frame_start - && transform->frame_start < transform->prev_frame_end + if (transform->prev_frame_start <= transform->frame_start && transform->frame_start < transform->prev_frame_end && transform->prev_frame_end <= transform->frame_end) { for (RowNumber i = transform->prev_frame_start; i < transform->frame_start; transform->advanceRowNumber(i)) @@ -2074,56 +1943,49 @@ struct WindowFunctionExponentialTimeDecayedAvg final : public StatefulWindowFunc state.previous_count = count; state.previous_time = back_t; - result = sum/count; + result = sum / count; } WindowFunctionHelpers::setValueToOutputColumn(transform, function_index, result); } - private: - const Float64 decay_length; +private: + const Float64 decay_length; }; struct WindowFunctionRowNumber final : public WindowFunction { - WindowFunctionRowNumber(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionRowNumber(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) : WindowFunction(name_, argument_types_, parameters_, std::make_shared()) - {} + { + } bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, - size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override { - IColumn & to = *transform->blockAt(transform->current_row) - .output_columns[function_index]; - assert_cast(to).getData().push_back( - transform->current_row_number); + IColumn & to = *transform->blockAt(transform->current_row).output_columns[function_index]; + assert_cast(to).getData().push_back(transform->current_row_number); } }; namespace { - struct NtileState - { - UInt64 buckets = 0; - RowNumber start_row; - UInt64 current_partition_rows = 0; - UInt64 current_partition_inserted_row = 0; +struct NtileState +{ + UInt64 buckets = 0; + RowNumber start_row; + UInt64 current_partition_rows = 0; + UInt64 current_partition_inserted_row = 0; - void windowInsertResultInto( - const WindowTransform * transform, - size_t function_index, - const DataTypes & argument_types); - }; + void windowInsertResultInto(const WindowTransform * transform, size_t function_index, const DataTypes & argument_types); +}; } // Usage: ntile(n). n is the number of buckets. struct WindowFunctionNtile final : public StatefulWindowFunction { - WindowFunctionNtile(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionNtile(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) : StatefulWindowFunction(name_, argument_types_, parameters_, std::make_shared()) { if (argument_types.size() != 1) @@ -2131,7 +1993,11 @@ struct WindowFunctionNtile final : public StatefulWindowFunction auto type_id = argument_types[0]->getTypeId(); if (type_id != TypeIndex::UInt8 && type_id != TypeIndex::UInt16 && type_id != TypeIndex::UInt32 && type_id != TypeIndex::UInt64) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "'{}' argument type must be an unsigned integer (not larger than 64-bit), got {}", name_, argument_types[0]->getName()); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "'{}' argument type must be an unsigned integer (not larger than 64-bit), got {}", + name_, + argument_types[0]->getName()); } bool allocatesMemoryInArena() const override { return false; } @@ -2162,13 +2028,13 @@ struct WindowFunctionNtile final : public StatefulWindowFunction std::optional getDefaultFrame() const override { WindowFrame frame; + frame.is_default = false; frame.type = WindowFrame::FrameType::ROWS; frame.end_type = WindowFrame::BoundaryType::Unbounded; return frame; } - void windowInsertResultInto(const WindowTransform * transform, - size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override { const auto & workspace = transform->workspaces[function_index]; auto & state = getState(workspace); @@ -2178,83 +2044,80 @@ struct WindowFunctionNtile final : public StatefulWindowFunction namespace { - void NtileState::windowInsertResultInto( - const WindowTransform * transform, - size_t function_index, - const DataTypes & argument_types) +void NtileState::windowInsertResultInto(const WindowTransform * transform, size_t function_index, const DataTypes & argument_types) +{ + if (!buckets) [[unlikely]] { - if (!buckets) [[unlikely]] + const auto & current_block = transform->blockAt(transform->current_row); + const auto & workspace = transform->workspaces[function_index]; + const auto & arg_col = *current_block.original_input_columns[workspace.argument_column_indices[0]]; + if (!isColumnConst(arg_col)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of 'ntile' function must be a constant"); + auto type_id = argument_types[0]->getTypeId(); + if (type_id == TypeIndex::UInt8) + buckets = arg_col[transform->current_row.row].get(); + else if (type_id == TypeIndex::UInt16) + buckets = arg_col[transform->current_row.row].get(); + else if (type_id == TypeIndex::UInt32) + buckets = arg_col[transform->current_row.row].get(); + else if (type_id == TypeIndex::UInt64) + buckets = arg_col[transform->current_row.row].get(); + + if (!buckets) { - const auto & current_block = transform->blockAt(transform->current_row); - const auto & workspace = transform->workspaces[function_index]; - const auto & arg_col = *current_block.original_input_columns[workspace.argument_column_indices[0]]; - if (!isColumnConst(arg_col)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of 'ntile' function must be a constant"); - auto type_id = argument_types[0]->getTypeId(); - if (type_id == TypeIndex::UInt8) - buckets = arg_col[transform->current_row.row].get(); - else if (type_id == TypeIndex::UInt16) - buckets = arg_col[transform->current_row.row].get(); - else if (type_id == TypeIndex::UInt32) - buckets = arg_col[transform->current_row.row].get(); - else if (type_id == TypeIndex::UInt64) - buckets = arg_col[transform->current_row.row].get(); - - if (!buckets) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of 'ntile' function must be greater than zero"); - } - } - // new partition - if (WindowFunctionHelpers::checkPartitionEnterFirstRow(transform)) [[unlikely]] - { - current_partition_rows = 0; - current_partition_inserted_row = 0; - start_row = transform->current_row; - } - current_partition_rows++; - - // Only do the action when we meet the last row in this partition. - if (!WindowFunctionHelpers::checkPartitionEnterLastRow(transform)) - return; - - auto bucket_capacity = current_partition_rows / buckets; - auto capacity_diff = current_partition_rows - bucket_capacity * buckets; - - // bucket number starts from 1. - UInt64 bucket_num = 1; - while (current_partition_inserted_row < current_partition_rows) - { - auto current_bucket_capacity = bucket_capacity; - if (capacity_diff > 0) - { - current_bucket_capacity += 1; - capacity_diff--; - } - auto left_rows = current_bucket_capacity; - while (left_rows) - { - auto available_block_rows = transform->blockRowsNumber(start_row) - start_row.row; - IColumn & to = *transform->blockAt(start_row).output_columns[function_index]; - auto & pod_array = assert_cast(to).getData(); - if (left_rows < available_block_rows) - { - pod_array.resize_fill(pod_array.size() + left_rows, bucket_num); - start_row.row += left_rows; - left_rows = 0; - } - else - { - pod_array.resize_fill(pod_array.size() + available_block_rows, bucket_num); - left_rows -= available_block_rows; - start_row.block++; - start_row.row = 0; - } - } - current_partition_inserted_row += current_bucket_capacity; - bucket_num += 1; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of 'ntile' function must be greater than zero"); } } + // new partition + if (WindowFunctionHelpers::checkPartitionEnterFirstRow(transform)) [[unlikely]] + { + current_partition_rows = 0; + current_partition_inserted_row = 0; + start_row = transform->current_row; + } + current_partition_rows++; + + // Only do the action when we meet the last row in this partition. + if (!WindowFunctionHelpers::checkPartitionEnterLastRow(transform)) + return; + + auto bucket_capacity = current_partition_rows / buckets; + auto capacity_diff = current_partition_rows - bucket_capacity * buckets; + + // bucket number starts from 1. + UInt64 bucket_num = 1; + while (current_partition_inserted_row < current_partition_rows) + { + auto current_bucket_capacity = bucket_capacity; + if (capacity_diff > 0) + { + current_bucket_capacity += 1; + capacity_diff--; + } + auto left_rows = current_bucket_capacity; + while (left_rows) + { + auto available_block_rows = transform->blockRowsNumber(start_row) - start_row.row; + IColumn & to = *transform->blockAt(start_row).output_columns[function_index]; + auto & pod_array = assert_cast(to).getData(); + if (left_rows < available_block_rows) + { + pod_array.resize_fill(pod_array.size() + left_rows, bucket_num); + start_row.row += left_rows; + left_rows = 0; + } + else + { + pod_array.resize_fill(pod_array.size() + available_block_rows, bucket_num); + left_rows -= available_block_rows; + start_row.block++; + start_row.row = 0; + } + } + current_partition_inserted_row += current_bucket_capacity; + bucket_num += 1; + } +} } namespace @@ -2269,23 +2132,21 @@ struct PercentRankState struct WindowFunctionPercentRank final : public StatefulWindowFunction { public: - WindowFunctionPercentRank(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionPercentRank(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) : StatefulWindowFunction(name_, argument_types_, parameters_, std::make_shared()) - {} + { + } bool allocatesMemoryInArena() const override { return false; } bool checkWindowFrameType(const WindowTransform * transform) const override { - if (transform->window_description.frame.type != WindowFrame::FrameType::RANGE - || transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded - || transform->window_description.frame.end_type != WindowFrame::BoundaryType::Current) - { - LOG_ERROR( - getLogger("WindowFunctionPercentRank"), - "Window frame for function 'percent_rank' should be 'RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT'"); - return false; + if (transform->window_description.frame != getDefaultFrame()) + { + LOG_ERROR( + getLogger("WindowFunctionPercentRank"), + "Window frame for function 'percent_rank' should be '{}'", getDefaultFrame()->toString()); + return false; } return true; } @@ -2293,9 +2154,10 @@ public: std::optional getDefaultFrame() const override { WindowFrame frame; + frame.is_default = false; frame.type = WindowFrame::FrameType::RANGE; frame.begin_type = WindowFrame::BoundaryType::Unbounded; - frame.end_type = WindowFrame::BoundaryType::Current; + frame.end_type = WindowFrame::BoundaryType::Unbounded; return frame; } @@ -2371,14 +2233,12 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction { FunctionBasePtr func_cast = nullptr; - WindowFunctionLagLeadInFrame(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionLagLeadInFrame(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) : WindowFunction(name_, argument_types_, parameters_, createResultType(argument_types_, name_)) { if (!parameters.empty()) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Function {} cannot be parameterized", name_); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} cannot be parameterized", name_); } if (argument_types.size() == 1) @@ -2388,9 +2248,7 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction if (!isInt64OrUInt64FieldType(argument_types[1]->getDefault().getType())) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Offset must be an integer, '{}' given", - argument_types[1]->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Offset must be an integer, '{}' given", argument_types[1]->getName()); } if (argument_types.size() == 2) @@ -2400,9 +2258,11 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction if (argument_types.size() > 3) { - throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, + throw Exception( + ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Function '{}' accepts at most 3 arguments, {} given", - name, argument_types.size()); + name, + argument_types.size()); } if (argument_types[0]->equals(*argument_types[2])) @@ -2411,14 +2271,16 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction const auto supertype = tryGetLeastSupertype(DataTypes{argument_types[0], argument_types[2]}); if (!supertype) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "There is no supertype for the argument type '{}' and the default value type '{}'", argument_types[0]->getName(), argument_types[2]->getName()); } if (!argument_types[0]->equals(*supertype)) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "The supertype '{}' for the argument type '{}' and the default value type '{}' is not the same as the argument type", supertype->getName(), argument_types[0]->getName(), @@ -2427,15 +2289,8 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction const auto from_name = argument_types[2]->getName(); const auto to_name = argument_types[0]->getName(); - ColumnsWithTypeAndName arguments - { - { argument_types[2], "" }, - { - DataTypeString().createColumnConst(0, to_name), - std::make_shared(), - "" - } - }; + ColumnsWithTypeAndName arguments{ + {argument_types[2], ""}, {DataTypeString().createColumnConst(0, to_name), std::make_shared(), ""}}; auto get_cast_func = [&arguments] { @@ -2444,7 +2299,6 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction }; func_cast = get_cast_func(); - } ColumnPtr castColumn(const Columns & columns, const std::vector & idx) override @@ -2452,15 +2306,11 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction if (!func_cast) return nullptr; - ColumnsWithTypeAndName arguments - { - { columns[idx[2]], argument_types[2], "" }, - { - DataTypeString().createColumnConst(columns[idx[2]]->size(), argument_types[0]->getName()), - std::make_shared(), - "" - } - }; + ColumnsWithTypeAndName arguments{ + {columns[idx[2]], argument_types[2], ""}, + {DataTypeString().createColumnConst(columns[idx[2]]->size(), argument_types[0]->getName()), + std::make_shared(), + ""}}; return func_cast->execute(arguments, argument_types[0], columns[idx[2]]->size()); } @@ -2469,8 +2319,7 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction { if (argument_types_.empty()) { - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, - "Function {} takes at least one argument", name_); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} takes at least one argument", name_); } return argument_types_[0]; @@ -2478,8 +2327,7 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, - size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override { const auto & current_block = transform->blockAt(transform->current_row); IColumn & to = *current_block.output_columns[function_index]; @@ -2488,34 +2336,27 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction Int64 offset = 1; if (argument_types.size() > 1) { - offset = (*current_block.input_columns[ - workspace.argument_column_indices[1]])[ - transform->current_row.row].get(); + offset = (*current_block.input_columns[workspace.argument_column_indices[1]])[transform->current_row.row].get(); /// Either overflow or really negative value, both is not acceptable. if (offset < 0) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "The offset for function {} must be in (0, {}], {} given", - getName(), INT64_MAX, offset); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "The offset for function {} must be in (0, {}], {} given", getName(), INT64_MAX, offset); } } - const auto [target_row, offset_left] = transform->moveRowNumber( - transform->current_row, offset * (is_lead ? 1 : -1)); + const auto [target_row, offset_left] = transform->moveRowNumber(transform->current_row, offset * (is_lead ? 1 : -1)); - if (offset_left != 0 - || target_row < transform->frame_start - || transform->frame_end <= target_row) + if (offset_left != 0 || target_row < transform->frame_start || transform->frame_end <= target_row) { // Offset is outside the frame. if (argument_types.size() > 2) { // Column with default values is specified. - const IColumn & default_column = - current_block.casted_columns[function_index] ? - *current_block.casted_columns[function_index].get() : - *current_block.input_columns[workspace.argument_column_indices[2]].get(); + const IColumn & default_column = current_block.casted_columns[function_index] + ? *current_block.casted_columns[function_index].get() + : *current_block.input_columns[workspace.argument_column_indices[2]].get(); to.insert(default_column[transform->current_row.row]); } @@ -2527,30 +2368,24 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction else { // Offset is inside the frame. - to.insertFrom(*transform->blockAt(target_row).input_columns[ - workspace.argument_column_indices[0]], - target_row.row); + to.insertFrom(*transform->blockAt(target_row).input_columns[workspace.argument_column_indices[0]], target_row.row); } } }; struct WindowFunctionNthValue final : public WindowFunction { - WindowFunctionNthValue(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionNthValue(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) : WindowFunction(name_, argument_types_, parameters_, createResultType(name_, argument_types_)) { if (!parameters.empty()) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Function {} cannot be parameterized", name_); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} cannot be parameterized", name_); } if (!isInt64OrUInt64FieldType(argument_types[1]->getDefault().getType())) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Offset must be an integer, '{}' given", - argument_types[1]->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Offset must be an integer, '{}' given", argument_types[1]->getName()); } } @@ -2558,8 +2393,7 @@ struct WindowFunctionNthValue final : public WindowFunction { if (argument_types_.size() != 2) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Function {} takes exactly two arguments", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly two arguments", name_); } return argument_types_[0]; @@ -2567,30 +2401,24 @@ struct WindowFunctionNthValue final : public WindowFunction bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, - size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override { const auto & current_block = transform->blockAt(transform->current_row); IColumn & to = *current_block.output_columns[function_index]; const auto & workspace = transform->workspaces[function_index]; - Int64 offset = (*current_block.input_columns[ - workspace.argument_column_indices[1]])[ - transform->current_row.row].get(); + Int64 offset = (*current_block.input_columns[workspace.argument_column_indices[1]])[transform->current_row.row].get(); /// Either overflow or really negative value, both is not acceptable. if (offset <= 0) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "The offset for function {} must be in (0, {}], {} given", - getName(), INT64_MAX, offset); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "The offset for function {} must be in (0, {}], {} given", getName(), INT64_MAX, offset); } --offset; const auto [target_row, offset_left] = transform->moveRowNumber(transform->frame_start, offset); - if (offset_left != 0 - || target_row < transform->frame_start - || transform->frame_end <= target_row) + if (offset_left != 0 || target_row < transform->frame_start || transform->frame_end <= target_row) { // Offset is outside the frame. to.insertDefault(); @@ -2598,9 +2426,7 @@ struct WindowFunctionNthValue final : public WindowFunction else { // Offset is inside the frame. - to.insertFrom(*transform->blockAt(target_row).input_columns[ - workspace.argument_column_indices[0]], - target_row.row); + to.insertFrom(*transform->blockAt(target_row).input_columns[workspace.argument_column_indices[0]], target_row.row); } } }; @@ -2621,35 +2447,34 @@ struct NonNegativeDerivativeParams bool interval_specified = false; Int64 ts_scale_multiplier = 0; - NonNegativeDerivativeParams( - const std::string & name_, const DataTypes & argument_types, const Array & parameters) + NonNegativeDerivativeParams(const std::string & name_, const DataTypes & argument_types, const Array & parameters) { if (!parameters.empty()) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Function {} cannot be parameterized", name_); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} cannot be parameterized", name_); } if (argument_types.size() != 2 && argument_types.size() != 3) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Function {} takes 2 or 3 arguments", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes 2 or 3 arguments", name_); } if (!isNumber(argument_types[ARGUMENT_METRIC])) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Argument {} must be a number, '{}' given", - ARGUMENT_METRIC, - argument_types[ARGUMENT_METRIC]->getName()); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Argument {} must be a number, '{}' given", + ARGUMENT_METRIC, + argument_types[ARGUMENT_METRIC]->getName()); } if (!isDateTime(argument_types[ARGUMENT_TIMESTAMP]) && !isDateTime64(argument_types[ARGUMENT_TIMESTAMP])) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Argument {} must be DateTime or DateTime64, '{}' given", - ARGUMENT_TIMESTAMP, - argument_types[ARGUMENT_TIMESTAMP]->getName()); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Argument {} must be DateTime or DateTime64, '{}' given", + ARGUMENT_TIMESTAMP, + argument_types[ARGUMENT_TIMESTAMP]->getName()); } if (isDateTime64(argument_types[ARGUMENT_TIMESTAMP])) @@ -2683,27 +2508,28 @@ struct NonNegativeDerivativeParams }; // nonNegativeDerivative(metric_column, timestamp_column[, INTERVAL 1 SECOND]) -struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction, public NonNegativeDerivativeParams +struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction, + public NonNegativeDerivativeParams { using Params = NonNegativeDerivativeParams; - WindowFunctionNonNegativeDerivative(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionNonNegativeDerivative(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) : StatefulWindowFunction(name_, argument_types_, parameters_, std::make_shared()) , NonNegativeDerivativeParams(name, argument_types, parameters) - {} + { + } bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, - size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override { const auto & current_block = transform->blockAt(transform->current_row); const auto & workspace = transform->workspaces[function_index]; auto & state = getState(workspace); - auto interval_duration = interval_specified ? interval_length * - (*current_block.input_columns[workspace.argument_column_indices[ARGUMENT_INTERVAL]]).getFloat64(0) : 1; + auto interval_duration = interval_specified + ? interval_length * (*current_block.input_columns[workspace.argument_column_indices[ARGUMENT_INTERVAL]]).getFloat64(0) + : 1; Float64 curr_metric = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_METRIC, transform->current_row); Float64 metric_diff = curr_metric - state.previous_metric; @@ -2711,16 +2537,18 @@ struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction if (ts_scale_multiplier) { - const auto & column = transform->blockAt(transform->current_row.block).input_columns[workspace.argument_column_indices[ARGUMENT_TIMESTAMP]]; + const auto & column + = transform->blockAt(transform->current_row.block).input_columns[workspace.argument_column_indices[ARGUMENT_TIMESTAMP]]; const auto & curr_timestamp = checkAndGetColumn(*column).getInt(transform->current_row.row); Float64 time_elapsed = curr_timestamp - state.previous_timestamp; - result = (time_elapsed > 0) ? (metric_diff * ts_scale_multiplier / time_elapsed * interval_duration) : 0; + result = (time_elapsed > 0) ? (metric_diff * ts_scale_multiplier / time_elapsed * interval_duration) : 0; state.previous_timestamp = curr_timestamp; } else { - Float64 curr_timestamp = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_TIMESTAMP, transform->current_row); + Float64 curr_timestamp + = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_TIMESTAMP, transform->current_row); Float64 time_elapsed = curr_timestamp - state.previous_timestamp; result = (time_elapsed > 0) ? (metric_diff / time_elapsed * interval_duration) : 0; state.previous_timestamp = curr_timestamp; diff --git a/src/Processors/Transforms/WindowTransform.h b/src/Processors/Transforms/WindowTransform.h index fe4f79e997c..cb672ad6841 100644 --- a/src/Processors/Transforms/WindowTransform.h +++ b/src/Processors/Transforms/WindowTransform.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -21,30 +22,6 @@ using ExpressionActionsPtr = std::shared_ptr; class Arena; -class IWindowFunction; - -// Runtime data for computing one window function. -struct WindowFunctionWorkspace -{ - AggregateFunctionPtr aggregate_function; - - // Cached value of aggregate function isState virtual method - bool is_aggregate_function_state = false; - - // This field is set for pure window functions. When set, we ignore the - // window_function.aggregate_function, and work through this interface - // instead. - IWindowFunction * window_function_impl = nullptr; - - std::vector argument_column_indices; - - // Will not be initialized for a pure window function. - mutable AlignedBuffer aggregate_function_state; - - // Argument columns. Be careful, this is a per-block cache. - std::vector argument_columns; - UInt64 cached_block_number = std::numeric_limits::max(); -}; struct WindowTransformBlock { From b35ff7e41784c50c04ad02df94b7a3ecd947789c Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 2 Aug 2024 15:33:31 +0800 Subject: [PATCH 353/644] update test --- src/Processors/Transforms/WindowTransform.cpp | 8 +------- .../0_stateless/01592_window_functions.reference | 11 +++++++++++ tests/queries/0_stateless/01592_window_functions.sql | 11 +++++++++++ 3 files changed, 23 insertions(+), 7 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index ae9a94bff90..a003c9a8e56 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -1341,13 +1341,6 @@ IProcessor::Status WindowTransform::prepare() { // Output the ready block. const auto i = next_output_block_number - first_block_number; - LOG_ERROR( - getLogger("WindowTransform"), - "xxx {} output block: {}, next_output_block_number: {} first_not_ready_row.block: {}", - fmt::ptr(this), - i, - next_output_block_number, - first_not_ready_row.block); auto & block = blocks[i]; auto columns = block.original_input_columns; for (auto & res : block.output_columns) @@ -2158,6 +2151,7 @@ public: frame.type = WindowFrame::FrameType::RANGE; frame.begin_type = WindowFrame::BoundaryType::Unbounded; frame.end_type = WindowFrame::BoundaryType::Unbounded; + //frame.end_type = WindowFrame::BoundaryType::Current; return frame; } diff --git a/tests/queries/0_stateless/01592_window_functions.reference b/tests/queries/0_stateless/01592_window_functions.reference index 06ec67ee82d..558f643c281 100644 --- a/tests/queries/0_stateless/01592_window_functions.reference +++ b/tests/queries/0_stateless/01592_window_functions.reference @@ -92,3 +92,14 @@ Kindle Fire Tablet 150 1 0 Samsung Galaxy Tab Tablet 200 2 0.5 iPad Tablet 700 3 1 Others Unknow 200 1 0 +---- Q9 ---- +0 1 0 +1 2 1 +2 3 2 +3 4 3 +4 5 4 +5 6 5 +6 7 6 +7 8 7 +8 9 8 +9 10 9 diff --git a/tests/queries/0_stateless/01592_window_functions.sql b/tests/queries/0_stateless/01592_window_functions.sql index a660fcca7b2..32c53763e40 100644 --- a/tests/queries/0_stateless/01592_window_functions.sql +++ b/tests/queries/0_stateless/01592_window_functions.sql @@ -124,3 +124,14 @@ ORDER BY drop table product_groups; drop table products; + +select '---- Q9 ----'; +select number, row_number, cast(percent_rank * 10000 as Int32) as percent_rank +from ( + select number, row_number() over () as row_number, percent_rank() over (order by number) as percent_rank + from numbers(10000) + order by number + limit 10 +) +settings max_block_size=100; + From 632ab91bbb389f7904296f6c9a3c89e43ff8a4df Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 2 Aug 2024 16:01:29 +0800 Subject: [PATCH 354/644] revert format --- src/Planner/PlannerActionsVisitor.cpp | 6 + src/Planner/PlannerWindowFunctions.cpp | 1 - src/Processors/Transforms/WindowTransform.cpp | 915 ++++++++++-------- 3 files changed, 508 insertions(+), 414 deletions(-) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 99b9c3f7482..f6c2c92cbb4 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -242,6 +243,11 @@ public: auto & window_frame = window_node.getWindowFrame(); if (!window_frame.is_default) return window_frame; + auto aggregate_function = function_node.getAggregateFunction(); + if (const auto * win_func = dynamic_cast(aggregate_function.get())) + { + return win_func->getDefaultFrame(); + } return {}; }; buffer << " OVER ("; diff --git a/src/Planner/PlannerWindowFunctions.cpp b/src/Planner/PlannerWindowFunctions.cpp index 7d0fc3a85b3..2a28787ba96 100644 --- a/src/Planner/PlannerWindowFunctions.cpp +++ b/src/Planner/PlannerWindowFunctions.cpp @@ -24,7 +24,6 @@ extern const int NOT_IMPLEMENTED; namespace { -//WindowDescription extractWindowDescriptionFromWindowNode(const QueryTreeNodePtr & node, const PlannerContext & planner_context) WindowDescription extractWindowDescriptionFromWindowNode(const FunctionNode & func_node, const PlannerContext & planner_context) { auto node = func_node.getWindowNode(); diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index a003c9a8e56..c26cd7cc8c3 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -6,12 +6,9 @@ #include #include #include -#include #include #include -#include #include -#include #include #include #include @@ -19,6 +16,9 @@ #include #include #include +#include +#include +#include #include #include @@ -57,31 +57,30 @@ struct Settings; namespace ErrorCodes { -extern const int BAD_ARGUMENTS; -extern const int NOT_IMPLEMENTED; -extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -extern const int ILLEGAL_TYPE_OF_ARGUMENT; -extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; -extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; + extern const int BAD_ARGUMENTS; + extern const int NOT_IMPLEMENTED; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } - // Compares ORDER BY column values at given rows to find the boundaries of frame: // [compared] with [reference] +/- offset. Return value is -1/0/+1, like in // sorting predicates -- -1 means [compared] is less than [reference] +/- offset. template -static int compareValuesWithOffset( - const IColumn * _compared_column, - size_t compared_row, - const IColumn * _reference_column, +static int compareValuesWithOffset(const IColumn * _compared_column, + size_t compared_row, const IColumn * _reference_column, size_t reference_row, const Field & _offset, bool offset_is_preceding) { // Casting the columns to the known type here makes it faster, probably // because the getData call can be devirtualized. - const auto * compared_column = assert_cast(_compared_column); - const auto * reference_column = assert_cast(_reference_column); + const auto * compared_column = assert_cast( + _compared_column); + const auto * reference_column = assert_cast( + _reference_column); using ValueType = typename ColumnType::ValueType; // Note that the storage type of offset returned by get<> is different, so @@ -91,11 +90,13 @@ static int compareValuesWithOffset( const auto compared_value_data = compared_column->getDataAt(compared_row); assert(compared_value_data.size == sizeof(ValueType)); - auto compared_value = unalignedLoad(compared_value_data.data); + auto compared_value = unalignedLoad( + compared_value_data.data); const auto reference_value_data = reference_column->getDataAt(reference_row); assert(reference_value_data.size == sizeof(ValueType)); - auto reference_value = unalignedLoad(reference_value_data.data); + auto reference_value = unalignedLoad( + reference_value_data.data); bool is_overflow; if (offset_is_preceding) @@ -120,34 +121,37 @@ static int compareValuesWithOffset( else { // No overflow, compare normally. - return compared_value < reference_value ? -1 : compared_value == reference_value ? 0 : 1; + return compared_value < reference_value ? -1 + : compared_value == reference_value ? 0 : 1; } } // A specialization of compareValuesWithOffset for floats. template -static int compareValuesWithOffsetFloat( - const IColumn * _compared_column, - size_t compared_row, - const IColumn * _reference_column, +static int compareValuesWithOffsetFloat(const IColumn * _compared_column, + size_t compared_row, const IColumn * _reference_column, size_t reference_row, const Field & _offset, bool offset_is_preceding) { // Casting the columns to the known type here makes it faster, probably // because the getData call can be devirtualized. - const auto * compared_column = assert_cast(_compared_column); - const auto * reference_column = assert_cast(_reference_column); + const auto * compared_column = assert_cast( + _compared_column); + const auto * reference_column = assert_cast( + _reference_column); const auto offset = _offset.get(); chassert(offset >= 0); const auto compared_value_data = compared_column->getDataAt(compared_row); assert(compared_value_data.size == sizeof(typename ColumnType::ValueType)); - auto compared_value = unalignedLoad(compared_value_data.data); + auto compared_value = unalignedLoad( + compared_value_data.data); const auto reference_value_data = reference_column->getDataAt(reference_row); assert(reference_value_data.size == sizeof(typename ColumnType::ValueType)); - auto reference_value = unalignedLoad(reference_value_data.data); + auto reference_value = unalignedLoad( + reference_value_data.data); /// Floats overflow to Inf and the comparison will work normally, so we don't have to do anything. if (offset_is_preceding) @@ -155,58 +159,58 @@ static int compareValuesWithOffsetFloat( else reference_value += static_cast(offset); - const auto result = compared_value < reference_value ? -1 : (compared_value == reference_value ? 0 : 1); + const auto result = compared_value < reference_value ? -1 + : (compared_value == reference_value ? 0 : 1); return result; } // Helper macros to dispatch on type of the ORDER BY column #define APPLY_FOR_ONE_NEST_TYPE(FUNCTION, TYPE) \ - else if (typeid_cast(nest_compared_column.get())) \ - { \ - /* clang-tidy you're dumb, I can't put FUNCTION in braces here. */ \ - nest_compare_function = FUNCTION; /* NOLINT */ \ - } +else if (typeid_cast(nest_compared_column.get())) \ +{ \ + /* clang-tidy you're dumb, I can't put FUNCTION in braces here. */ \ + nest_compare_function = FUNCTION; /* NOLINT */ \ +} #define APPLY_FOR_NEST_TYPES(FUNCTION) \ - if (false) /* NOLINT */ \ - { \ - /* Do nothing, a starter condition. */ \ - } \ - APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ - APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ - APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ - APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +if (false) /* NOLINT */ \ +{ \ + /* Do nothing, a starter condition. */ \ +} \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ \ - APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ - APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ - APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ - APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ - APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ \ - APPLY_FOR_ONE_NEST_TYPE(FUNCTION##Float, ColumnVector) \ - APPLY_FOR_ONE_NEST_TYPE(FUNCTION##Float, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION##Float, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION##Float, ColumnVector) \ \ - else \ - { \ - throw Exception( \ - ErrorCodes::NOT_IMPLEMENTED, \ - "The RANGE OFFSET frame for '{}' ORDER BY nest column is not implemented", \ - demangle(typeid(nest_compared_column).name())); \ - } +else \ +{ \ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, \ + "The RANGE OFFSET frame for '{}' ORDER BY nest column is not implemented", \ + demangle(typeid(nest_compared_column).name())); \ +} // A specialization of compareValuesWithOffset for nullable. template -static int compareValuesWithOffsetNullable( - const IColumn * _compared_column, - size_t compared_row, - const IColumn * _reference_column, +static int compareValuesWithOffsetNullable(const IColumn * _compared_column, + size_t compared_row, const IColumn * _reference_column, size_t reference_row, const Field & _offset, bool offset_is_preceding) { - const auto * compared_column = assert_cast(_compared_column); - const auto * reference_column = assert_cast(_reference_column); + const auto * compared_column = assert_cast( + _compared_column); + const auto * reference_column = assert_cast( + _reference_column); if (compared_column->isNullAt(compared_row) && !reference_column->isNullAt(reference_row)) { @@ -225,59 +229,54 @@ static int compareValuesWithOffsetNullable( ColumnPtr nest_reference_column = reference_column->getNestedColumnPtr(); std::function - nest_compare_function; + bool offset_is_preceding)> nest_compare_function; APPLY_FOR_NEST_TYPES(compareValuesWithOffset) - return nest_compare_function( - nest_compared_column.get(), compared_row, nest_reference_column.get(), reference_row, _offset, offset_is_preceding); + return nest_compare_function(nest_compared_column.get(), compared_row, + nest_reference_column.get(), reference_row, _offset, offset_is_preceding); } // Helper macros to dispatch on type of the ORDER BY column #define APPLY_FOR_ONE_TYPE(FUNCTION, TYPE) \ - else if (typeid_cast(column)) \ - { \ - /* clang-tidy you're dumb, I can't put FUNCTION in braces here. */ \ - compare_values_with_offset = FUNCTION; /* NOLINT */ \ - } +else if (typeid_cast(column)) \ +{ \ + /* clang-tidy you're dumb, I can't put FUNCTION in braces here. */ \ + compare_values_with_offset = FUNCTION; /* NOLINT */ \ +} #define APPLY_FOR_TYPES(FUNCTION) \ - if (false) /* NOLINT */ \ - { \ - /* Do nothing, a starter condition. */ \ - } \ - APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ - APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ - APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ - APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ +if (false) /* NOLINT */ \ +{ \ + /* Do nothing, a starter condition. */ \ +} \ +APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ \ - APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ - APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ - APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ - APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ - APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ \ - APPLY_FOR_ONE_TYPE(FUNCTION##Float, ColumnVector) \ - APPLY_FOR_ONE_TYPE(FUNCTION##Float, ColumnVector) \ +APPLY_FOR_ONE_TYPE(FUNCTION##Float, ColumnVector) \ +APPLY_FOR_ONE_TYPE(FUNCTION##Float, ColumnVector) \ \ - APPLY_FOR_ONE_TYPE(FUNCTION##Nullable, ColumnNullable) \ - else \ - { \ - throw Exception( \ - ErrorCodes::NOT_IMPLEMENTED, \ - "The RANGE OFFSET frame for '{}' ORDER BY column is not implemented", \ - demangle(typeid(*column).name())); \ - } +APPLY_FOR_ONE_TYPE(FUNCTION##Nullable, ColumnNullable) \ +else \ +{ \ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, \ + "The RANGE OFFSET frame for '{}' ORDER BY column is not implemented", \ + demangle(typeid(*column).name())); \ +} -WindowTransform::WindowTransform( - const Block & input_header_, - const Block & output_header_, - const WindowDescription & window_description_, - const std::vector & functions) +WindowTransform::WindowTransform(const Block & input_header_, + const Block & output_header_, + const WindowDescription & window_description_, + const std::vector & functions) : IProcessor({input_header_}, {output_header_}) , input(inputs.front()) , output(outputs.front()) @@ -308,7 +307,8 @@ WindowTransform::WindowTransform( workspace.argument_column_indices.reserve(f.argument_names.size()); for (const auto & argument_name : f.argument_names) { - workspace.argument_column_indices.push_back(input_header.getPositionByName(argument_name)); + workspace.argument_column_indices.push_back( + input_header.getPositionByName(argument_name)); } workspace.argument_columns.assign(f.argument_names.size(), nullptr); @@ -325,7 +325,9 @@ WindowTransform::WindowTransform( } workspace.is_aggregate_function_state = workspace.aggregate_function->isState(); - workspace.aggregate_function_state.reset(aggregate_function->sizeOfData(), aggregate_function->alignOfData()); + workspace.aggregate_function_state.reset( + aggregate_function->sizeOfData(), + aggregate_function->alignOfData()); aggregate_function->create(workspace.aggregate_function_state.data()); workspaces.push_back(std::move(workspace)); @@ -334,20 +336,24 @@ WindowTransform::WindowTransform( partition_by_indices.reserve(window_description.partition_by.size()); for (const auto & column : window_description.partition_by) { - partition_by_indices.push_back(input_header.getPositionByName(column.column_name)); + partition_by_indices.push_back( + input_header.getPositionByName(column.column_name)); } order_by_indices.reserve(window_description.order_by.size()); for (const auto & column : window_description.order_by) { - order_by_indices.push_back(input_header.getPositionByName(column.column_name)); + order_by_indices.push_back( + input_header.getPositionByName(column.column_name)); } // Choose a row comparison function for RANGE OFFSET frame based on the // type of the ORDER BY column. if (window_description.frame.type == WindowFrame::FrameType::RANGE - && (window_description.frame.begin_type == WindowFrame::BoundaryType::Offset - || window_description.frame.end_type == WindowFrame::BoundaryType::Offset)) + && (window_description.frame.begin_type + == WindowFrame::BoundaryType::Offset + || window_description.frame.end_type + == WindowFrame::BoundaryType::Offset)) { assert(order_by_indices.size() == 1); const auto & entry = input_header.getByPosition(order_by_indices[0]); @@ -357,26 +363,32 @@ WindowTransform::WindowTransform( // Convert the offsets to the ORDER BY column type. We can't just check // that the type matches, because e.g. the int literals are always // (U)Int64, but the column might be Int8 and so on. - if (window_description.frame.begin_type == WindowFrame::BoundaryType::Offset) + if (window_description.frame.begin_type + == WindowFrame::BoundaryType::Offset) { - window_description.frame.begin_offset = convertFieldToTypeOrThrow(window_description.frame.begin_offset, *entry.type); + window_description.frame.begin_offset = convertFieldToTypeOrThrow( + window_description.frame.begin_offset, + *entry.type); - if (applyVisitor(FieldVisitorAccurateLess{}, window_description.frame.begin_offset, Field(0))) + if (applyVisitor(FieldVisitorAccurateLess{}, + window_description.frame.begin_offset, Field(0))) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Window frame start offset must be nonnegative, {} given", window_description.frame.begin_offset); } } - if (window_description.frame.end_type == WindowFrame::BoundaryType::Offset) + if (window_description.frame.end_type + == WindowFrame::BoundaryType::Offset) { - window_description.frame.end_offset = convertFieldToTypeOrThrow(window_description.frame.end_offset, *entry.type); + window_description.frame.end_offset = convertFieldToTypeOrThrow( + window_description.frame.end_offset, + *entry.type); - if (applyVisitor(FieldVisitorAccurateLess{}, window_description.frame.end_offset, Field(0))) + if (applyVisitor(FieldVisitorAccurateLess{}, + window_description.frame.end_offset, Field(0))) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Window frame start offset must be nonnegative, {} given", window_description.frame.end_offset); } @@ -389,10 +401,11 @@ WindowTransform::WindowTransform( { if (!workspace.window_function_impl->checkWindowFrameType(this)) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Unsupported window frame type for function '{}'", workspace.aggregate_function->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported window frame type for function '{}'", + workspace.aggregate_function->getName()); } } + } } @@ -401,7 +414,8 @@ WindowTransform::~WindowTransform() // Some states may be not created yet if the creation failed. for (auto & ws : workspaces) { - ws.aggregate_function->destroy(ws.aggregate_function_state.data()); + ws.aggregate_function->destroy( + ws.aggregate_function_state.data()); } } @@ -475,10 +489,14 @@ void WindowTransform::advancePartitionEnd() size_t i = 0; for (; i < partition_by_columns; ++i) { - const auto * reference_column = inputAt(prev_frame_start)[partition_by_indices[i]].get(); - const auto * compared_column = inputAt(partition_end)[partition_by_indices[i]].get(); + const auto * reference_column + = inputAt(prev_frame_start)[partition_by_indices[i]].get(); + const auto * compared_column + = inputAt(partition_end)[partition_by_indices[i]].get(); - if (compared_column->compareAt(partition_end.row, prev_frame_start.row, *reference_column, 1 /* nan_direction_hint */) != 0) + if (compared_column->compareAt(partition_end.row, + prev_frame_start.row, *reference_column, + 1 /* nan_direction_hint */) != 0) { break; } @@ -590,8 +608,9 @@ auto WindowTransform::moveRowNumber(const RowNumber & original_row_number, Int64 void WindowTransform::advanceFrameStartRowsOffset() { // Just recalculate it each time by walking blocks. - const auto [moved_row, offset_left] = moveRowNumber( - current_row, window_description.frame.begin_offset.get() * (window_description.frame.begin_preceding ? -1 : 1)); + const auto [moved_row, offset_left] = moveRowNumber(current_row, + window_description.frame.begin_offset.get() + * (window_description.frame.begin_preceding ? -1 : 1)); frame_start = moved_row; @@ -628,17 +647,21 @@ void WindowTransform::advanceFrameStartRangeOffset() { // See the comment for advanceFrameEndRangeOffset(). const int direction = window_description.order_by[0].direction; - const bool preceding = window_description.frame.begin_preceding == (direction > 0); - const auto * reference_column = inputAt(current_row)[order_by_indices[0]].get(); + const bool preceding = window_description.frame.begin_preceding + == (direction > 0); + const auto * reference_column + = inputAt(current_row)[order_by_indices[0]].get(); for (; frame_start < partition_end; advanceRowNumber(frame_start)) { // The first frame value is [current_row] with offset, so we advance // while [frames_start] < [current_row] with offset. - const auto * compared_column = inputAt(frame_start)[order_by_indices[0]].get(); - if (compare_values_with_offset( - compared_column, frame_start.row, reference_column, current_row.row, window_description.frame.begin_offset, preceding) - * direction - >= 0) + const auto * compared_column + = inputAt(frame_start)[order_by_indices[0]].get(); + if (compare_values_with_offset(compared_column, frame_start.row, + reference_column, current_row.row, + window_description.frame.begin_offset, + preceding) + * direction >= 0) { frame_started = true; return; @@ -683,8 +706,7 @@ void WindowTransform::advanceFrameStart() advanceFrameStartRangeOffset(); break; default: - throw Exception( - ErrorCodes::NOT_IMPLEMENTED, + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Frame start type '{}' for frame '{}' is not implemented", window_description.frame.begin_type, window_description.frame.type); @@ -744,7 +766,8 @@ bool WindowTransform::arePeers(const RowNumber & x, const RowNumber & y) const { const auto * column_x = inputAt(x)[order_by_indices[i]].get(); const auto * column_y = inputAt(y)[order_by_indices[i]].get(); - if (column_x->compareAt(x.row, y.row, *column_y, 1 /* nan_direction_hint */) != 0) + if (column_x->compareAt(x.row, y.row, *column_y, + 1 /* nan_direction_hint */) != 0) { return false; } @@ -761,7 +784,8 @@ void WindowTransform::advanceFrameEndCurrentRow() // (only loop over rows and not over blocks), that should hopefully be more // efficient. // partition_end is either in this new block or past-the-end. - assert(frame_end.block == partition_end.block || frame_end.block + 1 == partition_end.block); + assert(frame_end.block == partition_end.block + || frame_end.block + 1 == partition_end.block); if (frame_end == partition_end) { @@ -823,8 +847,10 @@ void WindowTransform::advanceFrameEndRowsOffset() { // Walk the specified offset from the current row. The "+1" is needed // because the frame_end is a past-the-end pointer. - const auto [moved_row, offset_left] = moveRowNumber( - current_row, window_description.frame.end_offset.get() * (window_description.frame.end_preceding ? -1 : 1) + 1); + const auto [moved_row, offset_left] = moveRowNumber(current_row, + window_description.frame.end_offset.get() + * (window_description.frame.end_preceding ? -1 : 1) + + 1); if (partition_end <= moved_row) { @@ -857,18 +883,22 @@ void WindowTransform::advanceFrameEndRangeOffset() // PRECEDING/FOLLOWING change direction for DESC order. // See CD 9075-2:201?(E) 7.14 p. 429. const int direction = window_description.order_by[0].direction; - const bool preceding = window_description.frame.end_preceding == (direction > 0); - const auto * reference_column = inputAt(current_row)[order_by_indices[0]].get(); + const bool preceding = window_description.frame.end_preceding + == (direction > 0); + const auto * reference_column + = inputAt(current_row)[order_by_indices[0]].get(); for (; frame_end < partition_end; advanceRowNumber(frame_end)) { // The last frame value is current_row with offset, and we need a // past-the-end pointer, so we advance while // [frame_end] <= [current_row] with offset. - const auto * compared_column = inputAt(frame_end)[order_by_indices[0]].get(); - if (compare_values_with_offset( - compared_column, frame_end.row, reference_column, current_row.row, window_description.frame.end_offset, preceding) - * direction - > 0) + const auto * compared_column + = inputAt(frame_end)[order_by_indices[0]].get(); + if (compare_values_with_offset(compared_column, frame_end.row, + reference_column, current_row.row, + window_description.frame.end_offset, + preceding) + * direction > 0) { frame_ended = true; return; @@ -903,8 +933,9 @@ void WindowTransform::advanceFrameEnd() advanceFrameEndRangeOffset(); break; default: - throw Exception( - ErrorCodes::NOT_IMPLEMENTED, "The frame end type '{}' is not implemented", window_description.frame.end_type); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "The frame end type '{}' is not implemented", + window_description.frame.end_type); } break; } @@ -975,9 +1006,13 @@ void WindowTransform::updateAggregationState() // rows manually, instead of using advanceRowNumber(). // For this purpose, the past-the-end block can be different than the // block of the past-the-end row (it's usually the next block). - const auto past_the_end_block = rows_to_add_end.row == 0 ? rows_to_add_end.block : rows_to_add_end.block + 1; + const auto past_the_end_block = rows_to_add_end.row == 0 + ? rows_to_add_end.block + : rows_to_add_end.block + 1; - for (auto block_number = rows_to_add_start.block; block_number < past_the_end_block; ++block_number) + for (auto block_number = rows_to_add_start.block; + block_number < past_the_end_block; + ++block_number) { auto & block = blockAt(block_number); @@ -985,15 +1020,18 @@ void WindowTransform::updateAggregationState() { for (size_t i = 0; i < ws.argument_column_indices.size(); ++i) { - ws.argument_columns[i] = block.input_columns[ws.argument_column_indices[i]].get(); + ws.argument_columns[i] = block.input_columns[ + ws.argument_column_indices[i]].get(); } ws.cached_block_number = block_number; } // First and last blocks may be processed partially, and other blocks // are processed in full. - const auto first_row = block_number == rows_to_add_start.block ? rows_to_add_start.row : 0; - const auto past_the_end_row = block_number == rows_to_add_end.block ? rows_to_add_end.row : block.rows; + const auto first_row = block_number == rows_to_add_start.block + ? rows_to_add_start.row : 0; + const auto past_the_end_row = block_number == rows_to_add_end.block + ? rows_to_add_end.row : block.rows; // We should add an addBatch analog that can accept a starting offset. // For now, add the values one by one. @@ -1041,7 +1079,8 @@ void WindowTransform::writeOutCurrentRow() } } -static void assertSameColumns(const Columns & left_all, const Columns & right_all) +static void assertSameColumns(const Columns & left_all, + const Columns & right_all) { assert(left_all.size() == right_all.size()); @@ -1059,7 +1098,8 @@ static void assertSameColumns(const Columns & left_all, const Columns & right_al if (const auto * right_lc = typeid_cast(right_column)) right_column = right_lc->getDictionary().getNestedColumn().get(); - assert(typeid(*left_column).hash_code() == typeid(*right_column).hash_code()); + assert(typeid(*left_column).hash_code() + == typeid(*right_column).hash_code()); if (isColumnConst(*left_column)) { @@ -1120,7 +1160,8 @@ void WindowTransform::appendChunk(Chunk & chunk) if (ws.window_function_impl) block.casted_columns.push_back(ws.window_function_impl->castColumn(block.input_columns, ws.argument_column_indices)); - block.output_columns.push_back(ws.aggregate_function->getResultType()->createColumn()); + block.output_columns.push_back(ws.aggregate_function->getResultType() + ->createColumn()); block.output_columns.back()->reserve(block.rows); } @@ -1444,10 +1485,12 @@ void WindowTransform::work() // that the frame start can be further than current row for some frame specs // (e.g. EXCLUDE CURRENT ROW), so we have to check both. assert(prev_frame_start <= frame_start); - const auto first_used_block = std::min(next_output_block_number, std::min(prev_frame_start.block, current_row.block)); + const auto first_used_block = std::min(next_output_block_number, + std::min(prev_frame_start.block, current_row.block)); if (first_block_number < first_used_block) { - blocks.erase(blocks.begin(), blocks.begin() + (first_used_block - first_block_number)); + blocks.erase(blocks.begin(), + blocks.begin() + (first_used_block - first_block_number)); first_block_number = first_used_block; assert(next_output_block_number >= first_block_number); @@ -1458,82 +1501,83 @@ void WindowTransform::work() } } - struct WindowFunctionRank final : public WindowFunction { - WindowFunctionRank(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionRank(const std::string & name_, + const DataTypes & argument_types_, const Array & parameters_) : WindowFunction(name_, argument_types_, parameters_, std::make_shared()) - { - } + {} bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, + size_t function_index) const override { - IColumn & to = *transform->blockAt(transform->current_row).output_columns[function_index]; - assert_cast(to).getData().push_back(transform->peer_group_start_row_number); + IColumn & to = *transform->blockAt(transform->current_row) + .output_columns[function_index]; + assert_cast(to).getData().push_back( + transform->peer_group_start_row_number); } }; struct WindowFunctionDenseRank final : public WindowFunction { - WindowFunctionDenseRank(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionDenseRank(const std::string & name_, + const DataTypes & argument_types_, const Array & parameters_) : WindowFunction(name_, argument_types_, parameters_, std::make_shared()) - { - } + {} bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, + size_t function_index) const override { - IColumn & to = *transform->blockAt(transform->current_row).output_columns[function_index]; - assert_cast(to).getData().push_back(transform->peer_group_number); + IColumn & to = *transform->blockAt(transform->current_row) + .output_columns[function_index]; + assert_cast(to).getData().push_back( + transform->peer_group_number); } }; namespace recurrent_detail { -template -T getValue(const WindowTransform * /*transform*/, size_t /*function_index*/, size_t /*column_index*/, RowNumber /*row*/) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "recurrent_detail::getValue() is not implemented for {} type", typeid(T).name()); -} + template T getValue(const WindowTransform * /*transform*/, size_t /*function_index*/, size_t /*column_index*/, RowNumber /*row*/) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "recurrent_detail::getValue() is not implemented for {} type", typeid(T).name()); + } -template <> -Float64 getValue(const WindowTransform * transform, size_t function_index, size_t column_index, RowNumber row) -{ - const auto & workspace = transform->workspaces[function_index]; - const auto & column = transform->blockAt(row.block).input_columns[workspace.argument_column_indices[column_index]]; - return column->getFloat64(row.row); -} + template<> Float64 getValue(const WindowTransform * transform, size_t function_index, size_t column_index, RowNumber row) + { + const auto & workspace = transform->workspaces[function_index]; + const auto & column = transform->blockAt(row.block).input_columns[workspace.argument_column_indices[column_index]]; + return column->getFloat64(row.row); + } -template -void setValueToOutputColumn(const WindowTransform * /*transform*/, size_t /*function_index*/, T /*value*/) -{ - throw Exception( - ErrorCodes::NOT_IMPLEMENTED, "recurrent_detail::setValueToOutputColumn() is not implemented for {} type", typeid(T).name()); -} + template void setValueToOutputColumn(const WindowTransform * /*transform*/, size_t /*function_index*/, T /*value*/) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "recurrent_detail::setValueToOutputColumn() is not implemented for {} type", typeid(T).name()); + } -template <> -void setValueToOutputColumn(const WindowTransform * transform, size_t function_index, Float64 value) -{ - auto current_row = transform->current_row; - const auto & current_block = transform->blockAt(current_row); - IColumn & to = *current_block.output_columns[function_index]; + template<> void setValueToOutputColumn(const WindowTransform * transform, size_t function_index, Float64 value) + { + auto current_row = transform->current_row; + const auto & current_block = transform->blockAt(current_row); + IColumn & to = *current_block.output_columns[function_index]; - assert_cast(to).getData().push_back(value); -} + assert_cast(to).getData().push_back(value); + } } struct WindowFunctionHelpers { - template + template static T getValue(const WindowTransform * transform, size_t function_index, size_t column_index, RowNumber row) { return recurrent_detail::getValue(transform, function_index, column_index, row); } - template + template static void setValueToOutputColumn(const WindowTransform * transform, size_t function_index, T value) { recurrent_detail::setValueToOutputColumn(transform, function_index, value); @@ -1568,7 +1612,6 @@ struct WindowFunctionHelpers } }; - struct ExponentialTimeDecayedSumState { Float64 previous_time; @@ -1591,34 +1634,34 @@ struct WindowFunctionExponentialTimeDecayedSum final : public StatefulWindowFunc { if (parameters_.size() != 1) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one parameter", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} takes exactly one parameter", name_); } return applyVisitor(FieldVisitorConvertToNumber(), parameters_[0]); } - WindowFunctionExponentialTimeDecayedSum(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionExponentialTimeDecayedSum(const std::string & name_, + const DataTypes & argument_types_, const Array & parameters_) : StatefulWindowFunction(name_, argument_types_, parameters_, std::make_shared()) , decay_length(getDecayLength(parameters_, name_)) { if (argument_types.size() != 2) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly two arguments", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} takes exactly two arguments", name_); } if (!isNumber(argument_types[ARGUMENT_VALUE])) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument {} must be a number, '{}' given", ARGUMENT_VALUE, argument_types[ARGUMENT_VALUE]->getName()); } - if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) - && !isDateTime64(argument_types[ARGUMENT_TIME])) + if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) && !isDateTime64(argument_types[ARGUMENT_TIME])) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument {} must be DateTime, DateTime64 or a number, '{}' given", ARGUMENT_TIME, argument_types[ARGUMENT_TIME]->getName()); @@ -1627,7 +1670,8 @@ struct WindowFunctionExponentialTimeDecayedSum final : public StatefulWindowFunc bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, + size_t function_index) const override { const auto & workspace = transform->workspaces[function_index]; auto & state = getState(workspace); @@ -1639,7 +1683,8 @@ struct WindowFunctionExponentialTimeDecayedSum final : public StatefulWindowFunc RowNumber frame_back = transform->prevRowNumber(transform->frame_end); Float64 back_t = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_TIME, frame_back); - if (transform->prev_frame_start <= transform->frame_start && transform->frame_start < transform->prev_frame_end + if (transform->prev_frame_start <= transform->frame_start + && transform->frame_start < transform->prev_frame_end && transform->prev_frame_end <= transform->frame_end) { for (RowNumber i = transform->prev_frame_start; i < transform->frame_start; transform->advanceRowNumber(i)) @@ -1673,8 +1718,8 @@ struct WindowFunctionExponentialTimeDecayedSum final : public StatefulWindowFunc WindowFunctionHelpers::setValueToOutputColumn(transform, function_index, result); } -private: - const Float64 decay_length; + private: + const Float64 decay_length; }; struct WindowFunctionExponentialTimeDecayedMax final : public WindowFunction @@ -1686,34 +1731,34 @@ struct WindowFunctionExponentialTimeDecayedMax final : public WindowFunction { if (parameters_.size() != 1) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one parameter", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} takes exactly one parameter", name_); } return applyVisitor(FieldVisitorConvertToNumber(), parameters_[0]); } - WindowFunctionExponentialTimeDecayedMax(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionExponentialTimeDecayedMax(const std::string & name_, + const DataTypes & argument_types_, const Array & parameters_) : WindowFunction(name_, argument_types_, parameters_, std::make_shared()) , decay_length(getDecayLength(parameters_, name_)) { if (argument_types.size() != 2) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly two arguments", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} takes exactly two arguments", name_); } if (!isNumber(argument_types[ARGUMENT_VALUE])) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument {} must be a number, '{}' given", ARGUMENT_VALUE, argument_types[ARGUMENT_VALUE]->getName()); } - if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) - && !isDateTime64(argument_types[ARGUMENT_TIME])) + if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) && !isDateTime64(argument_types[ARGUMENT_TIME])) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument {} must be DateTime, DateTime64 or a number, '{}' given", ARGUMENT_TIME, argument_types[ARGUMENT_TIME]->getName()); @@ -1722,7 +1767,8 @@ struct WindowFunctionExponentialTimeDecayedMax final : public WindowFunction bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, + size_t function_index) const override { Float64 result = std::numeric_limits::quiet_NaN(); @@ -1748,8 +1794,8 @@ struct WindowFunctionExponentialTimeDecayedMax final : public WindowFunction WindowFunctionHelpers::setValueToOutputColumn(transform, function_index, result); } -private: - const Float64 decay_length; + private: + const Float64 decay_length; }; struct WindowFunctionExponentialTimeDecayedCount final : public StatefulWindowFunction @@ -1760,25 +1806,26 @@ struct WindowFunctionExponentialTimeDecayedCount final : public StatefulWindowFu { if (parameters_.size() != 1) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one parameter", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} takes exactly one parameter", name_); } return applyVisitor(FieldVisitorConvertToNumber(), parameters_[0]); } - WindowFunctionExponentialTimeDecayedCount(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionExponentialTimeDecayedCount(const std::string & name_, + const DataTypes & argument_types_, const Array & parameters_) : StatefulWindowFunction(name_, argument_types_, parameters_, std::make_shared()) , decay_length(getDecayLength(parameters_, name_)) { if (argument_types.size() != 1) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one argument", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} takes exactly one argument", name_); } - if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) - && !isDateTime64(argument_types[ARGUMENT_TIME])) + if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) && !isDateTime64(argument_types[ARGUMENT_TIME])) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument {} must be DateTime, DateTime64 or a number, '{}' given", ARGUMENT_TIME, argument_types[ARGUMENT_TIME]->getName()); @@ -1787,7 +1834,8 @@ struct WindowFunctionExponentialTimeDecayedCount final : public StatefulWindowFu bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, + size_t function_index) const override { const auto & workspace = transform->workspaces[function_index]; auto & state = getState(workspace); @@ -1799,7 +1847,8 @@ struct WindowFunctionExponentialTimeDecayedCount final : public StatefulWindowFu RowNumber frame_back = transform->prevRowNumber(transform->frame_end); Float64 back_t = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_TIME, frame_back); - if (transform->prev_frame_start <= transform->frame_start && transform->frame_start < transform->prev_frame_end + if (transform->prev_frame_start <= transform->frame_start + && transform->frame_start < transform->prev_frame_end && transform->prev_frame_end <= transform->frame_end) { for (RowNumber i = transform->prev_frame_start; i < transform->frame_start; transform->advanceRowNumber(i)) @@ -1830,8 +1879,8 @@ struct WindowFunctionExponentialTimeDecayedCount final : public StatefulWindowFu WindowFunctionHelpers::setValueToOutputColumn(transform, function_index, result); } -private: - const Float64 decay_length; + private: + const Float64 decay_length; }; struct WindowFunctionExponentialTimeDecayedAvg final : public StatefulWindowFunction @@ -1843,34 +1892,34 @@ struct WindowFunctionExponentialTimeDecayedAvg final : public StatefulWindowFunc { if (parameters_.size() != 1) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly one parameter", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} takes exactly one parameter", name_); } return applyVisitor(FieldVisitorConvertToNumber(), parameters_[0]); } - WindowFunctionExponentialTimeDecayedAvg(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionExponentialTimeDecayedAvg(const std::string & name_, + const DataTypes & argument_types_, const Array & parameters_) : StatefulWindowFunction(name_, argument_types_, parameters_, std::make_shared()) , decay_length(getDecayLength(parameters_, name_)) { if (argument_types.size() != 2) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly two arguments", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} takes exactly two arguments", name_); } if (!isNumber(argument_types[ARGUMENT_VALUE])) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument {} must be a number, '{}' given", ARGUMENT_VALUE, argument_types[ARGUMENT_VALUE]->getName()); } - if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) - && !isDateTime64(argument_types[ARGUMENT_TIME])) + if (!isNumber(argument_types[ARGUMENT_TIME]) && !isDateTime(argument_types[ARGUMENT_TIME]) && !isDateTime64(argument_types[ARGUMENT_TIME])) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument {} must be DateTime, DateTime64 or a number, '{}' given", ARGUMENT_TIME, argument_types[ARGUMENT_TIME]->getName()); @@ -1879,7 +1928,8 @@ struct WindowFunctionExponentialTimeDecayedAvg final : public StatefulWindowFunc bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, + size_t function_index) const override { const auto & workspace = transform->workspaces[function_index]; auto & state = getState(workspace); @@ -1893,7 +1943,8 @@ struct WindowFunctionExponentialTimeDecayedAvg final : public StatefulWindowFunc RowNumber frame_back = transform->prevRowNumber(transform->frame_end); Float64 back_t = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_TIME, frame_back); - if (transform->prev_frame_start <= transform->frame_start && transform->frame_start < transform->prev_frame_end + if (transform->prev_frame_start <= transform->frame_start + && transform->frame_start < transform->prev_frame_end && transform->prev_frame_end <= transform->frame_end) { for (RowNumber i = transform->prev_frame_start; i < transform->frame_start; transform->advanceRowNumber(i)) @@ -1936,49 +1987,56 @@ struct WindowFunctionExponentialTimeDecayedAvg final : public StatefulWindowFunc state.previous_count = count; state.previous_time = back_t; - result = sum / count; + result = sum/count; } WindowFunctionHelpers::setValueToOutputColumn(transform, function_index, result); } -private: - const Float64 decay_length; + private: + const Float64 decay_length; }; struct WindowFunctionRowNumber final : public WindowFunction { - WindowFunctionRowNumber(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionRowNumber(const std::string & name_, + const DataTypes & argument_types_, const Array & parameters_) : WindowFunction(name_, argument_types_, parameters_, std::make_shared()) - { - } + {} bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, + size_t function_index) const override { - IColumn & to = *transform->blockAt(transform->current_row).output_columns[function_index]; - assert_cast(to).getData().push_back(transform->current_row_number); + IColumn & to = *transform->blockAt(transform->current_row) + .output_columns[function_index]; + assert_cast(to).getData().push_back( + transform->current_row_number); } }; namespace { -struct NtileState -{ - UInt64 buckets = 0; - RowNumber start_row; - UInt64 current_partition_rows = 0; - UInt64 current_partition_inserted_row = 0; + struct NtileState + { + UInt64 buckets = 0; + RowNumber start_row; + UInt64 current_partition_rows = 0; + UInt64 current_partition_inserted_row = 0; - void windowInsertResultInto(const WindowTransform * transform, size_t function_index, const DataTypes & argument_types); -}; + void windowInsertResultInto( + const WindowTransform * transform, + size_t function_index, + const DataTypes & argument_types); + }; } // Usage: ntile(n). n is the number of buckets. struct WindowFunctionNtile final : public StatefulWindowFunction { - WindowFunctionNtile(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionNtile(const std::string & name_, + const DataTypes & argument_types_, const Array & parameters_) : StatefulWindowFunction(name_, argument_types_, parameters_, std::make_shared()) { if (argument_types.size() != 1) @@ -1986,11 +2044,7 @@ struct WindowFunctionNtile final : public StatefulWindowFunction auto type_id = argument_types[0]->getTypeId(); if (type_id != TypeIndex::UInt8 && type_id != TypeIndex::UInt16 && type_id != TypeIndex::UInt32 && type_id != TypeIndex::UInt64) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "'{}' argument type must be an unsigned integer (not larger than 64-bit), got {}", - name_, - argument_types[0]->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "'{}' argument type must be an unsigned integer (not larger than 64-bit), got {}", name_, argument_types[0]->getName()); } bool allocatesMemoryInArena() const override { return false; } @@ -2021,13 +2075,13 @@ struct WindowFunctionNtile final : public StatefulWindowFunction std::optional getDefaultFrame() const override { WindowFrame frame; - frame.is_default = false; frame.type = WindowFrame::FrameType::ROWS; frame.end_type = WindowFrame::BoundaryType::Unbounded; return frame; } - void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, + size_t function_index) const override { const auto & workspace = transform->workspaces[function_index]; auto & state = getState(workspace); @@ -2037,81 +2091,84 @@ struct WindowFunctionNtile final : public StatefulWindowFunction namespace { -void NtileState::windowInsertResultInto(const WindowTransform * transform, size_t function_index, const DataTypes & argument_types) -{ - if (!buckets) [[unlikely]] + void NtileState::windowInsertResultInto( + const WindowTransform * transform, + size_t function_index, + const DataTypes & argument_types) { - const auto & current_block = transform->blockAt(transform->current_row); - const auto & workspace = transform->workspaces[function_index]; - const auto & arg_col = *current_block.original_input_columns[workspace.argument_column_indices[0]]; - if (!isColumnConst(arg_col)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of 'ntile' function must be a constant"); - auto type_id = argument_types[0]->getTypeId(); - if (type_id == TypeIndex::UInt8) - buckets = arg_col[transform->current_row.row].get(); - else if (type_id == TypeIndex::UInt16) - buckets = arg_col[transform->current_row.row].get(); - else if (type_id == TypeIndex::UInt32) - buckets = arg_col[transform->current_row.row].get(); - else if (type_id == TypeIndex::UInt64) - buckets = arg_col[transform->current_row.row].get(); - - if (!buckets) + if (!buckets) [[unlikely]] { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of 'ntile' function must be greater than zero"); - } - } - // new partition - if (WindowFunctionHelpers::checkPartitionEnterFirstRow(transform)) [[unlikely]] - { - current_partition_rows = 0; - current_partition_inserted_row = 0; - start_row = transform->current_row; - } - current_partition_rows++; + const auto & current_block = transform->blockAt(transform->current_row); + const auto & workspace = transform->workspaces[function_index]; + const auto & arg_col = *current_block.original_input_columns[workspace.argument_column_indices[0]]; + if (!isColumnConst(arg_col)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of 'ntile' function must be a constant"); + auto type_id = argument_types[0]->getTypeId(); + if (type_id == TypeIndex::UInt8) + buckets = arg_col[transform->current_row.row].get(); + else if (type_id == TypeIndex::UInt16) + buckets = arg_col[transform->current_row.row].get(); + else if (type_id == TypeIndex::UInt32) + buckets = arg_col[transform->current_row.row].get(); + else if (type_id == TypeIndex::UInt64) + buckets = arg_col[transform->current_row.row].get(); - // Only do the action when we meet the last row in this partition. - if (!WindowFunctionHelpers::checkPartitionEnterLastRow(transform)) - return; - - auto bucket_capacity = current_partition_rows / buckets; - auto capacity_diff = current_partition_rows - bucket_capacity * buckets; - - // bucket number starts from 1. - UInt64 bucket_num = 1; - while (current_partition_inserted_row < current_partition_rows) - { - auto current_bucket_capacity = bucket_capacity; - if (capacity_diff > 0) - { - current_bucket_capacity += 1; - capacity_diff--; - } - auto left_rows = current_bucket_capacity; - while (left_rows) - { - auto available_block_rows = transform->blockRowsNumber(start_row) - start_row.row; - IColumn & to = *transform->blockAt(start_row).output_columns[function_index]; - auto & pod_array = assert_cast(to).getData(); - if (left_rows < available_block_rows) + if (!buckets) { - pod_array.resize_fill(pod_array.size() + left_rows, bucket_num); - start_row.row += left_rows; - left_rows = 0; - } - else - { - pod_array.resize_fill(pod_array.size() + available_block_rows, bucket_num); - left_rows -= available_block_rows; - start_row.block++; - start_row.row = 0; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of 'ntile' function must be greater than zero"); } } - current_partition_inserted_row += current_bucket_capacity; - bucket_num += 1; + // new partition + if (WindowFunctionHelpers::checkPartitionEnterFirstRow(transform)) [[unlikely]] + { + current_partition_rows = 0; + current_partition_inserted_row = 0; + start_row = transform->current_row; + } + current_partition_rows++; + + // Only do the action when we meet the last row in this partition. + if (!WindowFunctionHelpers::checkPartitionEnterLastRow(transform)) + return; + + auto bucket_capacity = current_partition_rows / buckets; + auto capacity_diff = current_partition_rows - bucket_capacity * buckets; + + // bucket number starts from 1. + UInt64 bucket_num = 1; + while (current_partition_inserted_row < current_partition_rows) + { + auto current_bucket_capacity = bucket_capacity; + if (capacity_diff > 0) + { + current_bucket_capacity += 1; + capacity_diff--; + } + auto left_rows = current_bucket_capacity; + while (left_rows) + { + auto available_block_rows = transform->blockRowsNumber(start_row) - start_row.row; + IColumn & to = *transform->blockAt(start_row).output_columns[function_index]; + auto & pod_array = assert_cast(to).getData(); + if (left_rows < available_block_rows) + { + pod_array.resize_fill(pod_array.size() + left_rows, bucket_num); + start_row.row += left_rows; + left_rows = 0; + } + else + { + pod_array.resize_fill(pod_array.size() + available_block_rows, bucket_num); + left_rows -= available_block_rows; + start_row.block++; + start_row.row = 0; + } + } + current_partition_inserted_row += current_bucket_capacity; + bucket_num += 1; + } } } -} namespace { @@ -2125,20 +2182,21 @@ struct PercentRankState struct WindowFunctionPercentRank final : public StatefulWindowFunction { public: - WindowFunctionPercentRank(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionPercentRank(const std::string & name_, + const DataTypes & argument_types_, const Array & parameters_) : StatefulWindowFunction(name_, argument_types_, parameters_, std::make_shared()) - { - } + {} bool allocatesMemoryInArena() const override { return false; } bool checkWindowFrameType(const WindowTransform * transform) const override { - if (transform->window_description.frame != getDefaultFrame()) + auto default_window_frame = getDefaultFrame(); + if (transform->window_description.frame != default_window_frame) { LOG_ERROR( getLogger("WindowFunctionPercentRank"), - "Window frame for function 'percent_rank' should be '{}'", getDefaultFrame()->toString()); + "Window frame for function 'percent_rank' should be '{}'", default_window_frame->toString()); return false; } return true; @@ -2147,11 +2205,9 @@ public: std::optional getDefaultFrame() const override { WindowFrame frame; - frame.is_default = false; frame.type = WindowFrame::FrameType::RANGE; frame.begin_type = WindowFrame::BoundaryType::Unbounded; frame.end_type = WindowFrame::BoundaryType::Unbounded; - //frame.end_type = WindowFrame::BoundaryType::Current; return frame; } @@ -2227,12 +2283,14 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction { FunctionBasePtr func_cast = nullptr; - WindowFunctionLagLeadInFrame(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionLagLeadInFrame(const std::string & name_, + const DataTypes & argument_types_, const Array & parameters_) : WindowFunction(name_, argument_types_, parameters_, createResultType(argument_types_, name_)) { if (!parameters.empty()) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} cannot be parameterized", name_); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Function {} cannot be parameterized", name_); } if (argument_types.size() == 1) @@ -2242,7 +2300,9 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction if (!isInt64OrUInt64FieldType(argument_types[1]->getDefault().getType())) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Offset must be an integer, '{}' given", argument_types[1]->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Offset must be an integer, '{}' given", + argument_types[1]->getName()); } if (argument_types.size() == 2) @@ -2252,11 +2312,9 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction if (argument_types.size() > 3) { - throw Exception( - ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Function '{}' accepts at most 3 arguments, {} given", - name, - argument_types.size()); + name, argument_types.size()); } if (argument_types[0]->equals(*argument_types[2])) @@ -2265,16 +2323,14 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction const auto supertype = tryGetLeastSupertype(DataTypes{argument_types[0], argument_types[2]}); if (!supertype) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no supertype for the argument type '{}' and the default value type '{}'", argument_types[0]->getName(), argument_types[2]->getName()); } if (!argument_types[0]->equals(*supertype)) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The supertype '{}' for the argument type '{}' and the default value type '{}' is not the same as the argument type", supertype->getName(), argument_types[0]->getName(), @@ -2283,8 +2339,15 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction const auto from_name = argument_types[2]->getName(); const auto to_name = argument_types[0]->getName(); - ColumnsWithTypeAndName arguments{ - {argument_types[2], ""}, {DataTypeString().createColumnConst(0, to_name), std::make_shared(), ""}}; + ColumnsWithTypeAndName arguments + { + { argument_types[2], "" }, + { + DataTypeString().createColumnConst(0, to_name), + std::make_shared(), + "" + } + }; auto get_cast_func = [&arguments] { @@ -2293,6 +2356,7 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction }; func_cast = get_cast_func(); + } ColumnPtr castColumn(const Columns & columns, const std::vector & idx) override @@ -2300,11 +2364,15 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction if (!func_cast) return nullptr; - ColumnsWithTypeAndName arguments{ - {columns[idx[2]], argument_types[2], ""}, - {DataTypeString().createColumnConst(columns[idx[2]]->size(), argument_types[0]->getName()), - std::make_shared(), - ""}}; + ColumnsWithTypeAndName arguments + { + { columns[idx[2]], argument_types[2], "" }, + { + DataTypeString().createColumnConst(columns[idx[2]]->size(), argument_types[0]->getName()), + std::make_shared(), + "" + } + }; return func_cast->execute(arguments, argument_types[0], columns[idx[2]]->size()); } @@ -2313,7 +2381,8 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction { if (argument_types_.empty()) { - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} takes at least one argument", name_); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, + "Function {} takes at least one argument", name_); } return argument_types_[0]; @@ -2321,7 +2390,8 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, + size_t function_index) const override { const auto & current_block = transform->blockAt(transform->current_row); IColumn & to = *current_block.output_columns[function_index]; @@ -2330,27 +2400,34 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction Int64 offset = 1; if (argument_types.size() > 1) { - offset = (*current_block.input_columns[workspace.argument_column_indices[1]])[transform->current_row.row].get(); + offset = (*current_block.input_columns[ + workspace.argument_column_indices[1]])[ + transform->current_row.row].get(); /// Either overflow or really negative value, both is not acceptable. if (offset < 0) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "The offset for function {} must be in (0, {}], {} given", getName(), INT64_MAX, offset); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "The offset for function {} must be in (0, {}], {} given", + getName(), INT64_MAX, offset); } } - const auto [target_row, offset_left] = transform->moveRowNumber(transform->current_row, offset * (is_lead ? 1 : -1)); + const auto [target_row, offset_left] = transform->moveRowNumber( + transform->current_row, offset * (is_lead ? 1 : -1)); - if (offset_left != 0 || target_row < transform->frame_start || transform->frame_end <= target_row) + if (offset_left != 0 + || target_row < transform->frame_start + || transform->frame_end <= target_row) { // Offset is outside the frame. if (argument_types.size() > 2) { // Column with default values is specified. - const IColumn & default_column = current_block.casted_columns[function_index] - ? *current_block.casted_columns[function_index].get() - : *current_block.input_columns[workspace.argument_column_indices[2]].get(); + const IColumn & default_column = + current_block.casted_columns[function_index] ? + *current_block.casted_columns[function_index].get() : + *current_block.input_columns[workspace.argument_column_indices[2]].get(); to.insert(default_column[transform->current_row.row]); } @@ -2362,24 +2439,30 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction else { // Offset is inside the frame. - to.insertFrom(*transform->blockAt(target_row).input_columns[workspace.argument_column_indices[0]], target_row.row); + to.insertFrom(*transform->blockAt(target_row).input_columns[ + workspace.argument_column_indices[0]], + target_row.row); } } }; struct WindowFunctionNthValue final : public WindowFunction { - WindowFunctionNthValue(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionNthValue(const std::string & name_, + const DataTypes & argument_types_, const Array & parameters_) : WindowFunction(name_, argument_types_, parameters_, createResultType(name_, argument_types_)) { if (!parameters.empty()) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} cannot be parameterized", name_); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Function {} cannot be parameterized", name_); } if (!isInt64OrUInt64FieldType(argument_types[1]->getDefault().getType())) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Offset must be an integer, '{}' given", argument_types[1]->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Offset must be an integer, '{}' given", + argument_types[1]->getName()); } } @@ -2387,7 +2470,8 @@ struct WindowFunctionNthValue final : public WindowFunction { if (argument_types_.size() != 2) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes exactly two arguments", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} takes exactly two arguments", name_); } return argument_types_[0]; @@ -2395,24 +2479,30 @@ struct WindowFunctionNthValue final : public WindowFunction bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, + size_t function_index) const override { const auto & current_block = transform->blockAt(transform->current_row); IColumn & to = *current_block.output_columns[function_index]; const auto & workspace = transform->workspaces[function_index]; - Int64 offset = (*current_block.input_columns[workspace.argument_column_indices[1]])[transform->current_row.row].get(); + Int64 offset = (*current_block.input_columns[ + workspace.argument_column_indices[1]])[ + transform->current_row.row].get(); /// Either overflow or really negative value, both is not acceptable. if (offset <= 0) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "The offset for function {} must be in (0, {}], {} given", getName(), INT64_MAX, offset); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "The offset for function {} must be in (0, {}], {} given", + getName(), INT64_MAX, offset); } --offset; const auto [target_row, offset_left] = transform->moveRowNumber(transform->frame_start, offset); - if (offset_left != 0 || target_row < transform->frame_start || transform->frame_end <= target_row) + if (offset_left != 0 + || target_row < transform->frame_start + || transform->frame_end <= target_row) { // Offset is outside the frame. to.insertDefault(); @@ -2420,7 +2510,9 @@ struct WindowFunctionNthValue final : public WindowFunction else { // Offset is inside the frame. - to.insertFrom(*transform->blockAt(target_row).input_columns[workspace.argument_column_indices[0]], target_row.row); + to.insertFrom(*transform->blockAt(target_row).input_columns[ + workspace.argument_column_indices[0]], + target_row.row); } } }; @@ -2441,34 +2533,35 @@ struct NonNegativeDerivativeParams bool interval_specified = false; Int64 ts_scale_multiplier = 0; - NonNegativeDerivativeParams(const std::string & name_, const DataTypes & argument_types, const Array & parameters) + NonNegativeDerivativeParams( + const std::string & name_, const DataTypes & argument_types, const Array & parameters) { if (!parameters.empty()) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} cannot be parameterized", name_); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Function {} cannot be parameterized", name_); } if (argument_types.size() != 2 && argument_types.size() != 3) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes 2 or 3 arguments", name_); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} takes 2 or 3 arguments", name_); } if (!isNumber(argument_types[ARGUMENT_METRIC])) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Argument {} must be a number, '{}' given", - ARGUMENT_METRIC, - argument_types[ARGUMENT_METRIC]->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument {} must be a number, '{}' given", + ARGUMENT_METRIC, + argument_types[ARGUMENT_METRIC]->getName()); } if (!isDateTime(argument_types[ARGUMENT_TIMESTAMP]) && !isDateTime64(argument_types[ARGUMENT_TIMESTAMP])) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Argument {} must be DateTime or DateTime64, '{}' given", - ARGUMENT_TIMESTAMP, - argument_types[ARGUMENT_TIMESTAMP]->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument {} must be DateTime or DateTime64, '{}' given", + ARGUMENT_TIMESTAMP, + argument_types[ARGUMENT_TIMESTAMP]->getName()); } if (isDateTime64(argument_types[ARGUMENT_TIMESTAMP])) @@ -2502,28 +2595,27 @@ struct NonNegativeDerivativeParams }; // nonNegativeDerivative(metric_column, timestamp_column[, INTERVAL 1 SECOND]) -struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction, - public NonNegativeDerivativeParams +struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction, public NonNegativeDerivativeParams { using Params = NonNegativeDerivativeParams; - WindowFunctionNonNegativeDerivative(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + WindowFunctionNonNegativeDerivative(const std::string & name_, + const DataTypes & argument_types_, const Array & parameters_) : StatefulWindowFunction(name_, argument_types_, parameters_, std::make_shared()) , NonNegativeDerivativeParams(name, argument_types, parameters) - { - } + {} bool allocatesMemoryInArena() const override { return false; } - void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override + void windowInsertResultInto(const WindowTransform * transform, + size_t function_index) const override { const auto & current_block = transform->blockAt(transform->current_row); const auto & workspace = transform->workspaces[function_index]; auto & state = getState(workspace); - auto interval_duration = interval_specified - ? interval_length * (*current_block.input_columns[workspace.argument_column_indices[ARGUMENT_INTERVAL]]).getFloat64(0) - : 1; + auto interval_duration = interval_specified ? interval_length * + (*current_block.input_columns[workspace.argument_column_indices[ARGUMENT_INTERVAL]]).getFloat64(0) : 1; Float64 curr_metric = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_METRIC, transform->current_row); Float64 metric_diff = curr_metric - state.previous_metric; @@ -2531,18 +2623,16 @@ struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction if (ts_scale_multiplier) { - const auto & column - = transform->blockAt(transform->current_row.block).input_columns[workspace.argument_column_indices[ARGUMENT_TIMESTAMP]]; + const auto & column = transform->blockAt(transform->current_row.block).input_columns[workspace.argument_column_indices[ARGUMENT_TIMESTAMP]]; const auto & curr_timestamp = checkAndGetColumn(*column).getInt(transform->current_row.row); Float64 time_elapsed = curr_timestamp - state.previous_timestamp; - result = (time_elapsed > 0) ? (metric_diff * ts_scale_multiplier / time_elapsed * interval_duration) : 0; + result = (time_elapsed > 0) ? (metric_diff * ts_scale_multiplier / time_elapsed * interval_duration) : 0; state.previous_timestamp = curr_timestamp; } else { - Float64 curr_timestamp - = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_TIMESTAMP, transform->current_row); + Float64 curr_timestamp = WindowFunctionHelpers::getValue(transform, function_index, ARGUMENT_TIMESTAMP, transform->current_row); Float64 time_elapsed = curr_timestamp - state.previous_timestamp; result = (time_elapsed > 0) ? (metric_diff / time_elapsed * interval_duration) : 0; state.previous_timestamp = curr_timestamp; @@ -2682,5 +2772,4 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) name, argument_types, parameters); }, properties}); } - } From 6e7bffa6ea100ad0f966d3e608eaf2462f6771f8 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 2 Aug 2024 16:13:09 +0800 Subject: [PATCH 355/644] remove unused codes --- src/Planner/PlannerActionsVisitor.cpp | 30 --------------------------- src/Planner/PlannerActionsVisitor.h | 12 ----------- 2 files changed, 42 deletions(-) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index f6c2c92cbb4..071da91839b 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -1040,36 +1040,6 @@ String calculateConstantActionNodeName(const Field & constant_literal) return ActionNodeNameHelper::calculateConstantActionNodeName(constant_literal); } -String calculateWindowNodeActionName(const QueryTreeNodePtr & node, - const PlannerContext & planner_context, - QueryTreeNodeToName & node_to_name, - bool use_column_identifier_as_action_node_name) -{ - ActionNodeNameHelper helper(node_to_name, planner_context, use_column_identifier_as_action_node_name); - auto get_window_frame = [&]()-> std::optional{ - auto & window_node = node->as(); - auto & window_frame = window_node.getWindowFrame(); - if (!window_frame.is_default) - return window_frame; - return {}; - }; - return helper.calculateWindowNodeActionName(node, get_window_frame); -} - -String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, bool use_column_identifier_as_action_node_name) -{ - QueryTreeNodeToName empty_map; - ActionNodeNameHelper helper(empty_map, planner_context, use_column_identifier_as_action_node_name); - auto get_window_frame = [&]()-> std::optional{ - auto & window_node = node->as(); - auto & window_frame = window_node.getWindowFrame(); - if (!window_frame.is_default) - return window_frame; - return {}; - }; - return helper.calculateWindowNodeActionName(node, get_window_frame); -} - String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, std::function()> get_window_frame, bool use_column_identifier_as_action_node_name) { QueryTreeNodeToName empty_map; diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 78d7c69357a..a7f95a38169 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -75,18 +75,6 @@ String calculateConstantActionNodeName(const Field & constant_literal); * Window node action name can only be part of window function action name. * For column node column node identifier from planner context is used, if use_column_identifier_as_action_node_name = true. */ -String calculateWindowNodeActionName(const QueryTreeNodePtr & node, - const PlannerContext & planner_context, - QueryTreeNodeToName & node_to_name, - bool use_column_identifier_as_action_node_name = true); - -/** Calculate action node name for window node. - * Window node action name can only be part of window function action name. - * For column node column node identifier from planner context is used, if use_column_identifier_as_action_node_name = true. - */ -String calculateWindowNodeActionName(const QueryTreeNodePtr & node, - const PlannerContext & planner_context, - bool use_column_identifier_as_action_node_name = true); String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, std::function()> get_window_frame, From b35dd3bc02a2082f9a3d6a6e507805d9e1bd87a9 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 2 Aug 2024 16:22:36 +0800 Subject: [PATCH 356/644] simplify codes --- src/Planner/PlannerActionsVisitor.cpp | 30 +++++++++++++++----------- src/Planner/PlannerActionsVisitor.h | 1 + src/Planner/PlannerWindowFunctions.cpp | 13 +---------- 3 files changed, 19 insertions(+), 25 deletions(-) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 071da91839b..288669e7050 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -237,19 +237,7 @@ public: if (function_node.isWindowFunction()) { - auto get_window_frame = [&]() -> std::optional - { - auto & window_node = function_node.getWindowNode()->as(); - auto & window_frame = window_node.getWindowFrame(); - if (!window_frame.is_default) - return window_frame; - auto aggregate_function = function_node.getAggregateFunction(); - if (const auto * win_func = dynamic_cast(aggregate_function.get())) - { - return win_func->getDefaultFrame(); - } - return {}; - }; + auto get_window_frame = [&]() { return extractWindowFrame(function_node); }; buffer << " OVER ("; buffer << calculateWindowNodeActionName(function_node.getWindowNode(), get_window_frame); buffer << ')'; @@ -1040,6 +1028,22 @@ String calculateConstantActionNodeName(const Field & constant_literal) return ActionNodeNameHelper::calculateConstantActionNodeName(constant_literal); } +std::optional extractWindowFrame(const FunctionNode & node) +{ + if (!node.isWindowFunction()) + return {}; + auto & window_node = node.getWindowNode()->as(); + const auto & window_frame = window_node.getWindowFrame(); + if (!window_frame.is_default) + return window_frame; + auto aggregate_function = node.getAggregateFunction(); + if (const auto * win_func = dynamic_cast(aggregate_function.get())) + { + return win_func->getDefaultFrame(); + } + return {}; +} + String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, std::function()> get_window_frame, bool use_column_identifier_as_action_node_name) { QueryTreeNodeToName empty_map; diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index a7f95a38169..71b8accb2a0 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -71,6 +71,7 @@ String calculateConstantActionNodeName(const Field & constant_literal, const Dat /// Calculate action node name for constant, data type will be derived from constant literal value String calculateConstantActionNodeName(const Field & constant_literal); +std::optional extractWindowFrame(const FunctionNode & node); /** Calculate action node name for window node. * Window node action name can only be part of window function action name. * For column node column node identifier from planner context is used, if use_column_identifier_as_action_node_name = true. diff --git a/src/Planner/PlannerWindowFunctions.cpp b/src/Planner/PlannerWindowFunctions.cpp index 2a28787ba96..a69dd95a650 100644 --- a/src/Planner/PlannerWindowFunctions.cpp +++ b/src/Planner/PlannerWindowFunctions.cpp @@ -29,18 +29,7 @@ WindowDescription extractWindowDescriptionFromWindowNode(const FunctionNode & fu auto node = func_node.getWindowNode(); auto & window_node = node->as(); - auto get_window_frame = [&]() -> std::optional - { - auto frame = window_node.getWindowFrame(); - if (!frame.is_default) - return frame; - auto aggregate_function = func_node.getAggregateFunction(); - if (const auto * win_func = dynamic_cast(aggregate_function.get())) - { - return win_func->getDefaultFrame(); - } - return {}; - }; + auto get_window_frame = [&]() { return extractWindowFrame(func_node); }; WindowDescription window_description; window_description.window_name = calculateWindowNodeActionName(node, planner_context, get_window_frame); From 2e521e17edcb273cfcb59c2224280a6f8e9f73da Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Sat, 3 Aug 2024 06:02:50 +0800 Subject: [PATCH 357/644] fixed --- src/Planner/PlannerActionsVisitor.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 288669e7050..7758d0c129f 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -1039,9 +1039,11 @@ std::optional extractWindowFrame(const FunctionNode & node) auto aggregate_function = node.getAggregateFunction(); if (const auto * win_func = dynamic_cast(aggregate_function.get())) { - return win_func->getDefaultFrame(); + auto function_default_window_frame = win_func->getDefaultFrame(); + if (function_default_window_frame) + return function_default_window_frame; } - return {}; + return window_frame; } String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, std::function()> get_window_frame, bool use_column_identifier_as_action_node_name) From 682c735fa6c128a9f5df82f1a776186ed3ea1065 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Sat, 3 Aug 2024 06:14:52 +0800 Subject: [PATCH 358/644] update tests --- .../01592_window_functions.reference | 24 --------- .../0_stateless/01592_window_functions.sql | 30 ----------- .../0_stateless/03037_precent_rank.reference | 22 ++++++++ .../0_stateless/03037_precent_rank.sql | 53 +++++++++++++++++++ 4 files changed, 75 insertions(+), 54 deletions(-) create mode 100644 tests/queries/0_stateless/03037_precent_rank.reference create mode 100644 tests/queries/0_stateless/03037_precent_rank.sql diff --git a/tests/queries/0_stateless/01592_window_functions.reference b/tests/queries/0_stateless/01592_window_functions.reference index 558f643c281..ec957dd7a02 100644 --- a/tests/queries/0_stateless/01592_window_functions.reference +++ b/tests/queries/0_stateless/01592_window_functions.reference @@ -79,27 +79,3 @@ iPhone 900 Smartphone 500 500 Kindle Fire 150 Tablet 150 350 Samsung Galaxy Tab 200 Tablet 175 350 iPad 700 Tablet 350 350 ----- Q8 ---- -Lenovo Thinkpad Laptop 700 1 0 -Sony VAIO Laptop 700 1 0 -Dell Vostro Laptop 800 3 0.6666666666666666 -HP Elite Laptop 1200 4 1 -Microsoft Lumia Smartphone 200 1 0 -HTC One Smartphone 400 2 0.3333333333333333 -Nexus Smartphone 500 3 0.6666666666666666 -iPhone Smartphone 900 4 1 -Kindle Fire Tablet 150 1 0 -Samsung Galaxy Tab Tablet 200 2 0.5 -iPad Tablet 700 3 1 -Others Unknow 200 1 0 ----- Q9 ---- -0 1 0 -1 2 1 -2 3 2 -3 4 3 -4 5 4 -5 6 5 -6 7 6 -7 8 7 -8 9 8 -9 10 9 diff --git a/tests/queries/0_stateless/01592_window_functions.sql b/tests/queries/0_stateless/01592_window_functions.sql index 32c53763e40..c6bb23bc7cf 100644 --- a/tests/queries/0_stateless/01592_window_functions.sql +++ b/tests/queries/0_stateless/01592_window_functions.sql @@ -101,37 +101,7 @@ SELECT FROM products INNER JOIN product_groups USING (group_id)) t order by group_name, product_name, price; -select '---- Q8 ----'; -INSERT INTO product_groups VALUES (4, 'Unknow'); -INSERT INTO products (product_id,product_name, group_id,price) VALUES (12, 'Others', 4, 200); - -SELECT * -FROM -( - SELECT - product_name, - group_name, - price, - rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS rank, - percent_rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS percent - FROM products - INNER JOIN product_groups USING (group_id) -) AS t -ORDER BY - group_name ASC, - price ASC, - product_name ASC; - drop table product_groups; drop table products; -select '---- Q9 ----'; -select number, row_number, cast(percent_rank * 10000 as Int32) as percent_rank -from ( - select number, row_number() over () as row_number, percent_rank() over (order by number) as percent_rank - from numbers(10000) - order by number - limit 10 -) -settings max_block_size=100; diff --git a/tests/queries/0_stateless/03037_precent_rank.reference b/tests/queries/0_stateless/03037_precent_rank.reference new file mode 100644 index 00000000000..6a23f3884cd --- /dev/null +++ b/tests/queries/0_stateless/03037_precent_rank.reference @@ -0,0 +1,22 @@ +Lenovo Thinkpad Laptop 700 1 0 +Sony VAIO Laptop 700 1 0 +Dell Vostro Laptop 800 3 0.6666666666666666 +HP Elite Laptop 1200 4 1 +Microsoft Lumia Smartphone 200 1 0 +HTC One Smartphone 400 2 0.3333333333333333 +Nexus Smartphone 500 3 0.6666666666666666 +iPhone Smartphone 900 4 1 +Kindle Fire Tablet 150 1 0 +Samsung Galaxy Tab Tablet 200 2 0.5 +iPad Tablet 700 3 1 +Others Unknow 200 1 0 +0 1 0 +1 2 1 +2 3 2 +3 4 3 +4 5 4 +5 6 5 +6 7 6 +7 8 7 +8 9 8 +9 10 9 diff --git a/tests/queries/0_stateless/03037_precent_rank.sql b/tests/queries/0_stateless/03037_precent_rank.sql new file mode 100644 index 00000000000..54880de53bc --- /dev/null +++ b/tests/queries/0_stateless/03037_precent_rank.sql @@ -0,0 +1,53 @@ +set allow_experimental_analyzer=1; +drop table if exists product_groups; +drop table if exists products; + +CREATE TABLE product_groups ( + group_id Int64, + group_name String +) Engine = Memory; + + +CREATE TABLE products ( + product_id Int64, + product_name String, + price DECIMAL(11, 2), + group_id Int64 +) Engine = Memory; + +INSERT INTO product_groups VALUES (1, 'Smartphone'),(2, 'Laptop'),(3, 'Tablet'); + +INSERT INTO products (product_id,product_name, group_id,price) VALUES (1, 'Microsoft Lumia', 1, 200), (2, 'HTC One', 1, 400), (3, 'Nexus', 1, 500), (4, 'iPhone', 1, 900),(5, 'HP Elite', 2, 1200),(6, 'Lenovo Thinkpad', 2, 700),(7, 'Sony VAIO', 2, 700),(8, 'Dell Vostro', 2, 800),(9, 'iPad', 3, 700),(10, 'Kindle Fire', 3, 150),(11, 'Samsung Galaxy Tab', 3, 200); + +INSERT INTO product_groups VALUES (4, 'Unknow'); +INSERT INTO products (product_id,product_name, group_id,price) VALUES (12, 'Others', 4, 200); + +SELECT * +FROM +( + SELECT + product_name, + group_name, + price, + rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS rank, + percent_rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS percent + FROM products + INNER JOIN product_groups USING (group_id) +) AS t +ORDER BY + group_name ASC, + price ASC, + product_name ASC; + +drop table product_groups; +drop table products; + +select number, row_number, cast(percent_rank * 10000 as Int32) as percent_rank +from ( + select number, row_number() over () as row_number, percent_rank() over (order by number) as percent_rank + from numbers(10000) + order by number + limit 10 +) +settings max_block_size=100; + From 8f5cf70aab732121072bf30a74e28d3f213f29ac Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Sat, 3 Aug 2024 06:29:22 +0800 Subject: [PATCH 359/644] add some comments --- src/Planner/PlannerActionsVisitor.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 71b8accb2a0..7b6b65eeb58 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -71,7 +71,11 @@ String calculateConstantActionNodeName(const Field & constant_literal, const Dat /// Calculate action node name for constant, data type will be derived from constant literal value String calculateConstantActionNodeName(const Field & constant_literal); +/// If the window frame is not set in sql, try to use the default frame from window function +/// if it have any one. Otherwise use the default window frame from `WindowNode`. +/// If the window frame is set in sql, use it anyway. std::optional extractWindowFrame(const FunctionNode & node); + /** Calculate action node name for window node. * Window node action name can only be part of window function action name. * For column node column node identifier from planner context is used, if use_column_identifier_as_action_node_name = true. From 08c48cf9444143b4642879f45b7e16bda5b9ccf3 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Sat, 3 Aug 2024 06:33:10 +0800 Subject: [PATCH 360/644] update --- src/Planner/PlannerActionsVisitor.cpp | 6 ++---- src/Planner/PlannerActionsVisitor.h | 2 +- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 7758d0c129f..288669e7050 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -1039,11 +1039,9 @@ std::optional extractWindowFrame(const FunctionNode & node) auto aggregate_function = node.getAggregateFunction(); if (const auto * win_func = dynamic_cast(aggregate_function.get())) { - auto function_default_window_frame = win_func->getDefaultFrame(); - if (function_default_window_frame) - return function_default_window_frame; + return win_func->getDefaultFrame(); } - return window_frame; + return {}; } String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, std::function()> get_window_frame, bool use_column_identifier_as_action_node_name) diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 7b6b65eeb58..17cce39f2a0 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -72,7 +72,7 @@ String calculateConstantActionNodeName(const Field & constant_literal, const Dat String calculateConstantActionNodeName(const Field & constant_literal); /// If the window frame is not set in sql, try to use the default frame from window function -/// if it have any one. Otherwise use the default window frame from `WindowNode`. +/// if it have any one. Otherwise return empty. /// If the window frame is set in sql, use it anyway. std::optional extractWindowFrame(const FunctionNode & node); From b8c6beeb7a86b823719631851477f58898a6b871 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Sat, 3 Aug 2024 07:21:31 +0800 Subject: [PATCH 361/644] update doc --- docs/en/sql-reference/window-functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 27d4bd763c7..668d831f4b1 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -24,7 +24,7 @@ ClickHouse supports the standard grammar for defining windows and window functio | `GROUPS` frame | ❌ | | Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) | | `rank()`, `dense_rank()`, `row_number()` | ✅
Alias: `denseRank()` | -| `percent_rank()` | ✅ Efficiently computes the relative standing of a value within a partition in a dataset. This function effectively replaces the more verbose and computationally intensive manual SQL calculation expressed as `ifNull((rank() OVER(PARTITION BY x ORDER BY y) - 1) / nullif(count(1) OVER(PARTITION BY x) - 1, 0), 0)`
Alias: `percentRank()`| +| `percent_rank()` | ✅ Efficiently computes the relative standing of a value within a partition in a dataset. This function effectively replaces the more verbose and computationally intensive manual SQL calculation expressed as `ifNull((rank() OVER(PARTITION BY x ORDER BY y) - 1) / nullif(count(1) OVER(PARTITION BY x) - 1, 0), 0)`. Need to use the new analyzer, set allow_experimental_analyzer=1.
Alias: `percentRank()`| | `lag/lead(value, offset)` | ❌
You can use one of the following workarounds:
1) `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead`
2) `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | | ntile(buckets) | ✅
Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | From 9845aeac0f71222d48a5be28e75df071d9d500c2 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 5 Aug 2024 10:18:28 +0800 Subject: [PATCH 362/644] support percent_rank in old analyzer --- .../sql-reference/window-functions/index.md | 2 +- src/Interpreters/ExpressionAnalyzer.cpp | 22 ++++++++++++++++--- src/Interpreters/ExpressionAnalyzer.h | 7 +++++- .../0_stateless/03037_precent_rank.sql | 1 - 4 files changed, 26 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 668d831f4b1..27d4bd763c7 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -24,7 +24,7 @@ ClickHouse supports the standard grammar for defining windows and window functio | `GROUPS` frame | ❌ | | Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) | | `rank()`, `dense_rank()`, `row_number()` | ✅
Alias: `denseRank()` | -| `percent_rank()` | ✅ Efficiently computes the relative standing of a value within a partition in a dataset. This function effectively replaces the more verbose and computationally intensive manual SQL calculation expressed as `ifNull((rank() OVER(PARTITION BY x ORDER BY y) - 1) / nullif(count(1) OVER(PARTITION BY x) - 1, 0), 0)`. Need to use the new analyzer, set allow_experimental_analyzer=1.
Alias: `percentRank()`| +| `percent_rank()` | ✅ Efficiently computes the relative standing of a value within a partition in a dataset. This function effectively replaces the more verbose and computationally intensive manual SQL calculation expressed as `ifNull((rank() OVER(PARTITION BY x ORDER BY y) - 1) / nullif(count(1) OVER(PARTITION BY x) - 1, 0), 0)`
Alias: `percentRank()`| | `lag/lead(value, offset)` | ❌
You can use one of the following workarounds:
1) `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead`
2) `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | | ntile(buckets) | ✅
Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index c767aeb2ec2..7063b2162a0 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -38,6 +38,7 @@ #include #include +#include #include #include @@ -590,6 +591,7 @@ void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAG & actions, Aggrega void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, const WindowDescriptions & existing_descriptions, + AggregateFunctionPtr aggregate_function, WindowDescription & desc, const IAST * ast) { const auto & definition = ast->as(); @@ -698,7 +700,21 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, ast->formatForErrorMessage()); } + const auto * window_function = aggregate_function ? dynamic_cast(aggregate_function.get()) : nullptr; desc.frame.is_default = definition.frame_is_default; + if (desc.frame.is_default && window_function) + { + auto default_window_frame_opt = window_function->getDefaultFrame(); + if (default_window_frame_opt) + { + desc.frame = *default_window_frame_opt; + /// Append the default frame description to window_name, make sure it will be put into + /// a proper window description. + desc.window_name += " " + desc.frame.toString(); + return; + } + } + desc.frame.type = definition.frame_type; desc.frame.begin_type = definition.frame_begin_type; desc.frame.begin_preceding = definition.frame_begin_preceding; @@ -734,7 +750,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAG & actions) WindowDescription desc; desc.window_name = elem.name; makeWindowDescriptionFromAST(*current_context, window_descriptions, - desc, elem.definition.get()); + nullptr, desc, elem.definition.get()); auto [it, inserted] = window_descriptions.insert( {elem.name, std::move(desc)}); @@ -821,12 +837,12 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAG & actions) WindowDescription desc; desc.window_name = default_window_name; makeWindowDescriptionFromAST(*current_context, window_descriptions, - desc, &definition); + window_function.aggregate_function, desc, &definition); auto full_sort_description = desc.full_sort_description; auto [it, inserted] = window_descriptions.insert( - {default_window_name, std::move(desc)}); + {desc.window_name, std::move(desc)}); if (!inserted) { diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 0c00247df85..dc038e10594 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -135,7 +135,12 @@ public: /// A list of windows for window functions. const WindowDescriptions & windowDescriptions() const { return window_descriptions; } - void makeWindowDescriptionFromAST(const Context & context, const WindowDescriptions & existing_descriptions, WindowDescription & desc, const IAST * ast); + void makeWindowDescriptionFromAST( + const Context & context, + const WindowDescriptions & existing_descriptions, + AggregateFunctionPtr aggregate_function, + WindowDescription & desc, + const IAST * ast); void makeWindowDescriptions(ActionsDAG & actions); /** Checks if subquery is not a plain StorageSet. diff --git a/tests/queries/0_stateless/03037_precent_rank.sql b/tests/queries/0_stateless/03037_precent_rank.sql index 54880de53bc..b0f83fa3340 100644 --- a/tests/queries/0_stateless/03037_precent_rank.sql +++ b/tests/queries/0_stateless/03037_precent_rank.sql @@ -1,4 +1,3 @@ -set allow_experimental_analyzer=1; drop table if exists product_groups; drop table if exists products; From 81b9d6d3f51192bee509cb76a8c731ac1aa2b388 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 5 Aug 2024 11:19:21 +0800 Subject: [PATCH 363/644] add doc --- .../window-functions/lagInFrame.md | 2 +- .../window-functions/leadInFrame.md | 2 +- .../window-functions/percent_rank.md | 72 +++++++++++++++++++ 3 files changed, 74 insertions(+), 2 deletions(-) create mode 100644 docs/en/sql-reference/window-functions/percent_rank.md diff --git a/docs/en/sql-reference/window-functions/lagInFrame.md b/docs/en/sql-reference/window-functions/lagInFrame.md index de6e9005baa..01bf809e76e 100644 --- a/docs/en/sql-reference/window-functions/lagInFrame.md +++ b/docs/en/sql-reference/window-functions/lagInFrame.md @@ -1,7 +1,7 @@ --- slug: /en/sql-reference/window-functions/lagInFrame sidebar_label: lagInFrame -sidebar_position: 8 +sidebar_position: 9 --- # lagInFrame diff --git a/docs/en/sql-reference/window-functions/leadInFrame.md b/docs/en/sql-reference/window-functions/leadInFrame.md index 4a82c03f6e6..dae4353b582 100644 --- a/docs/en/sql-reference/window-functions/leadInFrame.md +++ b/docs/en/sql-reference/window-functions/leadInFrame.md @@ -1,7 +1,7 @@ --- slug: /en/sql-reference/window-functions/leadInFrame sidebar_label: leadInFrame -sidebar_position: 9 +sidebar_position: 10 --- # leadInFrame diff --git a/docs/en/sql-reference/window-functions/percent_rank.md b/docs/en/sql-reference/window-functions/percent_rank.md new file mode 100644 index 00000000000..4b260f667b9 --- /dev/null +++ b/docs/en/sql-reference/window-functions/percent_rank.md @@ -0,0 +1,72 @@ +--- +slug: /en/sql-reference/window-functions/percent_rank +sidebar_label: percent_rank +sidebar_position: 8 +--- + +# percent_rank + +returns the relative rank (i.e. percentile) of rows within a window partition. + +**Syntax** + +Alias: `percentRank` (case-sensitive) + +```sql +percent_rank (column_name) + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] + [RANGE RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] | [window_name]) +FROM table_name +WINDOW window_name as ([PARTITION BY grouping_column] [ORDER BY sorting_column] RANGE RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +``` + +The default and required window frame definition is `RANGE RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING`. + +For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). + +**Example** + + +Query: + +```sql +CREATE TABLE salaries +( + `team` String, + `player` String, + `salary` UInt32, + `position` String +) +Engine = Memory; + +INSERT INTO salaries FORMAT Values + ('Port Elizabeth Barbarians', 'Gary Chen', 195000, 'F'), + ('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'), + ('Port Elizabeth Barbarians', 'Michael Stanley', 150000, 'D'), + ('New Coreystad Archdukes', 'Scott Harrison', 150000, 'D'), + ('Port Elizabeth Barbarians', 'Robert George', 195000, 'M'), + ('South Hampton Seagulls', 'Douglas Benson', 150000, 'M'), + ('South Hampton Seagulls', 'James Henderson', 140000, 'M'); +``` + +```sql +SELECT player, salary, + percent_rank() OVER (ORDER BY salary DESC) AS percent_rank +FROM salaries; +``` + +Result: + +```response + + ┌─player──────────┬─salary─┬───────percent_rank─┐ +1. │ Gary Chen │ 195000 │ 0 │ +2. │ Robert George │ 195000 │ 0 │ +3. │ Charles Juarez │ 190000 │ 0.3333333333333333 │ +4. │ Michael Stanley │ 150000 │ 0.5 │ +5. │ Scott Harrison │ 150000 │ 0.5 │ +6. │ Douglas Benson │ 150000 │ 0.5 │ +7. │ James Henderson │ 140000 │ 1 │ + └─────────────────┴────────┴────────────────────┘ + +``` From deb58b4ede1cabff8e4490fb39b2627ee94a347a Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 3 Jul 2024 18:23:38 +0800 Subject: [PATCH 364/644] any/anti/semi join support mixed join conditions --- src/Interpreters/HashJoin/HashJoin.cpp | 61 +++-- src/Interpreters/HashJoin/HashJoinMethods.h | 139 ++++++++---- src/Interpreters/HashJoin/JoinFeatures.h | 7 +- src/Interpreters/HashJoin/JoinUsedFlags.h | 33 ++- src/Interpreters/HashJoin/LeftHashJoin.cpp | 3 + src/Interpreters/joinDispatch.h | 108 +++++---- src/Storages/StorageJoin.cpp | 6 +- ..._join_on_inequal_expression_fast.reference | 214 ++++++++++++++++++ ...006_join_on_inequal_expression_fast.sql.j2 | 11 + 9 files changed, 469 insertions(+), 113 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 0c7cad4360d..769cb574ed7 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -291,12 +291,13 @@ void HashJoin::dataMapInit(MapsVariant & map) { if (kind == JoinKind::Cross) return; - joinDispatchInit(kind, strictness, map); - joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.create(data->type); }); + auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr; + joinDispatchInit(kind, strictness, map, prefer_use_maps_all); + joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { map_.create(data->type); }); if (reserve_num) { - joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.reserve(data->type, reserve_num); }); + joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { map_.reserve(data->type, reserve_num); }); } if (!data) @@ -327,9 +328,10 @@ size_t HashJoin::getTotalRowCount() const } else { + auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr; for (const auto & map : data->maps) { - joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { res += map_.getTotalRowCount(data->type); }); + joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { res += map_.getTotalRowCount(data->type); }); } } @@ -367,9 +369,10 @@ size_t HashJoin::getTotalByteCount() const if (data->type != Type::CROSS) { + auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr; for (const auto & map : data->maps) { - joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { res += map_.getTotalByteCountImpl(data->type); }); + joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { res += map_.getTotalByteCountImpl(data->type); }); } } return res; @@ -520,6 +523,8 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) return true; } + bool prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr; + size_t total_rows = 0; size_t total_bytes = 0; { @@ -592,7 +597,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) bool is_inserted = false; if (kind != JoinKind::Cross) { - joinDispatch(kind, strictness, data->maps[onexpr_idx], [&](auto kind_, auto strictness_, auto & map) + joinDispatch(kind, strictness, data->maps[onexpr_idx], prefer_use_maps_all, [&](auto kind_, auto strictness_, auto & map) { size_t size = HashJoinMethods>::insertFromBlockImpl( *this, @@ -608,10 +613,10 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) is_inserted); if (flag_per_row) - used_flags->reinit(stored_block); + used_flags->reinit, MapsAll>>(stored_block); else if (is_inserted) /// Number of buckets + 1 value from zero storage - used_flags->reinit(size + 1); + used_flags->reinit, MapsAll>>(size + 1); }); } @@ -869,7 +874,7 @@ ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block keys.insert(std::move(key)); } - static_assert(!MapGetter::flagged, + static_assert(!MapGetter::flagged, "joinGet are not protected from hash table changes between block processing"); std::vector maps_vector; @@ -910,16 +915,34 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) materializeBlockInplace(block); } + bool prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr; { std::vectormaps[0])> * > maps_vector; for (size_t i = 0; i < table_join->getClauses().size(); ++i) maps_vector.push_back(&data->maps[i]); - if (joinDispatch(kind, strictness, maps_vector, [&](auto kind_, auto strictness_, auto & maps_vector_) + if (joinDispatch(kind, strictness, maps_vector, prefer_use_maps_all, [&](auto kind_, auto strictness_, auto & maps_vector_) { - using MapType = typename MapGetter::Map; - Block remaining_block = HashJoinMethods::joinBlockImpl( - *this, block, sample_block_with_columns_to_add, maps_vector_); + Block remaining_block; + if constexpr (std::is_same_v, std::vector>) + { + remaining_block = HashJoinMethods::joinBlockImpl( + *this, block, sample_block_with_columns_to_add, maps_vector_); + } + else if constexpr (std::is_same_v, std::vector>) + { + remaining_block = HashJoinMethods::joinBlockImpl( + *this, block, sample_block_with_columns_to_add, maps_vector_); + } + else if constexpr (std::is_same_v, std::vector>) + { + remaining_block = HashJoinMethods::joinBlockImpl( + *this, block, sample_block_with_columns_to_add, maps_vector_); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown maps type"); + } if (remaining_block.rows()) not_processed = std::make_shared(ExtraBlock{std::move(remaining_block)}); else @@ -1019,7 +1042,8 @@ public: rows_added = fillColumnsFromMap(map, columns_right); }; - if (!joinDispatch(parent.kind, parent.strictness, parent.data->maps.front(), fill_callback)) + bool prefer_use_maps_all = parent.table_join->getMixedJoinExpression() != nullptr; + if (!joinDispatch(parent.kind, parent.strictness, parent.data->maps.front(), prefer_use_maps_all, fill_callback)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness '{}' (must be on of: ANY, ALL, ASOF)", parent.strictness); } @@ -1216,11 +1240,12 @@ void HashJoin::reuseJoinedData(const HashJoin & join) if (flag_per_row) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "StorageJoin with ORs is not supported"); + bool prefer_use_maps_all = join.table_join->getMixedJoinExpression() != nullptr; for (auto & map : data->maps) { - joinDispatch(kind, strictness, map, [this](auto kind_, auto strictness_, auto & map_) + joinDispatch(kind, strictness, map, prefer_use_maps_all, [this](auto kind_, auto strictness_, auto & map_) { - used_flags->reinit(map_.getBufferSizeInCells(data->type) + 1); + used_flags->reinit, MapsAll>>(map_.getBufferSizeInCells(data->type) + 1); }); } } @@ -1300,7 +1325,9 @@ void HashJoin::validateAdditionalFilterExpression(ExpressionActionsPtr additiona additional_filter_expression->dumpActions()); } - bool is_supported = (strictness == JoinStrictness::All) && (isInnerOrLeft(kind) || isRightOrFull(kind)); + bool is_supported = ((strictness == JoinStrictness::All) && (isInnerOrLeft(kind) || isRightOrFull(kind))) + || ((strictness == JoinStrictness::Semi || strictness == JoinStrictness::Any || strictness == JoinStrictness::Anti) + && (isLeft(kind) || isRight(kind))); if (!is_supported) { throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 0dfafa94efc..36785c08845 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -112,7 +112,7 @@ public: const MapsTemplateVector & maps_, bool is_join_get = false) { - constexpr JoinFeatures join_features; + constexpr JoinFeatures join_features; std::vector join_on_keys; const auto & onexprs = join.table_join->getClauses(); @@ -358,22 +358,20 @@ private: AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags) { - constexpr JoinFeatures join_features; - if constexpr (join_features.is_all_join) + constexpr JoinFeatures join_features; + if constexpr (join_features.is_maps_all) { - if (added_columns.additional_filter_expression) - { - bool mark_per_row_used = join_features.right || join_features.full || mapv.size() > 1; - return joinRightColumnsWithAddtitionalFilter( - std::forward>(key_getter_vector), - mapv, - added_columns, - used_flags, - need_filter, - join_features.need_flags, - join_features.add_missing, - mark_per_row_used); - } + if (added_columns.additional_filter_expression) + { + bool mark_per_row_used = join_features.right || join_features.full || mapv.size() > 1; + return joinRightColumnsWithAddtitionalFilter( + std::forward>(key_getter_vector), + mapv, + added_columns, + used_flags, + need_filter, + mark_per_row_used); + } } if (added_columns.additional_filter_expression) @@ -394,7 +392,7 @@ private: AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags) { - constexpr JoinFeatures join_features; + constexpr JoinFeatures join_features; size_t rows = added_columns.rows_to_add; if constexpr (need_filter) @@ -474,7 +472,7 @@ private: mapped, added_columns, current_offset, known_rows, used_flags_opt); } } - else if constexpr (join_features.is_any_join && KIND == JoinKind::Inner) + else if constexpr (join_features.is_any_join && join_features.inner) { bool used_once = used_flags.template setUsedOnce(find_result); @@ -655,24 +653,23 @@ private: } /// First to collect all matched rows refs by join keys, then filter out rows which are not true in additional filter expression. - template + template static size_t joinRightColumnsWithAddtitionalFilter( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]], bool need_filter [[maybe_unused]], - bool need_flags [[maybe_unused]], - bool add_missing [[maybe_unused]], bool flag_per_row [[maybe_unused]]) { + constexpr JoinFeatures join_features; size_t left_block_rows = added_columns.rows_to_add; if (need_filter) added_columns.filter = IColumn::Filter(left_block_rows, 0); std::unique_ptr pool; - if constexpr (need_replication) + if constexpr (join_features.need_replication) added_columns.offsets_to_replicate = std::make_unique(left_block_rows); std::vector row_replicate_offset; @@ -699,7 +696,7 @@ private: selected_rows.clear(); for (; left_row_iter < left_block_rows; ++left_row_iter) { - if constexpr (need_replication) + if constexpr (join_features.need_replication) { if (unlikely(total_added_rows + current_added_rows >= max_joined_block_rows)) { @@ -743,7 +740,7 @@ private: for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i) { bool any_matched = false; - /// For all right join, flag_per_row is true, we need mark used flags for each row. + /// For right join, flag_per_row is true, we need mark used flags for each row. if (flag_per_row) { for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) @@ -751,10 +748,26 @@ private: if (filter_flags[replicated_row]) { any_matched = true; - added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, add_missing); - total_added_rows += 1; - if (need_flags) - used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); + if constexpr (join_features.is_semi_join || join_features.is_any_join) + { + auto used_once = used_flags.template setUsedOnce(selected_right_row_it->block, selected_right_row_it->row_num, 0); + if (used_once) + { + total_added_rows += 1; + added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + } + } + else if constexpr (join_features.is_anti_join) + { + if constexpr (join_features.right && join_features.need_flags) + used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); + } + else + { + total_added_rows += 1; + added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); + } } ++selected_right_row_it; } @@ -763,34 +776,66 @@ private: { for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) { - if (filter_flags[replicated_row]) + if constexpr (join_features.is_anti_join) { - any_matched = true; - added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, add_missing); - total_added_rows += 1; + any_matched |= filter_flags[replicated_row]; + } + else if constexpr (join_features.need_replication) + { + if (filter_flags[replicated_row]) + { + any_matched = true; + added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + total_added_rows += 1; + } + ++selected_right_row_it; + } + else + { + if (filter_flags[replicated_row]) + { + any_matched = true; + added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + total_added_rows += 1; + selected_right_row_it = selected_right_row_it + row_replicate_offset[i] - replicated_row; + break; + } + else + ++selected_right_row_it; } - ++selected_right_row_it; } } - if (!any_matched) + + + if constexpr (join_features.is_anti_join) { - if (add_missing) - addNotFoundRow(added_columns, total_added_rows); - else - addNotFoundRow(added_columns, total_added_rows); + if (!any_matched) + { + if constexpr (join_features.left) + if (need_filter) + setUsed(added_columns.filter, left_start_row + i - 1); + addNotFoundRow(added_columns, total_added_rows); + } } else { - if (!flag_per_row && need_flags) - used_flags.template setUsed(find_results[find_result_index]); - if (need_filter) - setUsed(added_columns.filter, left_start_row + i - 1); - if (add_missing) - added_columns.applyLazyDefaults(); + if (!any_matched) + { + addNotFoundRow(added_columns, total_added_rows); + } + else + { + if (!flag_per_row) + used_flags.template setUsed(find_results[find_result_index]); + if (need_filter) + setUsed(added_columns.filter, left_start_row + i - 1); + if constexpr (join_features.add_missing) + added_columns.applyLazyDefaults(); + } } find_result_index += (prev_replicated_row != row_replicate_offset[i]); - if constexpr (need_replication) + if constexpr (join_features.need_replication) { (*added_columns.offsets_to_replicate)[left_start_row + i - 1] = total_added_rows; } @@ -817,7 +862,7 @@ private: auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns); copy_final_matched_rows(left_start_row, filter_col); - if constexpr (need_replication) + if constexpr (join_features.need_replication) { // Add a check for current_added_rows to avoid run the filter expression on too small size batch. if (total_added_rows >= max_joined_block_rows || current_added_rows < 1024) @@ -825,7 +870,7 @@ private: } } - if constexpr (need_replication) + if constexpr (join_features.need_replication) { added_columns.offsets_to_replicate->resize_assume_reserved(left_row_iter); added_columns.filter.resize_assume_reserved(left_row_iter); diff --git a/src/Interpreters/HashJoin/JoinFeatures.h b/src/Interpreters/HashJoin/JoinFeatures.h index 2f2bd1e29a2..a530179f0b4 100644 --- a/src/Interpreters/HashJoin/JoinFeatures.h +++ b/src/Interpreters/HashJoin/JoinFeatures.h @@ -3,15 +3,15 @@ #include namespace DB { -template +template struct JoinFeatures { static constexpr bool is_any_join = STRICTNESS == JoinStrictness::Any; - static constexpr bool is_any_or_semi_join = STRICTNESS == JoinStrictness::Any || STRICTNESS == JoinStrictness::RightAny || (STRICTNESS == JoinStrictness::Semi && KIND == JoinKind::Left); static constexpr bool is_all_join = STRICTNESS == JoinStrictness::All; static constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof; static constexpr bool is_semi_join = STRICTNESS == JoinStrictness::Semi; static constexpr bool is_anti_join = STRICTNESS == JoinStrictness::Anti; + static constexpr bool is_any_or_semi_join = is_any_join || STRICTNESS == JoinStrictness::RightAny || (is_semi_join && KIND == JoinKind::Left); static constexpr bool left = KIND == JoinKind::Left; static constexpr bool right = KIND == JoinKind::Right; @@ -22,7 +22,8 @@ struct JoinFeatures static constexpr bool need_filter = !need_replication && (inner || right || (is_semi_join && left) || (is_anti_join && left)); static constexpr bool add_missing = (left || full) && !is_semi_join; - static constexpr bool need_flags = MapGetter::flagged; + static constexpr bool need_flags = MapGetter, HashJoin::MapsOne>>::flagged; + static constexpr bool is_maps_all = std::is_same_v, HashJoin::MapsAll>; }; } diff --git a/src/Interpreters/HashJoin/JoinUsedFlags.h b/src/Interpreters/HashJoin/JoinUsedFlags.h index bd41ba2073f..c84c6ec3fea 100644 --- a/src/Interpreters/HashJoin/JoinUsedFlags.h +++ b/src/Interpreters/HashJoin/JoinUsedFlags.h @@ -26,10 +26,10 @@ public: /// Update size for vector with flags. /// Calling this method invalidates existing flags. /// It can be called several times, but all of them should happen before using this structure. - template + template void reinit(size_t size) { - if constexpr (MapGetter::flagged) + if constexpr (MapGetter::flagged) { assert(flags[nullptr].size() <= size); need_flags = true; @@ -43,10 +43,10 @@ public: } } - template + template void reinit(const Block * block_ptr) { - if constexpr (MapGetter::flagged) + if constexpr (MapGetter::flagged) { assert(flags[block_ptr].size() <= block_ptr->rows()); need_flags = true; @@ -148,6 +148,31 @@ public: } } + template + bool setUsedOnce(const Block * block, size_t row_num, size_t offset) + { + if constexpr (!use_flags) + return true; + + if constexpr (flag_per_row) + { + /// fast check to prevent heavy CAS with seq_cst order + if (flags[block][row_num].load(std::memory_order_relaxed)) + return false; + + bool expected = false; + return flags[block][row_num].compare_exchange_strong(expected, true); + } + else + { + /// fast check to prevent heavy CAS with seq_cst order + if (flags[nullptr][offset].load(std::memory_order_relaxed)) + return false; + + bool expected = false; + return flags[nullptr][offset].compare_exchange_strong(expected, true); + } + } }; } diff --git a/src/Interpreters/HashJoin/LeftHashJoin.cpp b/src/Interpreters/HashJoin/LeftHashJoin.cpp index 69e17ff70bd..a53ffaac0b5 100644 --- a/src/Interpreters/HashJoin/LeftHashJoin.cpp +++ b/src/Interpreters/HashJoin/LeftHashJoin.cpp @@ -4,8 +4,11 @@ namespace DB { template class HashJoinMethods; template class HashJoinMethods; +template class HashJoinMethods; template class HashJoinMethods; template class HashJoinMethods; +template class HashJoinMethods; template class HashJoinMethods; +template class HashJoinMethods; template class HashJoinMethods; } diff --git a/src/Interpreters/joinDispatch.h b/src/Interpreters/joinDispatch.h index 54c5c7dc83a..982c56e8210 100644 --- a/src/Interpreters/joinDispatch.h +++ b/src/Interpreters/joinDispatch.h @@ -12,38 +12,41 @@ namespace DB { -template +template struct MapGetter; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; /// Only SEMI LEFT and SEMI RIGHT are valid. INNER and FULL are here for templates instantiation. -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; /// Only SEMI LEFT and SEMI RIGHT are valid. INNER and FULL are here for templates instantiation. -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template -struct MapGetter { using Map = HashJoin::MapsAsof; static constexpr bool flagged = false; }; +template +struct MapGetter { using Map = HashJoin::MapsAsof; static constexpr bool flagged = false; }; static constexpr std::array STRICTNESSES = { JoinStrictness::RightAny, @@ -62,7 +65,7 @@ static constexpr std::array KINDS = { }; /// Init specified join map -inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin::MapsVariant & maps) +inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin::MapsVariant & maps, bool prefer_use_maps_all = false) { return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij) { @@ -70,7 +73,10 @@ inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin: constexpr auto j = ij % STRICTNESSES.size(); if (kind == KINDS[i] && strictness == STRICTNESSES[j]) { - maps = typename MapGetter::Map(); + if (prefer_use_maps_all) + maps = typename MapGetter::Map(); + else + maps = typename MapGetter::Map(); return true; } return false; @@ -79,7 +85,7 @@ inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin: /// Call function on specified join map template -inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant & maps, Func && func) +inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant & maps, bool prefer_use_maps_all, Func && func) { return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij) { @@ -89,10 +95,16 @@ inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant & constexpr auto j = ij % STRICTNESSES.size(); if (kind == KINDS[i] && strictness == STRICTNESSES[j]) { - func( - std::integral_constant(), - std::integral_constant(), - std::get::Map>(maps)); + if (prefer_use_maps_all) + func( + std::integral_constant(), + std::integral_constant(), + std::get::Map>(maps)); + else + func( + std::integral_constant(), + std::integral_constant(), + std::get::Map>(maps)); return true; } return false; @@ -101,7 +113,7 @@ inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant & /// Call function on specified join map template -inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, std::vector & mapsv, Func && func) +inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, std::vector & mapsv, bool prefer_use_maps_all, Func && func) { return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij) { @@ -111,17 +123,31 @@ inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, std::vector::Map; - std::vector v; - v.reserve(mapsv.size()); - for (const auto & el : mapsv) - v.push_back(&std::get(*el)); + if (prefer_use_maps_all) + { + using MapType = typename MapGetter::Map; + std::vector v; + v.reserve(mapsv.size()); + for (const auto & el : mapsv) + v.push_back(&std::get(*el)); - func( - std::integral_constant(), - std::integral_constant(), - v - /*std::get::Map>(maps)*/); + func( + std::integral_constant(), std::integral_constant(), v + /*std::get::Map>(maps)*/); + } + else + { + using MapType = typename MapGetter::Map; + std::vector v; + v.reserve(mapsv.size()); + for (const auto & el : mapsv) + v.push_back(&std::get(*el)); + + func( + std::integral_constant(), std::integral_constant(), v + /*std::get::Map>(maps)*/); + + } return true; } return false; diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index a0d6cf11b64..974df940e91 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -504,7 +504,11 @@ protected: return {}; Chunk chunk; - if (!joinDispatch(join->kind, join->strictness, join->data->maps.front(), + if (!joinDispatch( + join->kind, + join->strictness, + join->data->maps.front(), + join->table_join->getMixedJoinExpression() != nullptr, [&](auto kind, auto strictness, auto & map) { chunk = createChunk(map); })) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness"); return chunk; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference index 46f24f73356..3e413afd98e 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference @@ -382,6 +382,220 @@ key1 e 5 5 5 key1 C 3 4 5 key2 a2 1 1 1 0 0 \N key4 f 2 3 4 key4 F 1 1 1 SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 LEFT ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 B 2 1 2 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT t1.*, t2.* from t1 LEFT ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 0 0 \N +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 LEFT SEMI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 LEFT SEMI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 B 2 1 2 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 LEFT SEMI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT SEMI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 LEFT ANTI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 e 5 5 5 key1 0 0 \N +key2 a2 1 1 1 key2 0 0 \N +SELECT t1.*, t2.* from t1 LEFT ANTI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 e 5 5 5 key1 0 0 \N +key2 a2 1 1 1 key2 0 0 \N +key4 f 2 3 4 key4 0 0 \N +SELECT t1.*, t2.* from t1 LEFT ANTI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 d 4 7 2 key1 0 0 \N +key1 e 5 5 5 key1 0 0 \N +key2 a2 1 1 1 key2 0 0 \N +key4 f 2 3 4 key4 0 0 \N +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT ANTI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +SELECT t1.*, t2.* FROM t1 RIGHT ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 RIGHT ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key1 A 1 2 1 + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 RIGHT SEMI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT SEMI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 RIGHT SEMI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT SEMI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 RIGHT ANTI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key1 A 1 2 1 + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT ANTI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 LEFT ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 B 2 1 2 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT t1.*, t2.* from t1 LEFT ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 0 0 \N +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 LEFT SEMI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 LEFT SEMI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 B 2 1 2 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 LEFT SEMI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT SEMI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 LEFT ANTI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 e 5 5 5 key1 0 0 \N +key2 a2 1 1 1 key2 0 0 \N +SELECT t1.*, t2.* from t1 LEFT ANTI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 e 5 5 5 key1 0 0 \N +key2 a2 1 1 1 key2 0 0 \N +key4 f 2 3 4 key4 0 0 \N +SELECT t1.*, t2.* from t1 LEFT ANTI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 d 4 7 2 key1 0 0 \N +key1 e 5 5 5 key1 0 0 \N +key2 a2 1 1 1 key2 0 0 \N +key4 f 2 3 4 key4 0 0 \N +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT ANTI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +SELECT t1.*, t2.* FROM t1 RIGHT ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 RIGHT ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key1 A 1 2 1 + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 RIGHT SEMI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT SEMI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 RIGHT SEMI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT SEMI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 RIGHT ANTI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key1 A 1 2 1 + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT ANTI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +SET join_algorithm='hash'; SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr); key1 a 1 1 2 key1 A 1 2 1 key1 a 1 1 2 key1 B 2 1 2 diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 index 61ad5ec0bf1..1bf5a7870e7 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 @@ -22,6 +22,17 @@ SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 {% endfor -%} {% endfor -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +{% for join_type in ['LEFT', 'RIGHT'] -%} +{% for join_strictness in ['ANY', 'SEMI', 'ANTI'] -%} +SELECT t1.*, t2.* FROM t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +SELECT t1.*, t2.* from t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +SELECT t1.*, t2.* from t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 {{ join_type }} {{ join_strictness }} JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +{% endfor -%} +{% endfor -%} +{% endfor -%} {% for algorithm in ['hash'] -%} SET join_algorithm='{{ algorithm }}'; From 413834d04920072dd9a38b56902d799f71a476a3 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 4 Jul 2024 10:06:11 +0800 Subject: [PATCH 365/644] instance template classes --- src/Interpreters/HashJoin/HashJoinMethods.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 36785c08845..17c11b202ca 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -970,9 +970,12 @@ private: /// Instantiate template class ahead in different .cpp files to avoid `too large translation unit`. extern template class HashJoinMethods; extern template class HashJoinMethods; +extern template class HashJoinMethods; extern template class HashJoinMethods; extern template class HashJoinMethods; +extern template class HashJoinMethods; extern template class HashJoinMethods; +extern template class HashJoinMethods; extern template class HashJoinMethods; extern template class HashJoinMethods; From 377eed20fca56c74e4987fe4038fca6f0e019090 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 4 Jul 2024 15:05:24 +0800 Subject: [PATCH 366/644] reduce the size of HashJoin.cpp.o --- src/Interpreters/HashJoin/FullHashJoin.cpp | 2 +- src/Interpreters/HashJoin/HashJoinMethods.h | 834 +--------------- .../HashJoin/HashJoinMethodsImpl.h | 912 ++++++++++++++++++ src/Interpreters/HashJoin/InnerHashJoin.cpp | 2 +- src/Interpreters/HashJoin/LeftHashJoin.cpp | 2 +- src/Interpreters/HashJoin/RightHashJoin.cpp | 2 +- 6 files changed, 932 insertions(+), 822 deletions(-) create mode 100644 src/Interpreters/HashJoin/HashJoinMethodsImpl.h diff --git a/src/Interpreters/HashJoin/FullHashJoin.cpp b/src/Interpreters/HashJoin/FullHashJoin.cpp index 5d058d10fc2..4cdb2e757a4 100644 --- a/src/Interpreters/HashJoin/FullHashJoin.cpp +++ b/src/Interpreters/HashJoin/FullHashJoin.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 17c11b202ca..e3b8fbc1737 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -12,15 +12,8 @@ #include #include - namespace DB { -namespace ErrorCodes -{ - extern const int UNSUPPORTED_JOIN_KEYS; - extern const int LOGICAL_ERROR; -} - /// Inserting an element into a hash table of the form `key -> reference to a string`, which will then be used by JOIN. template struct Inserter @@ -64,7 +57,6 @@ struct Inserter } }; - /// MapsTemplate is one of MapsOne, MapsAll and MapsAsof template class HashJoinMethods @@ -81,27 +73,7 @@ public: ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, - bool & is_inserted) - { - switch (type) - { - case HashJoin::Type::EMPTY: - [[fallthrough]]; - case HashJoin::Type::CROSS: - /// Do nothing. We will only save block, and it is enough - is_inserted = true; - return 0; - - #define M(TYPE) \ - case HashJoin::Type::TYPE: \ - return insertFromBlockImplTypeCase>::Type>(\ - join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); \ - break; - - APPLY_FOR_JOIN_VARIANTS(M) - #undef M - } - } + bool & is_inserted); using MapsTemplateVector = std::vector; @@ -110,278 +82,36 @@ public: Block & block, const Block & block_with_columns_to_add, const MapsTemplateVector & maps_, - bool is_join_get = false) - { - constexpr JoinFeatures join_features; - - std::vector join_on_keys; - const auto & onexprs = join.table_join->getClauses(); - for (size_t i = 0; i < onexprs.size(); ++i) - { - const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right; - join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]); - } - size_t existing_columns = block.columns(); - - /** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized. - * Because if they are constants, then in the "not joined" rows, they may have different values - * - default values, which can differ from the values of these constants. - */ - if constexpr (join_features.right || join_features.full) - { - materializeBlockInplace(block); - } - - /** For LEFT/INNER JOIN, the saved blocks do not contain keys. - * For FULL/RIGHT JOIN, the saved blocks contain keys; - * but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped. - * For ASOF, the last column is used as the ASOF column - */ - AddedColumns added_columns( - block, - block_with_columns_to_add, - join.savedBlockSample(), - join, - std::move(join_on_keys), - join.table_join->getMixedJoinExpression(), - join_features.is_asof_join, - is_join_get); - - bool has_required_right_keys = (join.required_right_keys.columns() != 0); - added_columns.need_filter = join_features.need_filter || has_required_right_keys; - added_columns.max_joined_block_rows = join.max_joined_block_rows; - if (!added_columns.max_joined_block_rows) - added_columns.max_joined_block_rows = std::numeric_limits::max(); - else - added_columns.reserve(join_features.need_replication); - - size_t num_joined = switchJoinRightColumns(maps_, added_columns, join.data->type, *join.used_flags); - /// Do not hold memory for join_on_keys anymore - added_columns.join_on_keys.clear(); - Block remaining_block = sliceBlock(block, num_joined); - - added_columns.buildOutput(); - for (size_t i = 0; i < added_columns.size(); ++i) - block.insert(added_columns.moveColumn(i)); - - std::vector right_keys_to_replicate [[maybe_unused]]; - - if constexpr (join_features.need_filter) - { - /// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones. - for (size_t i = 0; i < existing_columns; ++i) - block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(added_columns.filter, -1); - - /// Add join key columns from right block if needed using value from left table because of equality - for (size_t i = 0; i < join.required_right_keys.columns(); ++i) - { - const auto & right_key = join.required_right_keys.getByPosition(i); - /// asof column is already in block. - if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back()) - continue; - - const auto & left_column = block.getByName(join.required_right_keys_sources[i]); - const auto & right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name); - auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column); - block.insert(std::move(right_col)); - } - } - else if (has_required_right_keys) - { - /// Add join key columns from right block if needed. - for (size_t i = 0; i < join.required_right_keys.columns(); ++i) - { - const auto & right_key = join.required_right_keys.getByPosition(i); - auto right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name); - /// asof column is already in block. - if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back()) - continue; - - const auto & left_column = block.getByName(join.required_right_keys_sources[i]); - auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column, &added_columns.filter); - block.insert(std::move(right_col)); - - if constexpr (join_features.need_replication) - right_keys_to_replicate.push_back(block.getPositionByName(right_col_name)); - } - } - - if constexpr (join_features.need_replication) - { - std::unique_ptr & offsets_to_replicate = added_columns.offsets_to_replicate; - - /// If ALL ... JOIN - we replicate all the columns except the new ones. - for (size_t i = 0; i < existing_columns; ++i) - { - block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicate(*offsets_to_replicate); - } - - /// Replicate additional right keys - for (size_t pos : right_keys_to_replicate) - { - block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate); - } - } - - return remaining_block; - } - + bool is_join_get = false); private: template - static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes) - { - if constexpr (is_asof_join) - { - auto key_column_copy = key_columns; - auto key_size_copy = key_sizes; - key_column_copy.pop_back(); - key_size_copy.pop_back(); - return KeyGetter(key_column_copy, key_size_copy, nullptr); - } - else - return KeyGetter(key_columns, key_sizes, nullptr); - } + static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes); template - static size_t NO_INLINE insertFromBlockImplTypeCase( + static size_t insertFromBlockImplTypeCase( HashJoin & join, HashMap & map, size_t rows, const ColumnRawPtrs & key_columns, - const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted) - { - [[maybe_unused]] constexpr bool mapped_one = std::is_same_v; - constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof; - - const IColumn * asof_column [[maybe_unused]] = nullptr; - if constexpr (is_asof_join) - asof_column = key_columns.back(); - - auto key_getter = createKeyGetter(key_columns, key_sizes); - - /// For ALL and ASOF join always insert values - is_inserted = !mapped_one || is_asof_join; - - for (size_t i = 0; i < rows; ++i) - { - if (null_map && (*null_map)[i]) - { - /// nulls are not inserted into hash table, - /// keep them for RIGHT and FULL joins - is_inserted = true; - continue; - } - - /// Check condition for right table from ON section - if (join_mask && !(*join_mask)[i]) - continue; - - if constexpr (is_asof_join) - Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); - else if constexpr (mapped_one) - is_inserted |= Inserter::insertOne(join, map, key_getter, stored_block, i, pool); - else - Inserter::insertAll(join, map, key_getter, stored_block, i, pool); - } - return map.getBufferSizeInCells(); - } + const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted); template static size_t switchJoinRightColumns( const std::vector & mapv, AddedColumns & added_columns, HashJoin::Type type, - JoinStuff::JoinUsedFlags & used_flags) - { - constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof; - switch (type) - { - case HashJoin::Type::EMPTY: { - if constexpr (!is_asof_join) - { - using KeyGetter = KeyGetterEmpty; - std::vector key_getter_vector; - key_getter_vector.emplace_back(); - - using MapTypeVal = typename KeyGetter::MappedType; - std::vector a_map_type_vector; - a_map_type_vector.emplace_back(); - return joinRightColumnsSwitchNullability( - std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags); - } - throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys. Type: {}", type); - } - #define M(TYPE) \ - case HashJoin::Type::TYPE: \ - { \ - using MapTypeVal = const typename std::remove_reference_t::element_type; \ - using KeyGetter = typename KeyGetterForType::Type; \ - std::vector a_map_type_vector(mapv.size()); \ - std::vector key_getter_vector; \ - for (size_t d = 0; d < added_columns.join_on_keys.size(); ++d) \ - { \ - const auto & join_on_key = added_columns.join_on_keys[d]; \ - a_map_type_vector[d] = mapv[d]->TYPE.get(); \ - key_getter_vector.push_back(std::move(createKeyGetter(join_on_key.key_columns, join_on_key.key_sizes))); \ - } \ - return joinRightColumnsSwitchNullability( \ - std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags); \ - } - APPLY_FOR_JOIN_VARIANTS(M) - #undef M - - default: - throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", type); - } - } + JoinStuff::JoinUsedFlags & used_flags); template static size_t joinRightColumnsSwitchNullability( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, - JoinStuff::JoinUsedFlags & used_flags) - { - if (added_columns.need_filter) - { - return joinRightColumnsSwitchMultipleDisjuncts( - std::forward>(key_getter_vector), mapv, added_columns, used_flags); - } - else - { - return joinRightColumnsSwitchMultipleDisjuncts( - std::forward>(key_getter_vector), mapv, added_columns, used_flags); - } - } + JoinStuff::JoinUsedFlags & used_flags); template static size_t joinRightColumnsSwitchMultipleDisjuncts( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, - JoinStuff::JoinUsedFlags & used_flags) - { - constexpr JoinFeatures join_features; - if constexpr (join_features.is_maps_all) - { - if (added_columns.additional_filter_expression) - { - bool mark_per_row_used = join_features.right || join_features.full || mapv.size() > 1; - return joinRightColumnsWithAddtitionalFilter( - std::forward>(key_getter_vector), - mapv, - added_columns, - used_flags, - need_filter, - mark_per_row_used); - } - } - - if (added_columns.additional_filter_expression) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Additional filter expression is not supported for this JOIN"); - - return mapv.size() > 1 ? joinRightColumns( - std::forward>(key_getter_vector), mapv, added_columns, used_flags) - : joinRightColumns( - std::forward>(key_getter_vector), mapv, added_columns, used_flags); - } + JoinStuff::JoinUsedFlags & used_flags); /// Joins right table columns which indexes are present in right_indexes using specified map. /// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). @@ -390,267 +120,17 @@ private: std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, - JoinStuff::JoinUsedFlags & used_flags) - { - constexpr JoinFeatures join_features; - - size_t rows = added_columns.rows_to_add; - if constexpr (need_filter) - added_columns.filter = IColumn::Filter(rows, 0); - - Arena pool; - - if constexpr (join_features.need_replication) - added_columns.offsets_to_replicate = std::make_unique(rows); - - IColumn::Offset current_offset = 0; - size_t max_joined_block_rows = added_columns.max_joined_block_rows; - size_t i = 0; - for (; i < rows; ++i) - { - if constexpr (join_features.need_replication) - { - if (unlikely(current_offset >= max_joined_block_rows)) - { - added_columns.offsets_to_replicate->resize_assume_reserved(i); - added_columns.filter.resize_assume_reserved(i); - break; - } - } - - bool right_row_found = false; - - KnownRowsHolder known_rows; - for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx) - { - const auto & join_keys = added_columns.join_on_keys[onexpr_idx]; - if (join_keys.null_map && (*join_keys.null_map)[i]) - continue; - - bool row_acceptable = !join_keys.isRowFiltered(i); - using FindResult = typename KeyGetter::FindResult; - auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult(); - - if (find_result.isFound()) - { - right_row_found = true; - auto & mapped = find_result.getMapped(); - if constexpr (join_features.is_asof_join) - { - const IColumn & left_asof_key = added_columns.leftAsofKey(); - - auto row_ref = mapped->findAsof(left_asof_key, i); - if (row_ref.block) - { - setUsed(added_columns.filter, i); - if constexpr (flag_per_row) - used_flags.template setUsed(row_ref.block, row_ref.row_num, 0); - else - used_flags.template setUsed(find_result); - - added_columns.appendFromBlock(*row_ref.block, row_ref.row_num, join_features.add_missing); - } - else - addNotFoundRow(added_columns, current_offset); - } - else if constexpr (join_features.is_all_join) - { - setUsed(added_columns.filter, i); - used_flags.template setUsed(find_result); - auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; - addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); - } - else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right) - { - /// Use first appeared left key + it needs left columns replication - bool used_once = used_flags.template setUsedOnce(find_result); - if (used_once) - { - auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; - setUsed(added_columns.filter, i); - addFoundRowAll( - mapped, added_columns, current_offset, known_rows, used_flags_opt); - } - } - else if constexpr (join_features.is_any_join && join_features.inner) - { - bool used_once = used_flags.template setUsedOnce(find_result); - - /// Use first appeared left key only - if (used_once) - { - setUsed(added_columns.filter, i); - added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing); - } - - break; - } - else if constexpr (join_features.is_any_join && join_features.full) - { - /// TODO - } - else if constexpr (join_features.is_anti_join) - { - if constexpr (join_features.right && join_features.need_flags) - used_flags.template setUsed(find_result); - } - else /// ANY LEFT, SEMI LEFT, old ANY (RightAny) - { - setUsed(added_columns.filter, i); - used_flags.template setUsed(find_result); - added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing); - - if (join_features.is_any_or_semi_join) - { - break; - } - } - } - } - - if (!right_row_found) - { - if constexpr (join_features.is_anti_join && join_features.left) - setUsed(added_columns.filter, i); - addNotFoundRow(added_columns, current_offset); - } - - if constexpr (join_features.need_replication) - { - (*added_columns.offsets_to_replicate)[i] = current_offset; - } - } - - added_columns.applyLazyDefaults(); - return i; - } + JoinStuff::JoinUsedFlags & used_flags); template - static void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]]) - { - if constexpr (need_filter) - filter[pos] = 1; - } + static void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]]); template static ColumnPtr buildAdditionalFilter( size_t left_start_row, const std::vector & selected_rows, const std::vector & row_replicate_offset, - AddedColumns & added_columns) - { - ColumnPtr result_column; - do - { - if (selected_rows.empty()) - { - result_column = ColumnUInt8::create(); - break; - } - const Block & sample_right_block = *selected_rows.begin()->block; - if (!sample_right_block || !added_columns.additional_filter_expression) - { - auto filter = ColumnUInt8::create(); - filter->insertMany(1, selected_rows.size()); - result_column = std::move(filter); - break; - } - - auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes(); - if (required_cols.empty()) - { - Block block; - added_columns.additional_filter_expression->execute(block); - result_column = block.getByPosition(0).column->cloneResized(selected_rows.size()); - break; - } - NameSet required_column_names; - for (auto & col : required_cols) - required_column_names.insert(col.name); - - Block executed_block; - size_t right_col_pos = 0; - for (const auto & col : sample_right_block.getColumnsWithTypeAndName()) - { - if (required_column_names.contains(col.name)) - { - auto new_col = col.column->cloneEmpty(); - for (const auto & selected_row : selected_rows) - { - const auto & src_col = selected_row.block->getByPosition(right_col_pos); - new_col->insertFrom(*src_col.column, selected_row.row_num); - } - executed_block.insert({std::move(new_col), col.type, col.name}); - } - right_col_pos += 1; - } - if (!executed_block) - { - result_column = ColumnUInt8::create(); - break; - } - - for (const auto & col_name : required_column_names) - { - const auto * src_col = added_columns.left_block.findByName(col_name); - if (!src_col) - continue; - auto new_col = src_col->column->cloneEmpty(); - size_t prev_left_offset = 0; - for (size_t i = 1; i < row_replicate_offset.size(); ++i) - { - const size_t & left_offset = row_replicate_offset[i]; - size_t rows = left_offset - prev_left_offset; - if (rows) - new_col->insertManyFrom(*src_col->column, left_start_row + i - 1, rows); - prev_left_offset = left_offset; - } - executed_block.insert({std::move(new_col), src_col->type, col_name}); - } - if (!executed_block) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "required columns: [{}], but not found any in left/right table. right table: {}, left table: {}", - required_cols.toString(), - sample_right_block.dumpNames(), - added_columns.left_block.dumpNames()); - } - - for (const auto & col : executed_block.getColumnsWithTypeAndName()) - if (!col.column || !col.type) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal nullptr column in input block: {}", executed_block.dumpStructure()); - - added_columns.additional_filter_expression->execute(executed_block); - result_column = executed_block.getByPosition(0).column->convertToFullColumnIfConst(); - executed_block.clear(); - } while (false); - - result_column = result_column->convertToFullIfNeeded(); - if (result_column->isNullable()) - { - /// Convert Nullable(UInt8) to UInt8 ensuring that nulls are zeros - /// Trying to avoid copying data, since we are the only owner of the column. - ColumnPtr mask_column = assert_cast(*result_column).getNullMapColumnPtr(); - - MutableColumnPtr mutable_column; - { - ColumnPtr nested_column = assert_cast(*result_column).getNestedColumnPtr(); - result_column.reset(); - mutable_column = IColumn::mutate(std::move(nested_column)); - } - - auto & column_data = assert_cast(*mutable_column).getData(); - const auto & mask_column_data = assert_cast(*mask_column).getData(); - for (size_t i = 0; i < column_data.size(); ++i) - { - if (mask_column_data[i]) - column_data[i] = 0; - } - return mutable_column; - } - return result_column; - } + AddedColumns & added_columns); /// First to collect all matched rows refs by join keys, then filter out rows which are not true in additional filter expression. template @@ -660,241 +140,10 @@ private: AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]], bool need_filter [[maybe_unused]], - bool flag_per_row [[maybe_unused]]) - { - constexpr JoinFeatures join_features; - size_t left_block_rows = added_columns.rows_to_add; - if (need_filter) - added_columns.filter = IColumn::Filter(left_block_rows, 0); - - std::unique_ptr pool; - - if constexpr (join_features.need_replication) - added_columns.offsets_to_replicate = std::make_unique(left_block_rows); - - std::vector row_replicate_offset; - row_replicate_offset.reserve(left_block_rows); - - using FindResult = typename KeyGetter::FindResult; - size_t max_joined_block_rows = added_columns.max_joined_block_rows; - size_t left_row_iter = 0; - PreSelectedRows selected_rows; - selected_rows.reserve(left_block_rows); - std::vector find_results; - find_results.reserve(left_block_rows); - bool exceeded_max_block_rows = false; - IColumn::Offset total_added_rows = 0; - IColumn::Offset current_added_rows = 0; - - auto collect_keys_matched_rows_refs = [&]() - { - pool = std::make_unique(); - find_results.clear(); - row_replicate_offset.clear(); - row_replicate_offset.push_back(0); - current_added_rows = 0; - selected_rows.clear(); - for (; left_row_iter < left_block_rows; ++left_row_iter) - { - if constexpr (join_features.need_replication) - { - if (unlikely(total_added_rows + current_added_rows >= max_joined_block_rows)) - { - break; - } - } - KnownRowsHolder all_flag_known_rows; - KnownRowsHolder single_flag_know_rows; - for (size_t join_clause_idx = 0; join_clause_idx < added_columns.join_on_keys.size(); ++join_clause_idx) - { - const auto & join_keys = added_columns.join_on_keys[join_clause_idx]; - if (join_keys.null_map && (*join_keys.null_map)[left_row_iter]) - continue; - - bool row_acceptable = !join_keys.isRowFiltered(left_row_iter); - auto find_result = row_acceptable - ? key_getter_vector[join_clause_idx].findKey(*(mapv[join_clause_idx]), left_row_iter, *pool) - : FindResult(); - - if (find_result.isFound()) - { - auto & mapped = find_result.getMapped(); - find_results.push_back(find_result); - if (flag_per_row) - addFoundRowAll(mapped, selected_rows, current_added_rows, all_flag_known_rows, nullptr); - else - addFoundRowAll(mapped, selected_rows, current_added_rows, single_flag_know_rows, nullptr); - } - } - row_replicate_offset.push_back(current_added_rows); - } - }; - - auto copy_final_matched_rows = [&](size_t left_start_row, ColumnPtr filter_col) - { - const PaddedPODArray & filter_flags = assert_cast(*filter_col).getData(); - - size_t prev_replicated_row = 0; - auto selected_right_row_it = selected_rows.begin(); - size_t find_result_index = 0; - for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i) - { - bool any_matched = false; - /// For right join, flag_per_row is true, we need mark used flags for each row. - if (flag_per_row) - { - for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) - { - if (filter_flags[replicated_row]) - { - any_matched = true; - if constexpr (join_features.is_semi_join || join_features.is_any_join) - { - auto used_once = used_flags.template setUsedOnce(selected_right_row_it->block, selected_right_row_it->row_num, 0); - if (used_once) - { - total_added_rows += 1; - added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); - } - } - else if constexpr (join_features.is_anti_join) - { - if constexpr (join_features.right && join_features.need_flags) - used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); - } - else - { - total_added_rows += 1; - added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); - used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); - } - } - ++selected_right_row_it; - } - } - else - { - for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) - { - if constexpr (join_features.is_anti_join) - { - any_matched |= filter_flags[replicated_row]; - } - else if constexpr (join_features.need_replication) - { - if (filter_flags[replicated_row]) - { - any_matched = true; - added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); - total_added_rows += 1; - } - ++selected_right_row_it; - } - else - { - if (filter_flags[replicated_row]) - { - any_matched = true; - added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); - total_added_rows += 1; - selected_right_row_it = selected_right_row_it + row_replicate_offset[i] - replicated_row; - break; - } - else - ++selected_right_row_it; - } - } - } - - - if constexpr (join_features.is_anti_join) - { - if (!any_matched) - { - if constexpr (join_features.left) - if (need_filter) - setUsed(added_columns.filter, left_start_row + i - 1); - addNotFoundRow(added_columns, total_added_rows); - } - } - else - { - if (!any_matched) - { - addNotFoundRow(added_columns, total_added_rows); - } - else - { - if (!flag_per_row) - used_flags.template setUsed(find_results[find_result_index]); - if (need_filter) - setUsed(added_columns.filter, left_start_row + i - 1); - if constexpr (join_features.add_missing) - added_columns.applyLazyDefaults(); - } - } - find_result_index += (prev_replicated_row != row_replicate_offset[i]); - - if constexpr (join_features.need_replication) - { - (*added_columns.offsets_to_replicate)[left_start_row + i - 1] = total_added_rows; - } - prev_replicated_row = row_replicate_offset[i]; - } - }; - - while (left_row_iter < left_block_rows && !exceeded_max_block_rows) - { - auto left_start_row = left_row_iter; - collect_keys_matched_rows_refs(); - if (selected_rows.size() != current_added_rows || row_replicate_offset.size() != left_row_iter - left_start_row + 1) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Sizes are mismatched. selected_rows.size:{}, current_added_rows:{}, row_replicate_offset.size:{}, left_row_iter: {}, " - "left_start_row: {}", - selected_rows.size(), - current_added_rows, - row_replicate_offset.size(), - left_row_iter, - left_start_row); - } - auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns); - copy_final_matched_rows(left_start_row, filter_col); - - if constexpr (join_features.need_replication) - { - // Add a check for current_added_rows to avoid run the filter expression on too small size batch. - if (total_added_rows >= max_joined_block_rows || current_added_rows < 1024) - exceeded_max_block_rows = true; - } - } - - if constexpr (join_features.need_replication) - { - added_columns.offsets_to_replicate->resize_assume_reserved(left_row_iter); - added_columns.filter.resize_assume_reserved(left_row_iter); - } - added_columns.applyLazyDefaults(); - return left_row_iter; - } + bool flag_per_row [[maybe_unused]]); /// Cut first num_rows rows from block in place and returns block with remaining rows - static Block sliceBlock(Block & block, size_t num_rows) - { - size_t total_rows = block.rows(); - if (num_rows >= total_rows) - return {}; - size_t remaining_rows = total_rows - num_rows; - Block remaining_block = block.cloneEmpty(); - for (size_t i = 0; i < block.columns(); ++i) - { - auto & col = block.getByPosition(i); - remaining_block.getByPosition(i).column = col.column->cut(num_rows, remaining_rows); - col.column = col.column->cut(0, num_rows); - } - return remaining_block; - } + static Block sliceBlock(Block & block, size_t num_rows); /** Since we do not store right key columns, * this function is used to copy left key columns to right key columns. @@ -909,62 +158,11 @@ private: const DataTypePtr & right_key_type, const String & renamed_right_column, const ColumnWithTypeAndName & left_column, - const IColumn::Filter * null_map_filter = nullptr) - { - ColumnWithTypeAndName right_column = left_column; - right_column.name = renamed_right_column; + const IColumn::Filter * null_map_filter = nullptr); - if (null_map_filter) - right_column.column = JoinCommon::filterWithBlanks(right_column.column, *null_map_filter); + static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable); - bool should_be_nullable = isNullableOrLowCardinalityNullable(right_key_type); - if (null_map_filter) - correctNullabilityInplace(right_column, should_be_nullable, *null_map_filter); - else - correctNullabilityInplace(right_column, should_be_nullable); - - if (!right_column.type->equals(*right_key_type)) - { - right_column.column = castColumnAccurate(right_column, right_key_type); - right_column.type = right_key_type; - } - - right_column.column = right_column.column->convertToFullColumnIfConst(); - return right_column; - } - - static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable) - { - if (nullable) - { - JoinCommon::convertColumnToNullable(column); - } - else - { - /// We have to replace values masked by NULLs with defaults. - if (column.column) - if (const auto * nullable_column = checkAndGetColumn(&*column.column)) - column.column = JoinCommon::filterWithBlanks(column.column, nullable_column->getNullMapColumn().getData(), true); - - JoinCommon::removeColumnNullability(column); - } - } - - static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map) - { - if (nullable) - { - JoinCommon::convertColumnToNullable(column); - if (column.type->isNullable() && !negative_null_map.empty()) - { - MutableColumnPtr mutable_column = IColumn::mutate(std::move(column.column)); - assert_cast(*mutable_column).applyNegatedNullMap(negative_null_map); - column.column = std::move(mutable_column); - } - } - else - JoinCommon::removeColumnNullability(column); - } + static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map); }; /// Instantiate template class ahead in different .cpp files to avoid `too large translation unit`. diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h new file mode 100644 index 00000000000..2bf5f6aef4a --- /dev/null +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -0,0 +1,912 @@ +#pragma once +#include +namespace DB +{ +namespace ErrorCodes +{ + extern const int UNSUPPORTED_JOIN_KEYS; + extern const int LOGICAL_ERROR; +} +template +size_t HashJoinMethods::insertFromBlockImpl( + HashJoin & join, + HashJoin::Type type, + MapsTemplate & maps, + size_t rows, + const ColumnRawPtrs & key_columns, + const Sizes & key_sizes, + Block * stored_block, + ConstNullMapPtr null_map, + UInt8ColumnDataPtr join_mask, + Arena & pool, + bool & is_inserted) +{ + switch (type) + { + case HashJoin::Type::EMPTY: + [[fallthrough]]; + case HashJoin::Type::CROSS: + /// Do nothing. We will only save block, and it is enough + is_inserted = true; + return 0; + +#define M(TYPE) \ + case HashJoin::Type::TYPE: \ + return insertFromBlockImplTypeCase< \ + typename KeyGetterForType>::Type>( \ + join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); \ + break; + + APPLY_FOR_JOIN_VARIANTS(M) +#undef M + } +} + +template +Block HashJoinMethods::joinBlockImpl( + const HashJoin & join, Block & block, const Block & block_with_columns_to_add, const MapsTemplateVector & maps_, bool is_join_get) +{ + constexpr JoinFeatures join_features; + + std::vector join_on_keys; + const auto & onexprs = join.table_join->getClauses(); + for (size_t i = 0; i < onexprs.size(); ++i) + { + const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right; + join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]); + } + size_t existing_columns = block.columns(); + + /** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized. + * Because if they are constants, then in the "not joined" rows, they may have different values + * - default values, which can differ from the values of these constants. + */ + if constexpr (join_features.right || join_features.full) + { + materializeBlockInplace(block); + } + + /** For LEFT/INNER JOIN, the saved blocks do not contain keys. + * For FULL/RIGHT JOIN, the saved blocks contain keys; + * but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped. + * For ASOF, the last column is used as the ASOF column + */ + AddedColumns added_columns( + block, + block_with_columns_to_add, + join.savedBlockSample(), + join, + std::move(join_on_keys), + join.table_join->getMixedJoinExpression(), + join_features.is_asof_join, + is_join_get); + + bool has_required_right_keys = (join.required_right_keys.columns() != 0); + added_columns.need_filter = join_features.need_filter || has_required_right_keys; + added_columns.max_joined_block_rows = join.max_joined_block_rows; + if (!added_columns.max_joined_block_rows) + added_columns.max_joined_block_rows = std::numeric_limits::max(); + else + added_columns.reserve(join_features.need_replication); + + size_t num_joined = switchJoinRightColumns(maps_, added_columns, join.data->type, *join.used_flags); + /// Do not hold memory for join_on_keys anymore + added_columns.join_on_keys.clear(); + Block remaining_block = sliceBlock(block, num_joined); + + added_columns.buildOutput(); + for (size_t i = 0; i < added_columns.size(); ++i) + block.insert(added_columns.moveColumn(i)); + + std::vector right_keys_to_replicate [[maybe_unused]]; + + if constexpr (join_features.need_filter) + { + /// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones. + for (size_t i = 0; i < existing_columns; ++i) + block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(added_columns.filter, -1); + + /// Add join key columns from right block if needed using value from left table because of equality + for (size_t i = 0; i < join.required_right_keys.columns(); ++i) + { + const auto & right_key = join.required_right_keys.getByPosition(i); + /// asof column is already in block. + if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back()) + continue; + + const auto & left_column = block.getByName(join.required_right_keys_sources[i]); + const auto & right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name); + auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column); + block.insert(std::move(right_col)); + } + } + else if (has_required_right_keys) + { + /// Add join key columns from right block if needed. + for (size_t i = 0; i < join.required_right_keys.columns(); ++i) + { + const auto & right_key = join.required_right_keys.getByPosition(i); + auto right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name); + /// asof column is already in block. + if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back()) + continue; + + const auto & left_column = block.getByName(join.required_right_keys_sources[i]); + auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column, &added_columns.filter); + block.insert(std::move(right_col)); + + if constexpr (join_features.need_replication) + right_keys_to_replicate.push_back(block.getPositionByName(right_col_name)); + } + } + + if constexpr (join_features.need_replication) + { + std::unique_ptr & offsets_to_replicate = added_columns.offsets_to_replicate; + + /// If ALL ... JOIN - we replicate all the columns except the new ones. + for (size_t i = 0; i < existing_columns; ++i) + { + block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicate(*offsets_to_replicate); + } + + /// Replicate additional right keys + for (size_t pos : right_keys_to_replicate) + { + block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate); + } + } + + return remaining_block; +} + +template +template +KeyGetter HashJoinMethods::createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes) +{ + if constexpr (is_asof_join) + { + auto key_column_copy = key_columns; + auto key_size_copy = key_sizes; + key_column_copy.pop_back(); + key_size_copy.pop_back(); + return KeyGetter(key_column_copy, key_size_copy, nullptr); + } + else + return KeyGetter(key_columns, key_sizes, nullptr); +} + +template +template +size_t HashJoinMethods::insertFromBlockImplTypeCase( + HashJoin & join, + HashMap & map, + size_t rows, + const ColumnRawPtrs & key_columns, + const Sizes & key_sizes, + Block * stored_block, + ConstNullMapPtr null_map, + UInt8ColumnDataPtr join_mask, + Arena & pool, + bool & is_inserted) +{ + [[maybe_unused]] constexpr bool mapped_one = std::is_same_v; + constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof; + + const IColumn * asof_column [[maybe_unused]] = nullptr; + if constexpr (is_asof_join) + asof_column = key_columns.back(); + + auto key_getter = createKeyGetter(key_columns, key_sizes); + + /// For ALL and ASOF join always insert values + is_inserted = !mapped_one || is_asof_join; + + for (size_t i = 0; i < rows; ++i) + { + if (null_map && (*null_map)[i]) + { + /// nulls are not inserted into hash table, + /// keep them for RIGHT and FULL joins + is_inserted = true; + continue; + } + + /// Check condition for right table from ON section + if (join_mask && !(*join_mask)[i]) + continue; + + if constexpr (is_asof_join) + Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); + else if constexpr (mapped_one) + is_inserted |= Inserter::insertOne(join, map, key_getter, stored_block, i, pool); + else + Inserter::insertAll(join, map, key_getter, stored_block, i, pool); + } + return map.getBufferSizeInCells(); +} + +template +template +size_t HashJoinMethods::switchJoinRightColumns( + const std::vector & mapv, + AddedColumns & added_columns, + HashJoin::Type type, + JoinStuff::JoinUsedFlags & used_flags) +{ + constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof; + switch (type) + { + case HashJoin::Type::EMPTY: { + if constexpr (!is_asof_join) + { + using KeyGetter = KeyGetterEmpty; + std::vector key_getter_vector; + key_getter_vector.emplace_back(); + + using MapTypeVal = typename KeyGetter::MappedType; + std::vector a_map_type_vector; + a_map_type_vector.emplace_back(); + return joinRightColumnsSwitchNullability( + std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags); + } + throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys. Type: {}", type); + } +#define M(TYPE) \ + case HashJoin::Type::TYPE: { \ + using MapTypeVal = const typename std::remove_reference_t::element_type; \ + using KeyGetter = typename KeyGetterForType::Type; \ + std::vector a_map_type_vector(mapv.size()); \ + std::vector key_getter_vector; \ + for (size_t d = 0; d < added_columns.join_on_keys.size(); ++d) \ + { \ + const auto & join_on_key = added_columns.join_on_keys[d]; \ + a_map_type_vector[d] = mapv[d]->TYPE.get(); \ + key_getter_vector.push_back( \ + std::move(createKeyGetter(join_on_key.key_columns, join_on_key.key_sizes))); \ + } \ + return joinRightColumnsSwitchNullability(std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags); \ + } + APPLY_FOR_JOIN_VARIANTS(M) +#undef M + + default: + throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", type); + } +} + +template +template +size_t HashJoinMethods::joinRightColumnsSwitchNullability( + std::vector && key_getter_vector, + const std::vector & mapv, + AddedColumns & added_columns, + JoinStuff::JoinUsedFlags & used_flags) +{ + if (added_columns.need_filter) + { + return joinRightColumnsSwitchMultipleDisjuncts( + std::forward>(key_getter_vector), mapv, added_columns, used_flags); + } + else + { + return joinRightColumnsSwitchMultipleDisjuncts( + std::forward>(key_getter_vector), mapv, added_columns, used_flags); + } +} + +template +template +size_t HashJoinMethods::joinRightColumnsSwitchMultipleDisjuncts( + std::vector && key_getter_vector, + const std::vector & mapv, + AddedColumns & added_columns, + JoinStuff::JoinUsedFlags & used_flags) +{ + constexpr JoinFeatures join_features; + if constexpr (join_features.is_maps_all) + { + if (added_columns.additional_filter_expression) + { + bool mark_per_row_used = join_features.right || join_features.full || mapv.size() > 1; + return joinRightColumnsWithAddtitionalFilter( + std::forward>(key_getter_vector), mapv, added_columns, used_flags, need_filter, mark_per_row_used); + } + } + + if (added_columns.additional_filter_expression) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Additional filter expression is not supported for this JOIN"); + + return mapv.size() > 1 ? joinRightColumns( + std::forward>(key_getter_vector), mapv, added_columns, used_flags) + : joinRightColumns( + std::forward>(key_getter_vector), mapv, added_columns, used_flags); +} + + +/// Joins right table columns which indexes are present in right_indexes using specified map. +/// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). +template +template +size_t HashJoinMethods::joinRightColumns( + std::vector && key_getter_vector, + const std::vector & mapv, + AddedColumns & added_columns, + JoinStuff::JoinUsedFlags & used_flags) +{ + constexpr JoinFeatures join_features; + + size_t rows = added_columns.rows_to_add; + if constexpr (need_filter) + added_columns.filter = IColumn::Filter(rows, 0); + + Arena pool; + + if constexpr (join_features.need_replication) + added_columns.offsets_to_replicate = std::make_unique(rows); + + IColumn::Offset current_offset = 0; + size_t max_joined_block_rows = added_columns.max_joined_block_rows; + size_t i = 0; + for (; i < rows; ++i) + { + if constexpr (join_features.need_replication) + { + if (unlikely(current_offset >= max_joined_block_rows)) + { + added_columns.offsets_to_replicate->resize_assume_reserved(i); + added_columns.filter.resize_assume_reserved(i); + break; + } + } + + bool right_row_found = false; + + KnownRowsHolder known_rows; + for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx) + { + const auto & join_keys = added_columns.join_on_keys[onexpr_idx]; + if (join_keys.null_map && (*join_keys.null_map)[i]) + continue; + + bool row_acceptable = !join_keys.isRowFiltered(i); + using FindResult = typename KeyGetter::FindResult; + auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult(); + + if (find_result.isFound()) + { + right_row_found = true; + auto & mapped = find_result.getMapped(); + if constexpr (join_features.is_asof_join) + { + const IColumn & left_asof_key = added_columns.leftAsofKey(); + + auto row_ref = mapped->findAsof(left_asof_key, i); + if (row_ref.block) + { + setUsed(added_columns.filter, i); + if constexpr (flag_per_row) + used_flags.template setUsed(row_ref.block, row_ref.row_num, 0); + else + used_flags.template setUsed(find_result); + + added_columns.appendFromBlock(*row_ref.block, row_ref.row_num, join_features.add_missing); + } + else + addNotFoundRow(added_columns, current_offset); + } + else if constexpr (join_features.is_all_join) + { + setUsed(added_columns.filter, i); + used_flags.template setUsed(find_result); + auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; + addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); + } + else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right) + { + /// Use first appeared left key + it needs left columns replication + bool used_once = used_flags.template setUsedOnce(find_result); + if (used_once) + { + auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; + setUsed(added_columns.filter, i); + addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); + } + } + else if constexpr (join_features.is_any_join && join_features.inner) + { + bool used_once = used_flags.template setUsedOnce(find_result); + + /// Use first appeared left key only + if (used_once) + { + setUsed(added_columns.filter, i); + added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing); + } + + break; + } + else if constexpr (join_features.is_any_join && join_features.full) + { + /// TODO + } + else if constexpr (join_features.is_anti_join) + { + if constexpr (join_features.right && join_features.need_flags) + used_flags.template setUsed(find_result); + } + else /// ANY LEFT, SEMI LEFT, old ANY (RightAny) + { + setUsed(added_columns.filter, i); + used_flags.template setUsed(find_result); + added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing); + + if (join_features.is_any_or_semi_join) + { + break; + } + } + } + } + + if (!right_row_found) + { + if constexpr (join_features.is_anti_join && join_features.left) + setUsed(added_columns.filter, i); + addNotFoundRow(added_columns, current_offset); + } + + if constexpr (join_features.need_replication) + { + (*added_columns.offsets_to_replicate)[i] = current_offset; + } + } + + added_columns.applyLazyDefaults(); + return i; +} + +template +template +void HashJoinMethods::setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]]) +{ + if constexpr (need_filter) + filter[pos] = 1; +} + +template +template +ColumnPtr HashJoinMethods::buildAdditionalFilter( + size_t left_start_row, + const std::vector & selected_rows, + const std::vector & row_replicate_offset, + AddedColumns & added_columns) +{ + ColumnPtr result_column; + do + { + if (selected_rows.empty()) + { + result_column = ColumnUInt8::create(); + break; + } + const Block & sample_right_block = *selected_rows.begin()->block; + if (!sample_right_block || !added_columns.additional_filter_expression) + { + auto filter = ColumnUInt8::create(); + filter->insertMany(1, selected_rows.size()); + result_column = std::move(filter); + break; + } + + auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes(); + if (required_cols.empty()) + { + Block block; + added_columns.additional_filter_expression->execute(block); + result_column = block.getByPosition(0).column->cloneResized(selected_rows.size()); + break; + } + NameSet required_column_names; + for (auto & col : required_cols) + required_column_names.insert(col.name); + + Block executed_block; + size_t right_col_pos = 0; + for (const auto & col : sample_right_block.getColumnsWithTypeAndName()) + { + if (required_column_names.contains(col.name)) + { + auto new_col = col.column->cloneEmpty(); + for (const auto & selected_row : selected_rows) + { + const auto & src_col = selected_row.block->getByPosition(right_col_pos); + new_col->insertFrom(*src_col.column, selected_row.row_num); + } + executed_block.insert({std::move(new_col), col.type, col.name}); + } + right_col_pos += 1; + } + if (!executed_block) + { + result_column = ColumnUInt8::create(); + break; + } + + for (const auto & col_name : required_column_names) + { + const auto * src_col = added_columns.left_block.findByName(col_name); + if (!src_col) + continue; + auto new_col = src_col->column->cloneEmpty(); + size_t prev_left_offset = 0; + for (size_t i = 1; i < row_replicate_offset.size(); ++i) + { + const size_t & left_offset = row_replicate_offset[i]; + size_t rows = left_offset - prev_left_offset; + if (rows) + new_col->insertManyFrom(*src_col->column, left_start_row + i - 1, rows); + prev_left_offset = left_offset; + } + executed_block.insert({std::move(new_col), src_col->type, col_name}); + } + if (!executed_block) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "required columns: [{}], but not found any in left/right table. right table: {}, left table: {}", + required_cols.toString(), + sample_right_block.dumpNames(), + added_columns.left_block.dumpNames()); + } + + for (const auto & col : executed_block.getColumnsWithTypeAndName()) + if (!col.column || !col.type) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal nullptr column in input block: {}", executed_block.dumpStructure()); + + added_columns.additional_filter_expression->execute(executed_block); + result_column = executed_block.getByPosition(0).column->convertToFullColumnIfConst(); + executed_block.clear(); + } while (false); + + result_column = result_column->convertToFullIfNeeded(); + if (result_column->isNullable()) + { + /// Convert Nullable(UInt8) to UInt8 ensuring that nulls are zeros + /// Trying to avoid copying data, since we are the only owner of the column. + ColumnPtr mask_column = assert_cast(*result_column).getNullMapColumnPtr(); + + MutableColumnPtr mutable_column; + { + ColumnPtr nested_column = assert_cast(*result_column).getNestedColumnPtr(); + result_column.reset(); + mutable_column = IColumn::mutate(std::move(nested_column)); + } + + auto & column_data = assert_cast(*mutable_column).getData(); + const auto & mask_column_data = assert_cast(*mask_column).getData(); + for (size_t i = 0; i < column_data.size(); ++i) + { + if (mask_column_data[i]) + column_data[i] = 0; + } + return mutable_column; + } + return result_column; +} + +template +template +size_t HashJoinMethods::joinRightColumnsWithAddtitionalFilter( + std::vector && key_getter_vector, + const std::vector & mapv, + AddedColumns & added_columns, + JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]], + bool need_filter [[maybe_unused]], + bool flag_per_row [[maybe_unused]]) +{ + constexpr JoinFeatures join_features; + size_t left_block_rows = added_columns.rows_to_add; + if (need_filter) + added_columns.filter = IColumn::Filter(left_block_rows, 0); + + std::unique_ptr pool; + + if constexpr (join_features.need_replication) + added_columns.offsets_to_replicate = std::make_unique(left_block_rows); + + std::vector row_replicate_offset; + row_replicate_offset.reserve(left_block_rows); + + using FindResult = typename KeyGetter::FindResult; + size_t max_joined_block_rows = added_columns.max_joined_block_rows; + size_t left_row_iter = 0; + PreSelectedRows selected_rows; + selected_rows.reserve(left_block_rows); + std::vector find_results; + find_results.reserve(left_block_rows); + bool exceeded_max_block_rows = false; + IColumn::Offset total_added_rows = 0; + IColumn::Offset current_added_rows = 0; + + auto collect_keys_matched_rows_refs = [&]() + { + pool = std::make_unique(); + find_results.clear(); + row_replicate_offset.clear(); + row_replicate_offset.push_back(0); + current_added_rows = 0; + selected_rows.clear(); + for (; left_row_iter < left_block_rows; ++left_row_iter) + { + if constexpr (join_features.need_replication) + { + if (unlikely(total_added_rows + current_added_rows >= max_joined_block_rows)) + { + break; + } + } + KnownRowsHolder all_flag_known_rows; + KnownRowsHolder single_flag_know_rows; + for (size_t join_clause_idx = 0; join_clause_idx < added_columns.join_on_keys.size(); ++join_clause_idx) + { + const auto & join_keys = added_columns.join_on_keys[join_clause_idx]; + if (join_keys.null_map && (*join_keys.null_map)[left_row_iter]) + continue; + + bool row_acceptable = !join_keys.isRowFiltered(left_row_iter); + auto find_result = row_acceptable + ? key_getter_vector[join_clause_idx].findKey(*(mapv[join_clause_idx]), left_row_iter, *pool) + : FindResult(); + + if (find_result.isFound()) + { + auto & mapped = find_result.getMapped(); + find_results.push_back(find_result); + if (flag_per_row) + addFoundRowAll(mapped, selected_rows, current_added_rows, all_flag_known_rows, nullptr); + else + addFoundRowAll(mapped, selected_rows, current_added_rows, single_flag_know_rows, nullptr); + } + } + row_replicate_offset.push_back(current_added_rows); + } + }; + + auto copy_final_matched_rows = [&](size_t left_start_row, ColumnPtr filter_col) + { + const PaddedPODArray & filter_flags = assert_cast(*filter_col).getData(); + + size_t prev_replicated_row = 0; + auto selected_right_row_it = selected_rows.begin(); + size_t find_result_index = 0; + for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i) + { + bool any_matched = false; + /// For right join, flag_per_row is true, we need mark used flags for each row. + if (flag_per_row) + { + for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) + { + if (filter_flags[replicated_row]) + { + any_matched = true; + if constexpr (join_features.is_semi_join || join_features.is_any_join) + { + auto used_once = used_flags.template setUsedOnce( + selected_right_row_it->block, selected_right_row_it->row_num, 0); + if (used_once) + { + total_added_rows += 1; + added_columns.appendFromBlock( + *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + } + } + else if constexpr (join_features.is_anti_join) + { + if constexpr (join_features.right && join_features.need_flags) + used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); + } + else + { + total_added_rows += 1; + added_columns.appendFromBlock( + *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + used_flags.template setUsed( + selected_right_row_it->block, selected_right_row_it->row_num, 0); + } + } + ++selected_right_row_it; + } + } + else + { + for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) + { + if constexpr (join_features.is_anti_join) + { + any_matched |= filter_flags[replicated_row]; + } + else if constexpr (join_features.need_replication) + { + if (filter_flags[replicated_row]) + { + any_matched = true; + added_columns.appendFromBlock( + *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + total_added_rows += 1; + } + ++selected_right_row_it; + } + else + { + if (filter_flags[replicated_row]) + { + any_matched = true; + added_columns.appendFromBlock( + *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + total_added_rows += 1; + selected_right_row_it = selected_right_row_it + row_replicate_offset[i] - replicated_row; + break; + } + else + ++selected_right_row_it; + } + } + } + + + if constexpr (join_features.is_anti_join) + { + if (!any_matched) + { + if constexpr (join_features.left) + if (need_filter) + setUsed(added_columns.filter, left_start_row + i - 1); + addNotFoundRow(added_columns, total_added_rows); + } + } + else + { + if (!any_matched) + { + addNotFoundRow(added_columns, total_added_rows); + } + else + { + if (!flag_per_row) + used_flags.template setUsed(find_results[find_result_index]); + if (need_filter) + setUsed(added_columns.filter, left_start_row + i - 1); + if constexpr (join_features.add_missing) + added_columns.applyLazyDefaults(); + } + } + find_result_index += (prev_replicated_row != row_replicate_offset[i]); + + if constexpr (join_features.need_replication) + { + (*added_columns.offsets_to_replicate)[left_start_row + i - 1] = total_added_rows; + } + prev_replicated_row = row_replicate_offset[i]; + } + }; + + while (left_row_iter < left_block_rows && !exceeded_max_block_rows) + { + auto left_start_row = left_row_iter; + collect_keys_matched_rows_refs(); + if (selected_rows.size() != current_added_rows || row_replicate_offset.size() != left_row_iter - left_start_row + 1) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Sizes are mismatched. selected_rows.size:{}, current_added_rows:{}, row_replicate_offset.size:{}, left_row_iter: {}, " + "left_start_row: {}", + selected_rows.size(), + current_added_rows, + row_replicate_offset.size(), + left_row_iter, + left_start_row); + } + auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns); + copy_final_matched_rows(left_start_row, filter_col); + + if constexpr (join_features.need_replication) + { + // Add a check for current_added_rows to avoid run the filter expression on too small size batch. + if (total_added_rows >= max_joined_block_rows || current_added_rows < 1024) + exceeded_max_block_rows = true; + } + } + + if constexpr (join_features.need_replication) + { + added_columns.offsets_to_replicate->resize_assume_reserved(left_row_iter); + added_columns.filter.resize_assume_reserved(left_row_iter); + } + added_columns.applyLazyDefaults(); + return left_row_iter; +} + +template +Block HashJoinMethods::sliceBlock(Block & block, size_t num_rows) +{ + size_t total_rows = block.rows(); + if (num_rows >= total_rows) + return {}; + size_t remaining_rows = total_rows - num_rows; + Block remaining_block = block.cloneEmpty(); + for (size_t i = 0; i < block.columns(); ++i) + { + auto & col = block.getByPosition(i); + remaining_block.getByPosition(i).column = col.column->cut(num_rows, remaining_rows); + col.column = col.column->cut(0, num_rows); + } + return remaining_block; +} + +template +ColumnWithTypeAndName HashJoinMethods::copyLeftKeyColumnToRight( + const DataTypePtr & right_key_type, + const String & renamed_right_column, + const ColumnWithTypeAndName & left_column, + const IColumn::Filter * null_map_filter) +{ + ColumnWithTypeAndName right_column = left_column; + right_column.name = renamed_right_column; + + if (null_map_filter) + right_column.column = JoinCommon::filterWithBlanks(right_column.column, *null_map_filter); + + bool should_be_nullable = isNullableOrLowCardinalityNullable(right_key_type); + if (null_map_filter) + correctNullabilityInplace(right_column, should_be_nullable, *null_map_filter); + else + correctNullabilityInplace(right_column, should_be_nullable); + + if (!right_column.type->equals(*right_key_type)) + { + right_column.column = castColumnAccurate(right_column, right_key_type); + right_column.type = right_key_type; + } + + right_column.column = right_column.column->convertToFullColumnIfConst(); + return right_column; +} + +template +void HashJoinMethods::correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable) +{ + if (nullable) + { + JoinCommon::convertColumnToNullable(column); + } + else + { + /// We have to replace values masked by NULLs with defaults. + if (column.column) + if (const auto * nullable_column = checkAndGetColumn(&*column.column)) + column.column = JoinCommon::filterWithBlanks(column.column, nullable_column->getNullMapColumn().getData(), true); + + JoinCommon::removeColumnNullability(column); + } +} + +template +void HashJoinMethods::correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map) +{ + if (nullable) + { + JoinCommon::convertColumnToNullable(column); + if (column.type->isNullable() && !negative_null_map.empty()) + { + MutableColumnPtr mutable_column = IColumn::mutate(std::move(column.column)); + assert_cast(*mutable_column).applyNegatedNullMap(negative_null_map); + column.column = std::move(mutable_column); + } + } + else + JoinCommon::removeColumnNullability(column); +} +} + diff --git a/src/Interpreters/HashJoin/InnerHashJoin.cpp b/src/Interpreters/HashJoin/InnerHashJoin.cpp index 85aedf3a8e5..258e3143996 100644 --- a/src/Interpreters/HashJoin/InnerHashJoin.cpp +++ b/src/Interpreters/HashJoin/InnerHashJoin.cpp @@ -1,5 +1,5 @@ -#include +#include namespace DB { diff --git a/src/Interpreters/HashJoin/LeftHashJoin.cpp b/src/Interpreters/HashJoin/LeftHashJoin.cpp index a53ffaac0b5..4e06789570e 100644 --- a/src/Interpreters/HashJoin/LeftHashJoin.cpp +++ b/src/Interpreters/HashJoin/LeftHashJoin.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { diff --git a/src/Interpreters/HashJoin/RightHashJoin.cpp b/src/Interpreters/HashJoin/RightHashJoin.cpp index 8e304754f5c..d9d41d7d63c 100644 --- a/src/Interpreters/HashJoin/RightHashJoin.cpp +++ b/src/Interpreters/HashJoin/RightHashJoin.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { From bd4d648f63d56d11c729f719f864a8b9985c43c3 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 23 Jul 2024 10:41:57 +0800 Subject: [PATCH 367/644] update doc --- docs/en/sql-reference/statements/select/join.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index 96d9d26977d..b228f7025c4 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -186,7 +186,7 @@ Otherwise, you'll get `INVALID_JOIN_ON_EXPRESSION`. ::: -Clickhouse currently supports `ALL INNER/LEFT/RIGHT/FULL JOIN` with inequality conditions in addition to equality conditions. The inequality conditions are supported only for `hash` and `grace_hash` join algorithms. The inequality conditions are not supported with `join_use_nulls`. +Clickhouse currently supports `ALL/ANY/SEMI/ANTI INNER/LEFT/RIGHT/FULL JOIN` with inequality conditions in addition to equality conditions. The inequality conditions are supported only for `hash` and `grace_hash` join algorithms. The inequality conditions are not supported with `join_use_nulls`. **Example** From 58b7ac2264eb6d2ba83d634e1d52b874ca54d9c7 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 7 Aug 2024 11:36:01 +0800 Subject: [PATCH 368/644] update --- src/Interpreters/HashJoin/HashJoin.cpp | 2 +- src/Interpreters/HashJoin/HashJoinMethods.h | 1 + src/Interpreters/HashJoin/InnerHashJoin.cpp | 1 + src/Interpreters/joinDispatch.h | 20 +++-- ..._join_on_inequal_expression_fast.reference | 78 +++++++++++++++++++ ...006_join_on_inequal_expression_fast.sql.j2 | 31 ++++++++ 6 files changed, 127 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 769cb574ed7..4033d1e3035 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -1327,7 +1327,7 @@ void HashJoin::validateAdditionalFilterExpression(ExpressionActionsPtr additiona bool is_supported = ((strictness == JoinStrictness::All) && (isInnerOrLeft(kind) || isRightOrFull(kind))) || ((strictness == JoinStrictness::Semi || strictness == JoinStrictness::Any || strictness == JoinStrictness::Anti) - && (isLeft(kind) || isRight(kind))); + && (isLeft(kind) || isRight(kind))) || (strictness == JoinStrictness::Any && (isInner(kind))); if (!is_supported) { throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index e3b8fbc1737..3b7a67467e3 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -185,6 +185,7 @@ extern template class HashJoinMethods; extern template class HashJoinMethods; +extern template class HashJoinMethods; extern template class HashJoinMethods; extern template class HashJoinMethods; extern template class HashJoinMethods; diff --git a/src/Interpreters/HashJoin/InnerHashJoin.cpp b/src/Interpreters/HashJoin/InnerHashJoin.cpp index 258e3143996..69f4c620cb8 100644 --- a/src/Interpreters/HashJoin/InnerHashJoin.cpp +++ b/src/Interpreters/HashJoin/InnerHashJoin.cpp @@ -5,6 +5,7 @@ namespace DB { template class HashJoinMethods; template class HashJoinMethods; +template class HashJoinMethods; template class HashJoinMethods; template class HashJoinMethods; template class HashJoinMethods; diff --git a/src/Interpreters/joinDispatch.h b/src/Interpreters/joinDispatch.h index 982c56e8210..5d4bd8f92e5 100644 --- a/src/Interpreters/joinDispatch.h +++ b/src/Interpreters/joinDispatch.h @@ -12,6 +12,15 @@ namespace DB { +/// HashJoin::MapsOne is more efficient, it only store one row for each key in the map. It is recommended to use it whenever possible. +/// When only need to match only one row from right table, use HashJoin::MapsOne. For example, LEFT ANY/SEMI/ANTI. +/// +/// HashJoin::MapsAll will store all rows for each key in the map. It is used when need to match multiple rows from right table. +/// For example, RIGHT ANY/ALL, FULL JOIN, INNER JOIN. +/// +/// prefer_use_maps_all is true when there is mixed inequal condition in the join condition. For example, `t1.a = t2.a AND t1.b > t2.b`. +/// In this case, we need to use HashJoin::MapsAll to store all rows for each key in the map. We will select all matched rows from the map +/// and filter them by `t1.b > t2.b`. template struct MapGetter; @@ -21,8 +30,9 @@ template struct MapGetter struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; -template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; @@ -33,14 +43,14 @@ template struct MapGetter struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -/// Only SEMI LEFT and SEMI RIGHT are valid. INNER and FULL are here for templates instantiation. +/// Only ANTI LEFT and ANTI RIGHT are valid. INNER and FULL are here for templates instantiation. template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference index 3e413afd98e..11ac01d24d5 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference @@ -1,3 +1,4 @@ +03006_join_on_inequal_expression_fast.sql -- { echoOn } SET join_algorithm='hash'; SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); @@ -596,6 +597,39 @@ SELECT t1.*, t2.* from t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key and (t1.a < t2. 0 0 \N key4 F 1 1 1 SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT ANTI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); SET join_algorithm='hash'; +SELECT t1.* FROM t1 LEFT ANY JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +key1 a 1 1 2 +key1 b 2 3 2 +key1 c 3 2 1 +key1 d 4 7 2 +key1 e 5 5 5 +key2 a2 1 1 1 +key4 f 2 3 4 +SELECT t1.* FROM t1 LEFT SEMI JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +key1 a 1 1 2 +key1 b 2 3 2 +key1 c 3 2 1 +key1 d 4 7 2 +key2 a2 1 1 1 +key4 f 2 3 4 +SELECT t1.* FROM t1 LEFT ANTI JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +key1 e 5 5 5 +SELECT t1.* FROM t1 RIGHT ANY JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +SELECT t1.* FROM t1 RIGHT SEMI JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +SELECT t1.* FROM t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +SET join_algorithm='hash'; SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr); key1 a 1 1 2 key1 A 1 2 1 key1 a 1 1 2 key1 B 2 1 2 @@ -666,3 +700,47 @@ key2 a2 1 1 1 key1 A 1 2 1 key2 a2 1 1 1 key3 a3 1 1 1 key2 a2 1 1 1 key4 F 1 1 1 key4 f 2 3 4 key1 B 2 1 2 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 INNER ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 B 2 1 2 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 INNER ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 INNER ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 B 2 1 2 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 INNER ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SET join_algorithm='hash'; +SELECT t1.* FROM t1 INNER ANY JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +key1 a 1 1 2 +key1 b 2 3 2 +key1 c 3 2 1 +key1 d 4 7 2 +key2 a2 1 1 1 +key4 f 2 3 4 diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 index 1bf5a7870e7..b300881c562 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 @@ -34,12 +34,43 @@ SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 {{ join_type }} {{ join_stri {% endfor -%} {% endfor -%} +{% for algorithm in ['hash'] -%} +SET join_algorithm='{{ algorithm }}'; +{% for join_type in ['LEFT', 'RIGHT'] -%} +{% for join_strictness in ['ANY', 'SEMI', 'ANTI'] -%} +SELECT t1.* FROM t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +{% endfor -%} +{% endfor -%} +{% endfor -%} + {% for algorithm in ['hash'] -%} SET join_algorithm='{{ algorithm }}'; {% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr); {% endfor -%} {% endfor -%} + +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +{% for join_type in ['INNER'] -%} +{% for join_strictness in ['ANY'] -%} +SELECT t1.*, t2.* FROM t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +SELECT t1.*, t2.* from t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +SELECT t1.*, t2.* from t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 {{ join_type }} {{ join_strictness }} JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +{% endfor -%} +{% endfor -%} +{% endfor -%} + +{% for algorithm in ['hash'] -%} +SET join_algorithm='{{ algorithm }}'; +{% for join_type in ['INNER'] -%} +{% for join_strictness in ['ANY'] -%} +SELECT t1.* FROM t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +{% endfor -%} +{% endfor -%} +{% endfor -%} + -- { echoOff } -- test error messages From 5eb896b9f1976feaa423071919e65d22e09da4ea Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 7 Aug 2024 10:43:41 +0200 Subject: [PATCH 369/644] Add documentation for toDecimal32 and variants --- .../functions/type-conversion-functions.md | 471 ++++++++++++------ 1 file changed, 305 insertions(+), 166 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 1e618b8cdab..24055bb99b7 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -95,7 +95,7 @@ SELECT toInt8(-8), toInt8(-8.8), toInt8('-8') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -156,7 +156,7 @@ Query: SELECT toInt8OrZero('-8'), toInt8OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -216,7 +216,7 @@ Query: SELECT toInt8OrNull('-8'), toInt8OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -281,7 +281,7 @@ Query: SELECT toInt8OrDefault('-8', CAST('-1', 'Int8')), toInt8OrDefault('abc', CAST('-1', 'Int8')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -345,7 +345,7 @@ SELECT toInt16(-16), toInt16(-16.16), toInt16('-16') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -406,7 +406,7 @@ Query: SELECT toInt16OrZero('-16'), toInt16OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -466,7 +466,7 @@ Query: SELECT toInt16OrNull('-16'), toInt16OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -531,7 +531,7 @@ Query: SELECT toInt16OrDefault('-16', CAST('-1', 'Int16')), toInt16OrDefault('abc', CAST('-1', 'Int16')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -595,7 +595,7 @@ SELECT toInt32(-32), toInt32(-32.32), toInt32('-32') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -656,7 +656,7 @@ Query: SELECT toInt32OrZero('-32'), toInt32OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -715,7 +715,7 @@ Query: SELECT toInt32OrNull('-32'), toInt32OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -780,7 +780,7 @@ Query: SELECT toInt32OrDefault('-32', CAST('-1', 'Int32')), toInt32OrDefault('abc', CAST('-1', 'Int32')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -844,7 +844,7 @@ SELECT toInt64(-64), toInt64(-64.64), toInt64('-64') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -905,7 +905,7 @@ Query: SELECT toInt64OrZero('-64'), toInt64OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -965,7 +965,7 @@ Query: SELECT toInt64OrNull('-64'), toInt64OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1030,7 +1030,7 @@ Query: SELECT toInt64OrDefault('-64', CAST('-1', 'Int64')), toInt64OrDefault('abc', CAST('-1', 'Int64')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1093,7 +1093,7 @@ SELECT toInt128(-128), toInt128(-128.8), toInt128('-128') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1154,7 +1154,7 @@ Query: SELECT toInt128OrZero('-128'), toInt128OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1214,7 +1214,7 @@ Query: SELECT toInt128OrNull('-128'), toInt128OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1279,7 +1279,7 @@ Query: SELECT toInt128OrDefault('-128', CAST('-1', 'Int128')), toInt128OrDefault('abc', CAST('-1', 'Int128')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1342,7 +1342,7 @@ SELECT toInt256(-256), toInt256(-256.256), toInt256('-256') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1403,7 +1403,7 @@ Query: SELECT toInt256OrZero('-256'), toInt256OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1463,7 +1463,7 @@ Query: SELECT toInt256OrNull('-256'), toInt256OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1528,7 +1528,7 @@ Query: SELECT toInt256OrDefault('-256', CAST('-1', 'Int256')), toInt256OrDefault('abc', CAST('-1', 'Int256')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1592,7 +1592,7 @@ SELECT toUInt8(8), toUInt8(8.8), toUInt8('8') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1653,7 +1653,7 @@ Query: SELECT toUInt8OrZero('-8'), toUInt8OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1713,7 +1713,7 @@ Query: SELECT toUInt8OrNull('8'), toUInt8OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1778,7 +1778,7 @@ Query: SELECT toUInt8OrDefault('8', CAST('0', 'UInt8')), toUInt8OrDefault('abc', CAST('0', 'UInt8')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1842,7 +1842,7 @@ SELECT toUInt16(16), toUInt16(16.16), toUInt16('16') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1903,7 +1903,7 @@ Query: SELECT toUInt16OrZero('16'), toUInt16OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1963,7 +1963,7 @@ Query: SELECT toUInt16OrNull('16'), toUInt16OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2028,7 +2028,7 @@ Query: SELECT toUInt16OrDefault('16', CAST('0', 'UInt16')), toUInt16OrDefault('abc', CAST('0', 'UInt16')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2092,7 +2092,7 @@ SELECT toUInt32(32), toUInt32(32.32), toUInt32('32') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2154,7 +2154,7 @@ Query: SELECT toUInt32OrZero('32'), toUInt32OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2214,7 +2214,7 @@ Query: SELECT toUInt32OrNull('32'), toUInt32OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2279,7 +2279,7 @@ Query: SELECT toUInt32OrDefault('32', CAST('0', 'UInt32')), toUInt32OrDefault('abc', CAST('0', 'UInt32')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2343,7 +2343,7 @@ SELECT toUInt64(64), toUInt64(64.64), toUInt64('64') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2404,7 +2404,7 @@ Query: SELECT toUInt64OrZero('64'), toUInt64OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2464,7 +2464,7 @@ Query: SELECT toUInt64OrNull('64'), toUInt64OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2529,7 +2529,7 @@ Query: SELECT toUInt64OrDefault('64', CAST('0', 'UInt64')), toUInt64OrDefault('abc', CAST('0', 'UInt64')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2592,7 +2592,7 @@ SELECT toUInt128(128), toUInt128(128.8), toUInt128('128') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2653,7 +2653,7 @@ Query: SELECT toUInt128OrZero('128'), toUInt128OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2713,7 +2713,7 @@ Query: SELECT toUInt128OrNull('128'), toUInt128OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2778,7 +2778,7 @@ Query: SELECT toUInt128OrDefault('128', CAST('0', 'UInt128')), toUInt128OrDefault('abc', CAST('0', 'UInt128')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2841,7 +2841,7 @@ SELECT toUInt256(256), toUInt256(256.256), toUInt256('256') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2902,7 +2902,7 @@ Query: SELECT toUInt256OrZero('256'), toUInt256OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2962,7 +2962,7 @@ Query: SELECT toUInt256OrNull('256'), toUInt256OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -3027,7 +3027,7 @@ Query: SELECT toUInt256OrDefault('-256', CAST('0', 'UInt256')), toUInt256OrDefault('abc', CAST('0', 'UInt256')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -3542,173 +3542,312 @@ SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Asia/Istanbul') AS value, toTypeN ## toDateTime64OrDefault -## toDecimal(32\|64\|128\|256) +## toDecimal32 -Converts `value` to the [Decimal](../data-types/decimal.md) data type with precision of `S`. The `value` can be a number or a string. The `S` (scale) parameter specifies the number of decimal places. +Converts an input value to a value of type [`Decimal(9, S)`](../data-types/decimal.md) with scale of `S`. Throws an exception in case of an error. -- `toDecimal32(value, S)` -- `toDecimal64(value, S)` -- `toDecimal128(value, S)` -- `toDecimal256(value, S)` +**Syntax** -## toDecimal(32\|64\|128\|256)OrNull - -Converts an input string to a [Nullable(Decimal(P,S))](../data-types/decimal.md) data type value. This family of functions includes: - -- `toDecimal32OrNull(expr, S)` — Results in `Nullable(Decimal32(S))` data type. -- `toDecimal64OrNull(expr, S)` — Results in `Nullable(Decimal64(S))` data type. -- `toDecimal128OrNull(expr, S)` — Results in `Nullable(Decimal128(S))` data type. -- `toDecimal256OrNull(expr, S)` — Results in `Nullable(Decimal256(S))` data type. - -These functions should be used instead of `toDecimal*()` functions, if you prefer to get a `NULL` value instead of an exception in the event of an input value parsing error. +```sql +toDecimal32(expr, S) +``` **Arguments** -- `expr` — [Expression](../syntax.md/#syntax-expressions), returns a value in the [String](../data-types/string.md) data type. ClickHouse expects the textual representation of the decimal number. For example, `'1.111'`. -- `S` — Scale, the number of decimal places in the resulting value. +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- Values or string representations of type (U)Int8/16/32/64/128/256. +- Values or string representations of type Float32/64. + +Unsupported arguments: +- Values or string representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32('0xc0fe', 1);`. + +:::note +Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an exception. +::: **Returned value** -A value in the `Nullable(Decimal(P,S))` data type. The value contains: +- Value of type `Decimal(9, S)`. [Decimal32(S)](../data-types/int-uint.md). -- Number with `S` decimal places, if ClickHouse interprets the input string as a number. -- `NULL`, if ClickHouse can’t interpret the input string as a number or if the input number contains more than `S` decimal places. - -**Examples** +**Example** Query: -``` sql -SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val); +```sql +SELECT + toDecimal32(2, 1) AS a, toTypeName(a) AS type_a, + toDecimal32(4.2, 2) AS b, toTypeName(b) AS type_b, + toDecimal32('4.2', 3) AS c, toTypeName(c) AS type_c +FORMAT Vertical; ``` Result: ```response -┌────val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 5))─┐ -│ -1.111 │ Nullable(Decimal(9, 5)) │ -└────────┴────────────────────────────────────────────────────┘ +Row 1: +────── +a: 2 +type_a: Decimal(9, 1) +b: 4.2 +type_b: Decimal(9, 2) +c: 4.2 +type_c: Decimal(9, 3) ``` -Query: +**See also** -``` sql -SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); +- [`toDecimal32OrZero`](#todecimal32orzero). +- [`toDecimal32OrNull`](#todecimal32ornull). +- [`toDecimal32OrDefault`](#todecimal32ordefault). + +## toDecimal32OrZero + +Like [`toDecimal32`](#todecimal32), this function converts an input value to a value of type [Decimal(9, S)](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal32OrZero(expr, S) ``` -Result: - -```response -┌──val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 2))─┐ -│ ᴺᵁᴸᴸ │ Nullable(Decimal(9, 2)) │ -└──────┴────────────────────────────────────────────────────┘ -``` - - -## toDecimal(32\|64\|128\|256)OrDefault - -Converts an input string to a [Decimal(P,S)](../data-types/decimal.md) data type value. This family of functions includes: - -- `toDecimal32OrDefault(expr, S)` — Results in `Decimal32(S)` data type. -- `toDecimal64OrDefault(expr, S)` — Results in `Decimal64(S)` data type. -- `toDecimal128OrDefault(expr, S)` — Results in `Decimal128(S)` data type. -- `toDecimal256OrDefault(expr, S)` — Results in `Decimal256(S)` data type. - -These functions should be used instead of `toDecimal*()` functions, if you prefer to get a default value instead of an exception in the event of an input value parsing error. - **Arguments** -- `expr` — [Expression](../syntax.md/#syntax-expressions), returns a value in the [String](../data-types/string.md) data type. ClickHouse expects the textual representation of the decimal number. For example, `'1.111'`. -- `S` — Scale, the number of decimal places in the resulting value. +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32OrZero('0xc0fe', 1);`. + +:::note +Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an exception. +::: **Returned value** -A value in the `Decimal(P,S)` data type. The value contains: - -- Number with `S` decimal places, if ClickHouse interprets the input string as a number. -- Default `Decimal(P,S)` data type value, if ClickHouse can’t interpret the input string as a number or if the input number contains more than `S` decimal places. - -**Examples** - -Query: - -``` sql -SELECT toDecimal32OrDefault(toString(-1.111), 5) AS val, toTypeName(val); -``` - -Result: - -```response -┌────val─┬─toTypeName(toDecimal32OrDefault(toString(-1.111), 5))─┐ -│ -1.111 │ Decimal(9, 5) │ -└────────┴───────────────────────────────────────────────────────┘ -``` - -Query: - -``` sql -SELECT toDecimal32OrDefault(toString(-1.111), 2) AS val, toTypeName(val); -``` - -Result: - -```response -┌─val─┬─toTypeName(toDecimal32OrDefault(toString(-1.111), 2))─┐ -│ 0 │ Decimal(9, 2) │ -└─────┴───────────────────────────────────────────────────────┘ -``` - -## toDecimal(32\|64\|128\|256)OrZero - -Converts an input value to the [Decimal(P,S)](../data-types/decimal.md) data type. This family of functions includes: - -- `toDecimal32OrZero( expr, S)` — Results in `Decimal32(S)` data type. -- `toDecimal64OrZero( expr, S)` — Results in `Decimal64(S)` data type. -- `toDecimal128OrZero( expr, S)` — Results in `Decimal128(S)` data type. -- `toDecimal256OrZero( expr, S)` — Results in `Decimal256(S)` data type. - -These functions should be used instead of `toDecimal*()` functions, if you prefer to get a `0` value instead of an exception in the event of an input value parsing error. - -**Arguments** - -- `expr` — [Expression](../syntax.md/#syntax-expressions), returns a value in the [String](../data-types/string.md) data type. ClickHouse expects the textual representation of the decimal number. For example, `'1.111'`. -- `S` — Scale, the number of decimal places in the resulting value. - -**Returned value** - -A value in the `Nullable(Decimal(P,S))` data type. The value contains: - -- Number with `S` decimal places, if ClickHouse interprets the input string as a number. -- 0 with `S` decimal places, if ClickHouse can’t interpret the input string as a number or if the input number contains more than `S` decimal places. +- Value of type `Decimal(9, S)` if successful, otherwise `0` with `S` decimal places. [Decimal32(S)](../data-types/decimal.md). **Example** Query: ``` sql -SELECT toDecimal32OrZero(toString(-1.111), 5) AS val, toTypeName(val); +SELECT + toDecimal32OrZero(toString(-1.111), 5) AS val, + toTypeName(val) +FORMAT Vertical; ``` Result: ```response -┌────val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 5))─┐ -│ -1.111 │ Decimal(9, 5) │ -└────────┴────────────────────────────────────────────────────┘ +Row 1: +────── +val: -1.111 +toTypeName(val): Decimal(9, 5) ``` Query: ``` sql -SELECT toDecimal32OrZero(toString(-1.111), 2) AS val, toTypeName(val); +SELECT + toDecimal32OrZero(toString(-1.111), 2) AS val, + toTypeName(val) +FORMAT Vertical; ``` Result: ```response -┌──val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 2))─┐ -│ 0.00 │ Decimal(9, 2) │ -└──────┴────────────────────────────────────────────────────┘ +Row 1: +────── +val: -1.11 +toTypeName(val): Decimal(9, 2) +``` + +**See also** + +- [`toDecimal32`](#todecimal32). +- [`toDecimal32OrNull`](#todecimal32ornull). +- [`toDecimal32OrDefault`](#todecimal32ordefault). + +## toDecimal32OrNull + +Like [`toDecimal32`](#todecimal32), this function converts an input value to a value of type [Nullable(Decimal(9, S))](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal32OrNull(expr, S) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32OrNull('0xc0fe', 1);`. + +:::note +Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an exception. +::: + +**Returned value** + +- Value of type `Nullable(Decimal(9, S))` if successful, otherwise value `NULL` of the same type. [Decimal32(S)](../data-types/decimal.md). + +**Examples** + +Query: + +``` sql +SELECT + toDecimal32OrNull(toString(-1.111), 5) AS val, + toTypeName(val) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +val: -1.111 +toTypeName(val): Nullable(Decimal(9, 5)) +``` + +Query: + +``` sql +SELECT + toDecimal32OrNull(toString(-1.111), 2) AS val, + toTypeName(val) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +val: -1.11 +toTypeName(val): Nullable(Decimal(9, 2)) +``` + +**See also** + +- [`toDecimal32`](#todecimal32). +- [`toDecimal32OrZero`](#todecimal32orzero). +- [`toDecimal32OrDefault`](#todecimal32ordefault). + +## toDecimal32OrDefault + +Like [`toDecimal32`](#todecimal32), this function converts an input value to a value of type [Decimal(9, S)](../data-types/decimal.md) but returns the default value in case of an error. + +**Syntax** + +```sql +toDecimal32OrDefault(expr, S[, default]) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Decimal32(S)` is unsuccessful. [Decimal32(S](../data-types/decimal.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32OrDefault('0xc0fe', 1);`. + +:::note +Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an exception. +::: + +**Returned value** + +- Value of type `Decimal(9, S)` if successful, otherwise returns the default value if passed or `0` if not. [Decimal32(S)](../data-types/decimal.md). + +**Examples** + +Query: + +``` sql +SELECT + toDecimal32OrDefault(toString(-1.111), 5) AS val, + toTypeName(val) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +val: -1.111 +toTypeName(val): Decimal(9, 5) +``` + +Query: + +``` sql +SELECT + toDecimal32OrDefault(toString(-1.111), 2) AS val, + toTypeName(val) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +val: -1.11 +toTypeName(val): Decimal(9, 2) +``` + +Query: + +``` sql +SELECT + toDecimal32OrDefault('Inf', 2, CAST('0', 'Decimal32(2)')) AS val, + toTypeName(val) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +val: 0 +toTypeName(val): Decimal(9, 2) ``` ## toString From c933a38955e288afbef5c246fed9640878b0a68f Mon Sep 17 00:00:00 2001 From: khodyrevyurii Date: Tue, 6 Aug 2024 22:53:24 +0500 Subject: [PATCH 370/644] change std::thread::hardware_concurrency on container friendly method getNumberOfPhysicalCPUCores --- programs/server/Server.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 7800ee9ff00..46bbc235fee 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1623,7 +1623,7 @@ try concurrent_threads_soft_limit = new_server_settings.concurrent_threads_soft_limit_num; if (new_server_settings.concurrent_threads_soft_limit_ratio_to_cores > 0) { - auto value = new_server_settings.concurrent_threads_soft_limit_ratio_to_cores * std::thread::hardware_concurrency(); + auto value = new_server_settings.concurrent_threads_soft_limit_ratio_to_cores * getNumberOfPhysicalCPUCores(); if (value > 0 && value < concurrent_threads_soft_limit) concurrent_threads_soft_limit = value; } From 00b62b1c0dccd16e45cef445cc8bf717b2da6486 Mon Sep 17 00:00:00 2001 From: khodyrevyurii Date: Wed, 7 Aug 2024 01:26:25 +0500 Subject: [PATCH 371/644] Minor clarifycation for method getNumberOfPhysicalCPUCores --- programs/server/Server.cpp | 7 ++++--- src/Common/getNumberOfPhysicalCPUCores.h | 1 + 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 46bbc235fee..618bd2b011c 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -814,10 +814,11 @@ try const size_t physical_server_memory = getMemoryAmount(); - LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.", + LOG_INFO(log, "Available RAM: {}; logical cores: {}; used cores: {}.", formatReadableSizeWithBinarySuffix(physical_server_memory), - getNumberOfPhysicalCPUCores(), // on ARM processors it can show only enabled at current moment cores - std::thread::hardware_concurrency()); + std::thread::hardware_concurrency(), + getNumberOfPhysicalCPUCores() // on ARM processors it can show only enabled at current moment cores + ); #if defined(__x86_64__) String cpu_info; diff --git a/src/Common/getNumberOfPhysicalCPUCores.h b/src/Common/getNumberOfPhysicalCPUCores.h index 827e95e1bea..9e3412fdcba 100644 --- a/src/Common/getNumberOfPhysicalCPUCores.h +++ b/src/Common/getNumberOfPhysicalCPUCores.h @@ -1,4 +1,5 @@ #pragma once /// Get number of CPU cores without hyper-threading. +/// The calculation respects possible cgroups limits. unsigned getNumberOfPhysicalCPUCores(); From 4f2b1c36b7115143a23462282dc5474ed5b90afd Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 7 Aug 2024 11:12:09 +0200 Subject: [PATCH 372/644] Fix typo from previous PR --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 24055bb99b7..5db44da3e2d 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -2362,7 +2362,7 @@ toUInt64('64'): 64 - [`toUInt64OrNull`](#touint64ornull). - [`toUInt64OrDefault`](#touint64ordefault). -## toInt64OrZero +## toUInt64OrZero Like [`toUInt64`](#touint64), this function converts an input value to a value of type [UInt64](../data-types/int-uint.md) but returns `0` in case of an error. From b76e4acbc0a260f5222249a250066c77d2fcaff8 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 7 Aug 2024 11:13:56 +0200 Subject: [PATCH 373/644] fix another typo --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 5db44da3e2d..8e72fea7fdb 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1546,7 +1546,7 @@ toInt256OrDefault('abc', CAST('-1', 'Int256')): -1 - [`toInt256OrZero`](#toint256orzero). - [`toInt256OrNull`](#toint256ornull). -# toUInt8 +## toUInt8 Converts an input value to a value of type [`UInt8`](../data-types/int-uint.md). Throws an exception in case of an error. From e4134f5a51a1ad6d46c60337b9a3b5f8695d8020 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 7 Aug 2024 09:16:19 +0000 Subject: [PATCH 374/644] catch exception in destructor of `LocalFileHolder` Signed-off-by: Duc Canh Le --- src/Storages/Cache/ExternalDataSourceCache.cpp | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index cffb1dc9ca3..8c778fd511a 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -57,8 +57,15 @@ LocalFileHolder::~LocalFileHolder() { if (original_readbuffer) { - assert_cast(original_readbuffer.get())->seek(0, SEEK_SET); - file_cache_controller->value().startBackgroundDownload(std::move(original_readbuffer), *thread_pool); + try + { + assert_cast(original_readbuffer.get())->seek(0, SEEK_SET); + file_cache_controller->value().startBackgroundDownload(std::move(original_readbuffer), *thread_pool); + } + catch (...) + { + tryLogCurrentException(getLogger("LocalFileHolder"), "Exception during destructor of LocalFileHolder."); + } } } From 6172c56c1fd27f39d11914542f9e2dcb94fffd36 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Aug 2024 09:48:38 +0000 Subject: [PATCH 375/644] Split tests to separate vaguely correlated tests --- ...3217_filtering_in_storage_merge.reference} | 6 +---- .../03217_filtering_in_storage_merge.sql | 16 +++++++++++ ...03217_filtering_in_system_tables.reference | 4 +++ ...l => 03217_filtering_in_system_tables.sql} | 27 +++++++------------ 4 files changed, 30 insertions(+), 23 deletions(-) rename tests/queries/0_stateless/{03217_read_rows_in_system_tables.reference => 03217_filtering_in_storage_merge.reference} (54%) create mode 100644 tests/queries/0_stateless/03217_filtering_in_storage_merge.sql create mode 100644 tests/queries/0_stateless/03217_filtering_in_system_tables.reference rename tests/queries/0_stateless/{03217_read_rows_in_system_tables.sql => 03217_filtering_in_system_tables.sql} (55%) diff --git a/tests/queries/0_stateless/03217_read_rows_in_system_tables.reference b/tests/queries/0_stateless/03217_filtering_in_storage_merge.reference similarity index 54% rename from tests/queries/0_stateless/03217_read_rows_in_system_tables.reference rename to tests/queries/0_stateless/03217_filtering_in_storage_merge.reference index b21ead49b1e..d366ad04c39 100644 --- a/tests/queries/0_stateless/03217_read_rows_in_system_tables.reference +++ b/tests/queries/0_stateless/03217_filtering_in_storage_merge.reference @@ -1,10 +1,6 @@ -information_schema tables -default test_replica_1 r1 Expression ((Project names + Projection)) Aggregating Expression (Before GROUP BY) ReadFromMerge Filter (( + ( + ))) - ReadFromMergeTree (default.test_replica_1) -1 1 -1 1 + ReadFromMergeTree (default.test_03217_merge_replica_1) diff --git a/tests/queries/0_stateless/03217_filtering_in_storage_merge.sql b/tests/queries/0_stateless/03217_filtering_in_storage_merge.sql new file mode 100644 index 00000000000..5ecc1e7c672 --- /dev/null +++ b/tests/queries/0_stateless/03217_filtering_in_storage_merge.sql @@ -0,0 +1,16 @@ +CREATE TABLE test_03217_merge_replica_1(x UInt32) + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217_merge_replica', 'r1') + ORDER BY x; +CREATE TABLE test_03217_merge_replica_2(x UInt32) + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217_merge_replica', 'r2') + ORDER BY x; + + +CREATE TABLE test_03217_all_replicas (x UInt32) + ENGINE = Merge(currentDatabase(), 'test_03217_merge_replica_*'); + +INSERT INTO test_03217_merge_replica_1 SELECT number AS x FROM numbers(10); +SYSTEM SYNC REPLICA test_03217_merge_replica_2; + +-- If the filter on _table is not applied, then the plan will show both replicas +EXPLAIN SELECT _table, count() FROM test_03217_all_replicas WHERE _table = 'test_03217_merge_replica_1' AND x >= 0 GROUP BY _table; diff --git a/tests/queries/0_stateless/03217_filtering_in_system_tables.reference b/tests/queries/0_stateless/03217_filtering_in_system_tables.reference new file mode 100644 index 00000000000..218fddf92e0 --- /dev/null +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.reference @@ -0,0 +1,4 @@ +information_schema tables +default test_03217_system_tables_replica_1 r1 +1 1 +1 1 diff --git a/tests/queries/0_stateless/03217_read_rows_in_system_tables.sql b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql similarity index 55% rename from tests/queries/0_stateless/03217_read_rows_in_system_tables.sql rename to tests/queries/0_stateless/03217_filtering_in_system_tables.sql index 3bea04ccccf..bbc755e478d 100644 --- a/tests/queries/0_stateless/03217_read_rows_in_system_tables.sql +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql @@ -1,27 +1,18 @@ +-- If filtering is not done correctly on databases, then this query report to read 3 rows, which are: `system.tables`, `information_schema.tables` and `INFORMATION_SCHEMA.tables` SELECT database, table FROM system.tables WHERE database = 'information_schema' AND table = 'tables'; --- To verify StorageSystemReplicas applies the filter properly -CREATE TABLE test_replica_1(x UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217/test_replica', 'r1') +CREATE TABLE test_03217_system_tables_replica_1(x UInt32) + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217_system_tables_replica', 'r1') ORDER BY x; -CREATE TABLE test_replica_2(x UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217/test_replica', 'r2') +CREATE TABLE test_03217_system_tables_replica_2(x UInt32) + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217_system_tables_replica', 'r2') ORDER BY x; -SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = 'test_replica_1' AND replica_name = 'r1'; - - --- To verify StorageMerge -CREATE TABLE all_replicas (x UInt32) - ENGINE = Merge(currentDatabase(), 'test_replica_*'); - -INSERT INTO test_replica_1 SELECT number AS x FROM numbers(10); -SYSTEM SYNC REPLICA test_replica_2; --- If the filter not applied, then the plan will show both replicas -EXPLAIN SELECT _table, count() FROM all_replicas WHERE _table = 'test_replica_1' AND x >= 0 GROUP BY _table; +-- If filtering is not done correctly on database-table column, then this query report to read 2 rows, which are the above tables +SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = 'test_03217_system_tables_replica_1' AND replica_name = 'r1'; SYSTEM FLUSH LOGS; --- argMin-argMax make the test repeatable +-- argMin-argMax is necessary to make the test repeatable -- StorageSystemTables SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 @@ -30,5 +21,5 @@ SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_ -- StorageSystemReplicas SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 - AND query LIKE '%SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = \'test_replica_1\' AND replica_name = \'r1\';' + AND query LIKE '%SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = \'test_03217_system_tables_replica_1\' AND replica_name = \'r1\';' AND type = 'QueryFinish'; From a9c284dd8efb439ff06cf0f95e2a9920a26fdf5d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Aug 2024 10:07:27 +0000 Subject: [PATCH 376/644] Include fixes aafe498 and cfaa852 --- src/Storages/Statistics/Statistics.cpp | 30 +++++++++++++++++-- src/Storages/Statistics/Statistics.h | 6 ++++ src/Storages/Statistics/StatisticsTDigest.cpp | 18 ++++------- .../02864_statistics_bugs.reference | 2 ++ .../0_stateless/02864_statistics_bugs.sql | 18 +++++++++++ 5 files changed, 60 insertions(+), 14 deletions(-) diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 52eec437ac2..fd686c5f0aa 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -1,14 +1,17 @@ #include +#include +#include +#include +#include #include #include +#include #include #include #include #include #include #include -#include -#include #include "config.h" /// USE_DATASKETCHES @@ -27,6 +30,29 @@ enum StatisticsFileVersion : UInt16 V0 = 0, }; +std::optional StatisticsUtils::tryConvertToFloat64(const Field & value, const DataTypePtr & data_type) +{ + if (data_type->isValueRepresentedByNumber()) + { + Field value_converted; + + if (isInteger(data_type) && (value.getType() == Field::Types::Float64 || value.getType() == Field::Types::String)) + /// For case val_int32 < 10.5 or val_int32 < '10.5' we should convert 10.5 to Float64. + value_converted = convertFieldToType(value, *DataTypeFactory::instance().get("Float64")); + else + /// We should convert value to the real column data type and then translate it to Float64. + /// For example for expression col_date > '2024-08-07', if we directly convert '2024-08-07' to Float64, we will get null. + value_converted = convertFieldToType(value, *data_type); + + if (value_converted.isNull()) + return {}; + + Float64 value_as_float = applyVisitor(FieldVisitorConvertToNumber(), value_converted); + return value_as_float; + } + return {}; +} + IStatistics::IStatistics(const SingleStatisticsDescription & stat_) : stat(stat_) { diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 593ac20edb5..2a30c0de315 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -14,6 +14,12 @@ namespace DB constexpr auto STATS_FILE_PREFIX = "statistics_"; constexpr auto STATS_FILE_SUFFIX = ".stats"; +struct StatisticsUtils +{ + /// Returns std::nullopt if input Field cannot be converted to a concrete value + /// - `data_type` is the type of the column on which the statistics object was build on + static std::optional tryConvertToFloat64(const Field & value, const DataTypePtr & data_type); +}; /// Statistics describe properties of the values in the column, /// e.g. how many unique values exist, diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index fd9b922ffc8..285b779036f 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -1,8 +1,6 @@ #include -#include #include #include -#include namespace DB { @@ -41,22 +39,18 @@ void StatisticsTDigest::deserialize(ReadBuffer & buf) Float64 StatisticsTDigest::estimateLess(const Field & val) const { - Field val_converted = convertFieldToType(val, *data_type); - if (val_converted.isNull()) + auto val_as_float = StatisticsUtils::tryConvertToFloat64(val, data_type); + if (!val_as_float.has_value()) return 0; - - auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); - return t_digest.getCountLessThan(val_as_float); + return t_digest.getCountLessThan(*val_as_float); } Float64 StatisticsTDigest::estimateEqual(const Field & val) const { - Field val_converted = convertFieldToType(val, *data_type); - if (val_converted.isNull()) + auto val_as_float = StatisticsUtils::tryConvertToFloat64(val, data_type); + if (!val_as_float.has_value()) return 0; - - auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); - return t_digest.getCountEqual(val_as_float); + return t_digest.getCountEqual(*val_as_float); } void tdigestStatisticsValidator(const SingleStatisticsDescription & /*description*/, const DataTypePtr & data_type) diff --git a/tests/queries/0_stateless/02864_statistics_bugs.reference b/tests/queries/0_stateless/02864_statistics_bugs.reference index f599e28b8ab..a7eeae9def6 100644 --- a/tests/queries/0_stateless/02864_statistics_bugs.reference +++ b/tests/queries/0_stateless/02864_statistics_bugs.reference @@ -1 +1,3 @@ 10 +11 +0 diff --git a/tests/queries/0_stateless/02864_statistics_bugs.sql b/tests/queries/0_stateless/02864_statistics_bugs.sql index ef1735550e6..01bbe221b0f 100644 --- a/tests/queries/0_stateless/02864_statistics_bugs.sql +++ b/tests/queries/0_stateless/02864_statistics_bugs.sql @@ -7,3 +7,21 @@ CREATE TABLE bug_67742 (a Float64 STATISTICS(tdigest)) Engine = MergeTree() ORDE INSERT INTO bug_67742 SELECT number FROM system.numbers LIMIT 10000; SELECT count(*) FROM bug_67742 WHERE a < '10'; DROP TABLE bug_67742; + +DROP TABLE IF EXISTS bug_67742; +CREATE TABLE bug_67742 (a Int32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); +INSERT INTO bug_67742 SELECT number FROM system.numbers LIMIT 10000; +SELECT count(*) FROM bug_67742 WHERE a < '10.5'; -- { serverError TYPE_MISMATCH } +DROP TABLE bug_67742; + +DROP TABLE IF EXISTS bug_67742; +CREATE TABLE bug_67742 (a Int32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); +INSERT INTO bug_67742 SELECT number FROM system.numbers LIMIT 10000; +SELECT count(*) FROM bug_67742 WHERE a < 10.5; +DROP TABLE bug_67742; + +DROP TABLE IF EXISTS bug_67742; +CREATE TABLE bug_67742 (a Int16 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); +INSERT INTO bug_67742 SELECT number FROM system.numbers LIMIT 10000; +SELECT count(*) FROM bug_67742 WHERE a < '9999999999999999999999999'; +DROP TABLE bug_67742; From 3c531d314d0b0cfb64fe21fc7bc910ce3327cc79 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Aug 2024 10:09:35 +0000 Subject: [PATCH 377/644] Fix build --- src/Storages/Kafka/KafkaConfigLoader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Kafka/KafkaConfigLoader.cpp b/src/Storages/Kafka/KafkaConfigLoader.cpp index 000e08e2276..df6ccec4b7f 100644 --- a/src/Storages/Kafka/KafkaConfigLoader.cpp +++ b/src/Storages/Kafka/KafkaConfigLoader.cpp @@ -356,7 +356,7 @@ void updateGlobalConfiguration( } #else // USE_KRB5 if (kafka_config.has_property("sasl.kerberos.keytab") || kafka_config.has_property("sasl.kerberos.principal")) - LOG_WARNING(log, "Ignoring Kerberos-related parameters because ClickHouse was built without krb5 library support."); + LOG_WARNING(params.log, "Ignoring Kerberos-related parameters because ClickHouse was built without krb5 library support."); #endif // USE_KRB5 // No need to add any prefix, messages can be distinguished kafka_config.set_log_callback( From bcc5201c99e00998beab8088a988a66f921415b6 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 Jul 2024 17:17:07 +0200 Subject: [PATCH 378/644] init --- src/Common/SystemLogBase.cpp | 138 ++++++++++++-------- src/Common/SystemLogBase.h | 58 +++++--- src/Interpreters/InterpreterSystemQuery.cpp | 9 +- src/Interpreters/SystemLog.cpp | 38 +++--- 4 files changed, 148 insertions(+), 95 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 7d2c15714e2..748cf4744ae 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -27,12 +27,15 @@ #include #include +#define LOGICAL_IF_THEN(A, B) (!(A) || !!(B)) + namespace DB { namespace ErrorCodes { extern const int TIMEOUT_EXCEEDED; + extern const int ABORTED; } ISystemLog::~ISystemLog() = default; @@ -86,9 +89,8 @@ void SystemLogQueue::push(LogElement&& element) // by one, under exclusive lock, so we will see each message count. // It is enough to only wake the flushing thread once, after the message // count increases past half available size. - const uint64_t queue_end = queue_front_index + queue.size(); - requested_flush_up_to = std::max(requested_flush_up_to, queue_end); - + const auto last_log_index = queue_front_index + queue.size(); + requested_flush_index = std::max(requested_flush_index, last_log_index); flush_event.notify_all(); } @@ -127,20 +129,46 @@ template void SystemLogQueue::handleCrash() { if (settings.notify_flush_on_crash) - notifyFlush(/* force */ true); + { + notifyFlush(getLastLogIndex(), /* should_prepare_tables_anyway */ true); + } } template -void SystemLogQueue::waitFlush(uint64_t expected_flushed_up_to) +void SystemLogQueue::notifyFlush(SystemLogQueue::Index expected_flushed_index, bool should_prepare_tables_anyway) { + std::unique_lock lock(mutex); + // Publish our flush request, taking care not to overwrite the requests + // made by other threads. + force_prepare_tables_requested |= should_prepare_tables_anyway; + requested_flush_index = std::max(requested_flush_index, expected_flushed_index); + flush_event.notify_all(); +} + +template +void SystemLogQueue::waitFlush(SystemLogQueue::Index expected_flushed_index, bool should_prepare_tables_anyway) +{ + LOG_DEBUG(log, "Requested flush up to offset {}", expected_flushed_index); + // Use an arbitrary timeout to avoid endless waiting. 60s proved to be // too fast for our parallel functional tests, probably because they // heavily load the disk. const int timeout_seconds = 180; + std::unique_lock lock(mutex); - bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] + + // there is no obligation to call notifyFlush before waitFlush, than we have to be sure that flush_event has been triggered + force_prepare_tables_requested |= should_prepare_tables_anyway; + if (requested_flush_index < expected_flushed_index) { - return flushed_up_to >= expected_flushed_up_to && !is_force_prepare_tables; + requested_flush_index = expected_flushed_index; + flush_event.notify_all(); + } + + auto result = confirm_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] + { + const bool if_should_prepare_then_it_is_done = LOGICAL_IF_THEN(should_prepare_tables_anyway, prepare_tables_done); + return (flushed_index >= expected_flushed_index && if_should_prepare_then_it_is_done) || is_shutdown; }); if (!result) @@ -148,67 +176,54 @@ void SystemLogQueue::waitFlush(uint64_t expected_flushed_up_to) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Timeout exceeded ({} s) while flushing system log '{}'.", toString(timeout_seconds), demangle(typeid(*this).name())); } -} - -template -uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyway) -{ - uint64_t this_thread_requested_offset; + if (is_shutdown) { - std::lock_guard lock(mutex); - if (is_shutdown) - return uint64_t(-1); - - this_thread_requested_offset = queue_front_index + queue.size(); - - // Publish our flush request, taking care not to overwrite the requests - // made by other threads. - is_force_prepare_tables |= should_prepare_tables_anyway; - requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); - - flush_event.notify_all(); + throw Exception(ErrorCodes::ABORTED, "Shutdown has been called while flushing system log '{}'. Aborting.", + demangle(typeid(*this).name())); } - - LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); - return this_thread_requested_offset; } template -void SystemLogQueue::confirm(uint64_t to_flush_end) +SystemLogQueue::Index SystemLogQueue::getLastLogIndex() { std::lock_guard lock(mutex); - flushed_up_to = to_flush_end; - is_force_prepare_tables = false; - flush_event.notify_all(); + return queue_front_index + queue.size(); } template -typename SystemLogQueue::Index SystemLogQueue::pop(std::vector & output, - bool & should_prepare_tables_anyway, - bool & exit_this_thread) +void SystemLogQueue::confirm(SystemLogQueue::Index last_flashed_index) { - /// Call dtors and deallocate strings without holding the global lock - output.resize(0); + std::lock_guard lock(mutex); + prepare_tables_done = true; + flushed_index = std::max(flushed_index, last_flashed_index); + confirm_event.notify_all(); +} +template +typename SystemLogQueue::PopResult SystemLogQueue::pop() +{ std::unique_lock lock(mutex); - flush_event.wait_for(lock, - std::chrono::milliseconds(settings.flush_interval_milliseconds), - [&] () - { - return requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables; - } - ); + + flush_event.wait_for(lock, std::chrono::milliseconds(settings.flush_interval_milliseconds), [&] () + { + const bool if_prepare_requested_and_it_is_not_done = force_prepare_tables_requested && !prepare_tables_done; + return requested_flush_index > flushed_index || if_prepare_requested_and_it_is_not_done || is_shutdown; + }); + + if (is_shutdown) + return PopResult{.is_shutdown = true}; queue_front_index += queue.size(); - // Swap with existing array from previous flush, to save memory - // allocations. - queue.swap(output); - should_prepare_tables_anyway = is_force_prepare_tables; + PopResult result; + result.logs_index = queue_front_index; + result.logs_elemets.swap(queue); - exit_this_thread = is_shutdown; - return queue_front_index; + const bool if_prepare_requested_and_it_is_not_done = force_prepare_tables_requested && !prepare_tables_done; + result.create_table_force = if_prepare_requested_and_it_is_not_done; + + return result; } template @@ -229,13 +244,21 @@ SystemLogBase::SystemLogBase( } template -void SystemLogBase::flush(bool force) +SystemLogBase::Index SystemLogBase::getLastLogIndex() { - uint64_t this_thread_requested_offset = queue->notifyFlush(force); - if (this_thread_requested_offset == uint64_t(-1)) - return; + return queue->getLastLogIndex(); +} - queue->waitFlush(this_thread_requested_offset); +template +void SystemLogBase::notifyFlush(Index expected_flushed_index) +{ + queue->notifyFlush(expected_flushed_index, /* should_prepare_tables_anyway */ true); +} + +template +void SystemLogBase::flush(Index expected_flushed_index) +{ + queue->waitFlush(expected_flushed_index, /* should_prepare_tables_anyway */ true); } template @@ -257,9 +280,6 @@ void SystemLogBase::add(LogElement element) queue->push(std::move(element)); } -template -void SystemLogBase::notifyFlush(bool force) { queue->notifyFlush(force); } - #define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogBase; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) @@ -267,3 +287,5 @@ SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_QUEUE) } + +#undef LOGICAL_IF_THEN diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index b87fcf419d3..6c60ffafc4c 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -54,10 +54,19 @@ struct StorageID; class ISystemLog { public: + using Index = uint64_t; + virtual String getName() const = 0; - //// force -- force table creation (used for SYSTEM FLUSH LOGS) - virtual void flush(bool force = false) = 0; /// NOLINT + /// Return the index of the lastest added log element. That index no less than the flashed index. + /// The flashed index is the index of the last log element which has been flushed successfully. + /// Thereby all the records whose index is less than the flashed index are flushed already. + virtual Index getLastLogIndex() = 0; + /// Call this method to wake up the flush thread and flush the data in the background. It is non blocking call + virtual void notifyFlush(Index expected_flushed_index) = 0; + /// Call this method to wait intill the logs are flushed up to expected_flushed_index. It is blocking call. + virtual void flush(Index expected_flushed_index) = 0; + virtual void prepareTable() = 0; /// Start the background thread. @@ -97,24 +106,34 @@ struct SystemLogQueueSettings template class SystemLogQueue { - using Index = uint64_t; - public: + using Index = ISystemLog::Index; + explicit SystemLogQueue(const SystemLogQueueSettings & settings_); void shutdown(); // producer methods void push(LogElement && element); - Index notifyFlush(bool should_prepare_tables_anyway); - void waitFlush(Index expected_flushed_up_to); + + Index getLastLogIndex(); + void notifyFlush(Index expected_flushed_index, bool should_prepare_tables_anyway); + void waitFlush(Index expected_flushed_index, bool should_prepare_tables_anyway); /// Handles crash, flushes log without blocking if notify_flush_on_crash is set void handleCrash(); + struct PopResult + { + Index logs_index = 0; + std::vector logs_elemets = {}; + bool create_table_force = false; + bool is_shutdown = false; + }; + // consumer methods - Index pop(std::vector& output, bool & should_prepare_tables_anyway, bool & exit_this_thread); - void confirm(Index to_flush_end); + PopResult pop(); + void confirm(Index last_flashed_index); private: /// Data shared between callers of add()/flush()/shutdown(), and the saving thread @@ -124,22 +143,29 @@ private: // Queue is bounded. But its size is quite large to not block in all normal cases. std::vector queue; + // An always-incrementing index of the first message currently in the queue. // We use it to give a global sequential index to every message, so that we // can wait until a particular message is flushed. This is used to implement // synchronous log flushing for SYSTEM FLUSH LOGS. Index queue_front_index = 0; + // A flag that says we must create the tables even if the queue is empty. - bool is_force_prepare_tables = false; + bool force_prepare_tables_requested = false; + bool prepare_tables_done = false; + // Requested to flush logs up to this index, exclusive - Index requested_flush_up_to = 0; + Index requested_flush_index = 0; + // Flushed log up to this index, exclusive - Index flushed_up_to = 0; + Index flushed_index = 0; + // Logged overflow message at this queue front index Index logged_queue_full_at_index = -1; bool is_shutdown = false; + std::condition_variable confirm_event; std::condition_variable flush_event; const SystemLogQueueSettings settings; @@ -150,6 +176,7 @@ template class SystemLogBase : public ISystemLog { public: + using Index = ISystemLog::Index; using Self = SystemLogBase; explicit SystemLogBase( @@ -163,15 +190,16 @@ public: */ void add(LogElement element); + Index getLastLogIndex() override; + + void notifyFlush(Index expected_flushed_index) override; + /// Flush data in the buffer to disk. Block the thread until the data is stored on disk. - void flush(bool force) override; + void flush(Index expected_flushed_index) override; /// Handles crash, flushes log without blocking if notify_flush_on_crash is set void handleCrash() override; - /// Non-blocking flush data in the buffer to disk. - void notifyFlush(bool force); - String getName() const override { return LogElement::name(); } static const char * getDefaultOrderBy() { return "event_date, event_time"; } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index c284acfa308..9b483bac25c 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -712,11 +712,18 @@ BlockIO InterpreterSystemQuery::execute() getContext()->checkAccess(AccessType::SYSTEM_FLUSH_LOGS); auto logs = getContext()->getSystemLogs(); + std::vector> commands; commands.reserve(logs.size()); for (auto * system_log : logs) - commands.emplace_back([system_log] { system_log->flush(true); }); + { + auto current_index = system_log->getLastLogIndex(); + /// The data is started to being flushed in the background after notifyFlush call + system_log->notifyFlush(current_index); + commands.emplace_back([system_log, current_index] { system_log->flush(current_index); }); + } + /// The data is flashing in the background, we need to wait until it is done executeCommandsAndThrowIfError(commands); break; } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 572481e6b12..7042564799a 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -1,6 +1,7 @@ #include #include +#include "Common/SystemLogBase.h" #include #include #include @@ -462,33 +463,26 @@ void SystemLog::savingThreadFunction() { setThreadName("SystemLogFlush"); - std::vector to_flush; - bool exit_this_thread = false; - while (!exit_this_thread) + while (true) { try { - // The end index (exclusive, like std end()) of the messages we are - // going to flush. - uint64_t to_flush_end = 0; - // Should we prepare table even if there are no new messages. - bool should_prepare_tables_anyway = false; + auto result = queue->pop(); - to_flush_end = queue->pop(to_flush, should_prepare_tables_anyway, exit_this_thread); - - if (to_flush.empty()) + if (result.is_shutdown) { - if (should_prepare_tables_anyway) - { - prepareTable(); - LOG_TRACE(log, "Table created (force)"); - - queue->confirm(to_flush_end); - } + LOG_TRACE(log, "Terminating"); + return; } - else + + if (!result.logs_elemets.empty()) { - flushImpl(to_flush, to_flush_end); + flushImpl(result.logs_elemets, result.logs_index); + } + else if (result.create_table_force) + { + prepareTable(); + queue->confirm(/* last_flashed_index */ 0); } } catch (...) @@ -496,7 +490,6 @@ void SystemLog::savingThreadFunction() tryLogCurrentException(__PRETTY_FUNCTION__); } } - LOG_TRACE(log, "Terminating"); } @@ -579,6 +572,9 @@ StoragePtr SystemLog::getStorage() const template void SystemLog::prepareTable() { + if (is_prepared) + return; + String description = table_id.getNameForLogs(); auto table = getStorage(); From aa42ccf0531aa416c5525b6d8c01c057f781b0e3 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 Jul 2024 19:04:22 +0200 Subject: [PATCH 379/644] move LOGICAL_IF_THEN to base/defines.h --- base/base/defines.h | 2 ++ src/Common/SystemLogBase.cpp | 3 --- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/base/base/defines.h b/base/base/defines.h index 5685a6d9833..7860cebd359 100644 --- a/base/base/defines.h +++ b/base/base/defines.h @@ -164,3 +164,5 @@ template constexpr void UNUSED(Args &&... args [[maybe_unused]]) // NOLINT(cppcoreguidelines-missing-std-forward) { } + +#define LOGICAL_IF_THEN(A, B) (!(A) || !!(B)) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 748cf4744ae..6840c461ce6 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -27,7 +27,6 @@ #include #include -#define LOGICAL_IF_THEN(A, B) (!(A) || !!(B)) namespace DB { @@ -287,5 +286,3 @@ SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_QUEUE) } - -#undef LOGICAL_IF_THEN From abd5dfe1d0c7b88212a2f0dc4ed1a8b470dcedb1 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 Jul 2024 19:12:13 +0200 Subject: [PATCH 380/644] fix typo --- src/Common/SystemLogBase.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 6c60ffafc4c..3915b99f8aa 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -58,7 +58,7 @@ public: virtual String getName() const = 0; - /// Return the index of the lastest added log element. That index no less than the flashed index. + /// Return the index of the latest added log element. That index no less than the flashed index. /// The flashed index is the index of the last log element which has been flushed successfully. /// Thereby all the records whose index is less than the flashed index are flushed already. virtual Index getLastLogIndex() = 0; From 8e5577ad8f8cb4e4d3ccb977af8536957088b8ca Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 Jul 2024 19:18:03 +0200 Subject: [PATCH 381/644] fix includes --- src/Interpreters/SystemLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 7042564799a..c236b524c60 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -1,7 +1,7 @@ #include #include -#include "Common/SystemLogBase.h" +#include #include #include #include From 86267418f9a74915d5089770d658b328684b9189 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 31 Jul 2024 23:28:03 +0200 Subject: [PATCH 382/644] fix tests, rework recreation tables conditions, add log about ignored logs --- base/base/defines.h | 2 - src/Backups/BackupsWorker.cpp | 2 + src/Common/SystemLogBase.cpp | 104 +++++++++--------- src/Common/SystemLogBase.h | 28 +++-- src/Interpreters/SystemLog.cpp | 9 +- .../test_system_flush_logs/test.py | 25 ++++- .../test_system_logs_recreate/test.py | 13 ++- 7 files changed, 105 insertions(+), 78 deletions(-) diff --git a/base/base/defines.h b/base/base/defines.h index 7860cebd359..5685a6d9833 100644 --- a/base/base/defines.h +++ b/base/base/defines.h @@ -164,5 +164,3 @@ template constexpr void UNUSED(Args &&... args [[maybe_unused]]) // NOLINT(cppcoreguidelines-missing-std-forward) { } - -#define LOGICAL_IF_THEN(A, B) (!(A) || !!(B)) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 0b93ae6d547..363aaae9c8d 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -1171,6 +1171,8 @@ void BackupsWorker::waitAll() for (const auto & id : current_operations) wait(id, /* rethrow_exception= */ false); + backup_log->flush(backup_log->getLastLogIndex()); + LOG_INFO(log, "Backups and restores finished"); } diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 6840c461ce6..a35a46c49cc 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -88,31 +88,18 @@ void SystemLogQueue::push(LogElement&& element) // by one, under exclusive lock, so we will see each message count. // It is enough to only wake the flushing thread once, after the message // count increases past half available size. + const auto last_log_index = queue_front_index + queue.size(); - requested_flush_index = std::max(requested_flush_index, last_log_index); - flush_event.notify_all(); + notifyFlushUnlocked(last_log_index, /* should_prepare_tables_anyway */ false); } if (queue.size() >= settings.max_size_rows) { + chassert(queue.size() == settings.max_size_rows); + // Ignore all further entries until the queue is flushed. - // Log a message about that. Don't spam it -- this might be especially - // problematic in case of trace log. Remember what the front index of the - // queue was when we last logged the message. If it changed, it means the - // queue was flushed, and we can log again. - if (queue_front_index != logged_queue_full_at_index) - { - logged_queue_full_at_index = queue_front_index; - - // TextLog sets its logger level to 0, so this log is a noop and - // there is no recursive logging. - lock.unlock(); - LOG_ERROR(log, "Queue is full for system log '{}' at {}. max_size_rows {}", - demangle(typeid(*this).name()), - queue_front_index, - settings.max_size_rows); - } - + // To the next batch we add a log message about how much we have lost + ++ignored_logs; return; } @@ -133,15 +120,22 @@ void SystemLogQueue::handleCrash() } } +template +void SystemLogQueue::notifyFlushUnlocked(Index expected_flushed_index, bool should_prepare_tables_anyway) +{ + if (should_prepare_tables_anyway) + requested_prepare_tables = std::max(requested_prepare_tables, expected_flushed_index); + + requested_flush_index = std::max(requested_flush_index, expected_flushed_index); + + flush_event.notify_all(); +} + template void SystemLogQueue::notifyFlush(SystemLogQueue::Index expected_flushed_index, bool should_prepare_tables_anyway) { - std::unique_lock lock(mutex); - // Publish our flush request, taking care not to overwrite the requests - // made by other threads. - force_prepare_tables_requested |= should_prepare_tables_anyway; - requested_flush_index = std::max(requested_flush_index, expected_flushed_index); - flush_event.notify_all(); + std::lock_guard lock(mutex); + notifyFlushUnlocked(expected_flushed_index, should_prepare_tables_anyway); } template @@ -156,18 +150,15 @@ void SystemLogQueue::waitFlush(SystemLogQueue::Index exp std::unique_lock lock(mutex); - // there is no obligation to call notifyFlush before waitFlush, than we have to be sure that flush_event has been triggered - force_prepare_tables_requested |= should_prepare_tables_anyway; - if (requested_flush_index < expected_flushed_index) - { - requested_flush_index = expected_flushed_index; - flush_event.notify_all(); - } + // there is no obligation to call notifyFlush before waitFlush, than we have to be sure that flush_event has been triggered before we wait the result + notifyFlushUnlocked(expected_flushed_index, should_prepare_tables_anyway); auto result = confirm_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] { - const bool if_should_prepare_then_it_is_done = LOGICAL_IF_THEN(should_prepare_tables_anyway, prepare_tables_done); - return (flushed_index >= expected_flushed_index && if_should_prepare_then_it_is_done) || is_shutdown; + if (should_prepare_tables_anyway) + return (flushed_index >= expected_flushed_index && prepared_tables >= requested_prepare_tables) || is_shutdown; + else + return (flushed_index >= expected_flushed_index) || is_shutdown; }); if (!result) @@ -194,7 +185,7 @@ template void SystemLogQueue::confirm(SystemLogQueue::Index last_flashed_index) { std::lock_guard lock(mutex); - prepare_tables_done = true; + prepared_tables = std::max(prepared_tables, last_flashed_index); flushed_index = std::max(flushed_index, last_flashed_index); confirm_event.notify_all(); } @@ -202,25 +193,34 @@ void SystemLogQueue::confirm(SystemLogQueue::Index last_ template typename SystemLogQueue::PopResult SystemLogQueue::pop() { - std::unique_lock lock(mutex); - - flush_event.wait_for(lock, std::chrono::milliseconds(settings.flush_interval_milliseconds), [&] () - { - const bool if_prepare_requested_and_it_is_not_done = force_prepare_tables_requested && !prepare_tables_done; - return requested_flush_index > flushed_index || if_prepare_requested_and_it_is_not_done || is_shutdown; - }); - - if (is_shutdown) - return PopResult{.is_shutdown = true}; - - queue_front_index += queue.size(); - PopResult result; - result.logs_index = queue_front_index; - result.logs_elemets.swap(queue); + size_t prev_ignored_logs = 0; - const bool if_prepare_requested_and_it_is_not_done = force_prepare_tables_requested && !prepare_tables_done; - result.create_table_force = if_prepare_requested_and_it_is_not_done; + { + std::unique_lock lock(mutex); + + flush_event.wait_for(lock, std::chrono::milliseconds(settings.flush_interval_milliseconds), [&] () + { + return requested_flush_index > flushed_index || requested_prepare_tables > prepared_tables || is_shutdown; + }); + + if (is_shutdown) + return PopResult{.is_shutdown = true}; + + queue_front_index += queue.size(); + prev_ignored_logs = ignored_logs; + ignored_logs = 0; + + result.last_log_index = queue_front_index; + result.logs.swap(queue); + result.create_table_force = requested_prepare_tables > prepared_tables; + } + + if (prev_ignored_logs) + LOG_ERROR(log, "Queue had been full at {}, accepted {} logs, ignored {} logs.", + result.last_log_index - result.logs.size(), + result.logs.size(), + prev_ignored_logs); return result; } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 3915b99f8aa..c359287a73f 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -54,7 +55,7 @@ struct StorageID; class ISystemLog { public: - using Index = uint64_t; + using Index = int64_t; virtual String getName() const = 0; @@ -125,8 +126,8 @@ public: struct PopResult { - Index logs_index = 0; - std::vector logs_elemets = {}; + Index last_log_index = 0; + std::vector logs = {}; bool create_table_force = false; bool is_shutdown = false; }; @@ -136,6 +137,8 @@ public: void confirm(Index last_flashed_index); private: + void notifyFlushUnlocked(Index expected_flushed_index, bool should_prepare_tables_anyway); + /// Data shared between callers of add()/flush()/shutdown(), and the saving thread std::mutex mutex; @@ -150,18 +153,21 @@ private: // synchronous log flushing for SYSTEM FLUSH LOGS. Index queue_front_index = 0; - // A flag that says we must create the tables even if the queue is empty. - bool force_prepare_tables_requested = false; - bool prepare_tables_done = false; - // Requested to flush logs up to this index, exclusive - Index requested_flush_index = 0; - + Index requested_flush_index = std::numeric_limits::min(); // Flushed log up to this index, exclusive Index flushed_index = 0; - // Logged overflow message at this queue front index - Index logged_queue_full_at_index = -1; + // The same logic for the prepare tables: if requested_prepar_tables > prepared_tables we need to do prepare + // except that initial prepared_tables is -1 + // it is due to the difference: when no logs have been written and we call flush logs + // it becomes in the state: requested_flush_index = 0 and flushed_index = 0 -- we do not want to do anything + // but if we need to prepare tables it becomes requested_prepare_tables = 0 and prepared_tables = -1 + // we trigger background thread and do prepare + Index requested_prepare_tables = std::numeric_limits::min(); + Index prepared_tables = -1; + + size_t ignored_logs = 0; bool is_shutdown = false; diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index c236b524c60..9d07184a0e5 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -475,14 +475,14 @@ void SystemLog::savingThreadFunction() return; } - if (!result.logs_elemets.empty()) + if (!result.logs.empty()) { - flushImpl(result.logs_elemets, result.logs_index); + flushImpl(result.logs, result.last_log_index); } else if (result.create_table_force) { prepareTable(); - queue->confirm(/* last_flashed_index */ 0); + queue->confirm(result.last_log_index); } } catch (...) @@ -572,9 +572,6 @@ StoragePtr SystemLog::getStorage() const template void SystemLog::prepareTable() { - if (is_prepared) - return; - String description = table_id.getNameForLogs(); auto table = getStorage(); diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index 713b327eb76..0399122406a 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -4,7 +4,7 @@ import pytest from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry +from helpers.test_tools import assert_eq_with_retry, assert_logs_contain_with_retry, TSV cluster = ClickHouseCluster(__file__) node = cluster.add_instance( @@ -75,6 +75,8 @@ def test_system_suspend(): def test_log_max_size(start_cluster): + # we do misconfiguration here: buffer_size_rows_flush_threshold > max_size_rows, flush_interval_milliseconds is huge + # no auto flush by size not by time has a chance node.exec_in_container( [ "bash", @@ -83,6 +85,7 @@ def test_log_max_size(start_cluster): 1000000 + 1000000 10 10 @@ -91,11 +94,23 @@ def test_log_max_size(start_cluster): """, ] ) - node.restart_clickhouse() - for i in range(10): - node.query(f"select {i}") - assert node.query("select count() >= 10 from system.query_log") == "1\n" + node.query(f"TRUNCATE TABLE IF EXISTS system.query_log") + node.restart_clickhouse() + + # all logs records above max_size_rows are lost + # The accepted logs records are never flushed until system flush logs is called by us + for i in range(21): + node.query(f"select {i}") + node.query("system flush logs") + + assert_logs_contain_with_retry( + node, "Queue had been full at 0, accepted 10 logs, ignored 34 logs." + ) + assert node.query( + "select count() >= 10, count() < 20 from system.query_log" + ) == TSV([[1, 1]]) + node.exec_in_container( ["rm", f"/etc/clickhouse-server/config.d/yyy-override-query_log.xml"] ) diff --git a/tests/integration/test_system_logs_recreate/test.py b/tests/integration/test_system_logs_recreate/test.py index 1bdb1fe3261..1a4ed31278d 100644 --- a/tests/integration/test_system_logs_recreate/test.py +++ b/tests/integration/test_system_logs_recreate/test.py @@ -173,11 +173,20 @@ def test_drop_system_log(): node.query("system flush logs") node.query("select 2") node.query("system flush logs") - assert node.query("select count() > 0 from system.query_log") == "1\n" + assert node.query("select count() >= 2 from system.query_log") == "1\n" + node.query("drop table system.query_log sync") node.query("select 3") node.query("system flush logs") - assert node.query("select count() > 0 from system.query_log") == "1\n" + assert node.query("select count() >= 1 from system.query_log") == "1\n" + + node.query("drop table system.query_log sync") + node.restart_clickhouse() + node.query("system flush logs") + assert ( + node.query("select count() >= 0 from system.query_log") == "1\n" + ) # we check that query_log just exists + node.exec_in_container( ["rm", f"/etc/clickhouse-server/config.d/yyy-override-query_log.xml"] ) From 00efd5fc0da7124cb1671487250476185101a488 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 31 Jul 2024 22:23:16 +0000 Subject: [PATCH 383/644] Automatic style fix --- tests/integration/test_system_logs_recreate/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_system_logs_recreate/test.py b/tests/integration/test_system_logs_recreate/test.py index 1a4ed31278d..d5347ae7dea 100644 --- a/tests/integration/test_system_logs_recreate/test.py +++ b/tests/integration/test_system_logs_recreate/test.py @@ -185,7 +185,7 @@ def test_drop_system_log(): node.query("system flush logs") assert ( node.query("select count() >= 0 from system.query_log") == "1\n" - ) # we check that query_log just exists + ) # we check that query_log just exists node.exec_in_container( ["rm", f"/etc/clickhouse-server/config.d/yyy-override-query_log.xml"] From e3290c782066aaf7d1891d865547706387bf773c Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 1 Aug 2024 16:58:17 +0200 Subject: [PATCH 384/644] rework Context::getSystemLogs, add system logs flush at shutdown --- src/Backups/BackupsWorker.cpp | 4 +- src/Common/SystemLogBase.cpp | 8 +- src/Common/SystemLogBase.h | 8 +- src/Interpreters/Context.cpp | 7 +- src/Interpreters/Context.h | 4 +- src/Interpreters/InterpreterSystemQuery.cpp | 17 +--- src/Interpreters/SystemLog.cpp | 86 ++++++++-------- src/Interpreters/SystemLog.h | 104 +++++++++----------- 8 files changed, 109 insertions(+), 129 deletions(-) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 363aaae9c8d..106aa89082d 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -1171,7 +1171,7 @@ void BackupsWorker::waitAll() for (const auto & id : current_operations) wait(id, /* rethrow_exception= */ false); - backup_log->flush(backup_log->getLastLogIndex()); + backup_log->flush(backup_log->getLastLogIndex(), /* should_prepare_tables_anyway */ false); LOG_INFO(log, "Backups and restores finished"); } @@ -1223,6 +1223,8 @@ void BackupsWorker::cancelAll(bool wait_) for (const auto & id : current_operations) wait(id, /* rethrow_exception= */ false); + backup_log->flush(backup_log->getLastLogIndex(), /* should_prepare_tables_anyway */ false); + LOG_INFO(log, "Backups and restores finished or stopped"); } diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index a35a46c49cc..127c8862a35 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -249,15 +249,15 @@ SystemLogBase::Index SystemLogBase::getLastLogIndex() } template -void SystemLogBase::notifyFlush(Index expected_flushed_index) +void SystemLogBase::notifyFlush(Index expected_flushed_index, bool should_prepare_tables_anyway) { - queue->notifyFlush(expected_flushed_index, /* should_prepare_tables_anyway */ true); + queue->notifyFlush(expected_flushed_index, should_prepare_tables_anyway); } template -void SystemLogBase::flush(Index expected_flushed_index) +void SystemLogBase::flush(Index expected_flushed_index, bool should_prepare_tables_anyway) { - queue->waitFlush(expected_flushed_index, /* should_prepare_tables_anyway */ true); + queue->waitFlush(expected_flushed_index, should_prepare_tables_anyway); } template diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index c359287a73f..0d7b04d5c57 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -64,9 +64,9 @@ public: /// Thereby all the records whose index is less than the flashed index are flushed already. virtual Index getLastLogIndex() = 0; /// Call this method to wake up the flush thread and flush the data in the background. It is non blocking call - virtual void notifyFlush(Index expected_flushed_index) = 0; + virtual void notifyFlush(Index expected_flushed_index, bool should_prepare_tables_anyway) = 0; /// Call this method to wait intill the logs are flushed up to expected_flushed_index. It is blocking call. - virtual void flush(Index expected_flushed_index) = 0; + virtual void flush(Index expected_flushed_index, bool should_prepare_tables_anyway) = 0; virtual void prepareTable() = 0; @@ -198,10 +198,10 @@ public: Index getLastLogIndex() override; - void notifyFlush(Index expected_flushed_index) override; + void notifyFlush(Index expected_flushed_index, bool should_prepare_tables_anyway) override; /// Flush data in the buffer to disk. Block the thread until the data is stored on disk. - void flush(Index expected_flushed_index) override; + void flush(Index expected_flushed_index, bool should_prepare_tables_anyway) override; /// Handles crash, flushes log without blocking if notify_flush_on_crash is set void handleCrash() override; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5413b568068..3051ed3e567 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -99,6 +99,7 @@ #include #include #include +#include #include #include #include @@ -618,7 +619,7 @@ struct ContextSharedPart : boost::noncopyable /** After system_logs have been shut down it is guaranteed that no system table gets created or written to. * Note that part changes at shutdown won't be logged to part log. */ - SHUTDOWN(log, "system logs", system_logs, shutdown()); + SHUTDOWN(log, "system logs", system_logs, flushAndShutdown()); LOG_TRACE(log, "Shutting down database catalog"); DatabaseCatalog::shutdown(); @@ -4312,13 +4313,13 @@ std::shared_ptr Context::getBlobStorageLog() const return shared->system_logs->blob_storage_log; } -std::vector Context::getSystemLogs() const +SystemLogs Context::getSystemLogs() const { SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; - return shared->system_logs->logs; + return *shared->system_logs; } std::optional Context::getDashboards() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index d5e35c3e4b3..3da4f124553 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -48,6 +48,8 @@ namespace DB class ASTSelectQuery; +class SystemLogs; + struct ContextSharedPart; class ContextAccess; class ContextAccessWrapper; @@ -1150,7 +1152,7 @@ public: std::shared_ptr getBackupLog() const; std::shared_ptr getBlobStorageLog() const; - std::vector getSystemLogs() const; + SystemLogs getSystemLogs() const; using Dashboards = std::vector>; std::optional getDashboards() const; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 9b483bac25c..ef6d1040c5e 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -710,21 +710,8 @@ BlockIO InterpreterSystemQuery::execute() case Type::FLUSH_LOGS: { getContext()->checkAccess(AccessType::SYSTEM_FLUSH_LOGS); - - auto logs = getContext()->getSystemLogs(); - - std::vector> commands; - commands.reserve(logs.size()); - for (auto * system_log : logs) - { - auto current_index = system_log->getLastLogIndex(); - /// The data is started to being flushed in the background after notifyFlush call - system_log->notifyFlush(current_index); - commands.emplace_back([system_log, current_index] { system_log->flush(current_index); }); - } - - /// The data is flashing in the background, we need to wait until it is done - executeCommandsAndThrowIfError(commands); + auto system_logs = getContext()->getSystemLogs(); + system_logs.flush(true); break; } case Type::STOP_LISTEN: diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 9d07184a0e5..9b58da3f545 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -50,6 +50,7 @@ #include + namespace DB { @@ -312,56 +313,13 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf azure_queue_log = createSystemLog(global_context, "system", "azure_queue_log", config, "azure_queue_log", "Contains logging entries with the information files processes by S3Queue engine."); blob_storage_log = createSystemLog(global_context, "system", "blob_storage_log", config, "blob_storage_log", "Contains logging entries with information about various blob storage operations such as uploads and deletes."); - if (query_log) - logs.emplace_back(query_log.get()); - if (query_thread_log) - logs.emplace_back(query_thread_log.get()); - if (part_log) - logs.emplace_back(part_log.get()); - if (trace_log) - logs.emplace_back(trace_log.get()); - if (crash_log) - logs.emplace_back(crash_log.get()); - if (text_log) - logs.emplace_back(text_log.get()); - if (metric_log) - logs.emplace_back(metric_log.get()); - if (error_log) - logs.emplace_back(error_log.get()); - if (asynchronous_metric_log) - logs.emplace_back(asynchronous_metric_log.get()); - if (opentelemetry_span_log) - logs.emplace_back(opentelemetry_span_log.get()); - if (query_views_log) - logs.emplace_back(query_views_log.get()); - if (zookeeper_log) - logs.emplace_back(zookeeper_log.get()); if (session_log) - { - logs.emplace_back(session_log.get()); global_context->addWarningMessage("Table system.session_log is enabled. It's unreliable and may contain garbage. Do not use it for any kind of security monitoring."); - } - if (transactions_info_log) - logs.emplace_back(transactions_info_log.get()); - if (processors_profile_log) - logs.emplace_back(processors_profile_log.get()); - if (filesystem_cache_log) - logs.emplace_back(filesystem_cache_log.get()); - if (filesystem_read_prefetches_log) - logs.emplace_back(filesystem_read_prefetches_log.get()); - if (asynchronous_insert_log) - logs.emplace_back(asynchronous_insert_log.get()); - if (backup_log) - logs.emplace_back(backup_log.get()); - if (s3_queue_log) - logs.emplace_back(s3_queue_log.get()); - if (blob_storage_log) - logs.emplace_back(blob_storage_log.get()); bool should_prepare = global_context->getServerSettings().prepare_system_log_tables_on_startup; try { - for (auto & log : logs) + for (auto & log : getAllLogs()) { log->startup(); if (should_prepare) @@ -395,20 +353,56 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf } } - -SystemLogs::~SystemLogs() +std::vector SystemLogs::getAllLogs() const { +/// NOLINTBEGIN(bugprone-macro-parentheses) +#define GET_RAW_POINTERS(log_type, member, descr) \ + member.get(), \ + + std::vector result = { + LIST_OF_ALL_SYSTEM_LOGS(GET_RAW_POINTERS) + }; +#undef GET_RAW_POINTERS +/// NOLINTEND(bugprone-macro-parentheses) + + auto last_it = std::remove(result.begin(), result.end(), nullptr); + result.erase(last_it, result.end()); + + return result; +} + +void SystemLogs::flush(bool should_prepare_tables_anyway) +{ + auto logs = getAllLogs(); + std::vector logs_indexes(logs.size(), 0); + + for (size_t i = 0; i < logs.size(); ++i) + { + auto last_log_index = logs[i]->getLastLogIndex(); + logs_indexes[i] = last_log_index; + logs[i]->notifyFlush(last_log_index, should_prepare_tables_anyway); + } + + for (size_t i = 0; i < logs.size(); ++i) + logs[i]->flush(logs_indexes[i], should_prepare_tables_anyway); +} + +void SystemLogs::flushAndShutdown() +{ + flush(/* should_prepare_tables_anyway */ false); shutdown(); } void SystemLogs::shutdown() { + auto logs = getAllLogs(); for (auto & log : logs) log->shutdown(); } void SystemLogs::handleCrash() { + auto logs = getAllLogs(); for (auto & log : logs) log->handleCrash(); } diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 0ac468b15ec..093be203282 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -5,6 +5,32 @@ #include #include +#include + +#define LIST_OF_ALL_SYSTEM_LOGS(M) \ + M(QueryLog, query_log, "Used to log queries.") \ + M(QueryThreadLog, query_thread_log, "Used to log query threads.") \ + M(PartLog, part_log, "Used to log operations with parts.") \ + M(TraceLog, trace_log, "Used to log traces from query profiler.") \ + M(CrashLog, crash_log, "Used to log server crashes.") \ + M(TextLog, text_log, "Used to log all text messages.") \ + M(MetricLog, metric_log, "Used to log all metrics.") \ + M(ErrorLog, error_log, "Used to log errors.") \ + M(FilesystemCacheLog, filesystem_cache_log, "") \ + M(FilesystemReadPrefetchesLog, filesystem_read_prefetches_log, "") \ + M(ObjectStorageQueueLog, s3_queue_log, "") \ + M(ObjectStorageQueueLog, azure_queue_log, "") \ + M(AsynchronousMetricLog, asynchronous_metric_log, "Metrics from system.asynchronous_metrics") \ + M(OpenTelemetrySpanLog, opentelemetry_span_log, "OpenTelemetry trace spans.") \ + M(QueryViewsLog, query_views_log, "Used to log queries of materialized and live views.") \ + M(ZooKeeperLog, zookeeper_log, "Used to log all actions of ZooKeeper client.") \ + M(SessionLog, session_log, "Login, LogOut and Login failure events.") \ + M(TransactionsInfoLog, transactions_info_log, "Events related to transactions.") \ + M(ProcessorsProfileLog, processors_profile_log, "Used to log processors profiling") \ + M(AsynchronousInsertLog, asynchronous_insert_log, "") \ + M(BackupLog, backup_log, "Backup and restore events") \ + M(BlobStorageLog, blob_storage_log, "Log blob storage operations") \ + namespace DB { @@ -34,71 +60,39 @@ namespace DB }; */ -class QueryLog; -class QueryThreadLog; -class PartLog; -class TextLog; -class TraceLog; -class CrashLog; -class ErrorLog; -class MetricLog; -class AsynchronousMetricLog; -class OpenTelemetrySpanLog; -class QueryViewsLog; -class ZooKeeperLog; -class SessionLog; -class TransactionsInfoLog; -class ProcessorsProfileLog; -class FilesystemCacheLog; -class FilesystemReadPrefetchesLog; -class AsynchronousInsertLog; -class BackupLog; -class ObjectStorageQueueLog; -class BlobStorageLog; +/// NOLINTBEGIN(bugprone-macro-parentheses) +#define FORWARD_DECLARATION(log_type, member, descr) \ + class log_type; \ + +LIST_OF_ALL_SYSTEM_LOGS(FORWARD_DECLARATION) +#undef FORWARD_DECLARATION +/// NOLINTEND(bugprone-macro-parentheses) + /// System logs should be destroyed in destructor of the last Context and before tables, /// because SystemLog destruction makes insert query while flushing data into underlying tables -struct SystemLogs +class SystemLogs { +public: + SystemLogs() = default; SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConfiguration & config); - ~SystemLogs(); + SystemLogs(const SystemLogs & other) = default; + void flush(bool should_prepare_tables_anyway); + void flushAndShutdown(); void shutdown(); void handleCrash(); - std::shared_ptr query_log; /// Used to log queries. - std::shared_ptr query_thread_log; /// Used to log query threads. - std::shared_ptr part_log; /// Used to log operations with parts - std::shared_ptr trace_log; /// Used to log traces from query profiler - std::shared_ptr crash_log; /// Used to log server crashes. - std::shared_ptr text_log; /// Used to log all text messages. - std::shared_ptr metric_log; /// Used to log all metrics. - std::shared_ptr error_log; /// Used to log errors. - std::shared_ptr filesystem_cache_log; - std::shared_ptr filesystem_read_prefetches_log; - std::shared_ptr s3_queue_log; - std::shared_ptr azure_queue_log; - /// Metrics from system.asynchronous_metrics. - std::shared_ptr asynchronous_metric_log; - /// OpenTelemetry trace spans. - std::shared_ptr opentelemetry_span_log; - /// Used to log queries of materialized and live views - std::shared_ptr query_views_log; - /// Used to log all actions of ZooKeeper client - std::shared_ptr zookeeper_log; - /// Login, LogOut and Login failure events - std::shared_ptr session_log; - /// Events related to transactions - std::shared_ptr transactions_info_log; - /// Used to log processors profiling - std::shared_ptr processors_profile_log; - std::shared_ptr asynchronous_insert_log; - /// Backup and restore events - std::shared_ptr backup_log; - /// Log blob storage operations - std::shared_ptr blob_storage_log; +/// NOLINTBEGIN(bugprone-macro-parentheses) +#define PUBLIC_MEMBERS(log_type, member, descr) \ + std::shared_ptr member; \ - std::vector logs; + LIST_OF_ALL_SYSTEM_LOGS(PUBLIC_MEMBERS) +#undef PUBLIC_MEMBERS +/// NOLINTEND(bugprone-macro-parentheses) + +private: + std::vector getAllLogs() const; }; struct SystemLogSettings From 633f700df60b878e59fe17d6204faf4e984b1009 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 2 Aug 2024 12:14:58 +0200 Subject: [PATCH 385/644] adjust tests --- .../test_system_flush_logs/test.py | 63 +++-- .../test_system_logs_recreate/test.py | 225 +++++++++--------- 2 files changed, 146 insertions(+), 142 deletions(-) diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index 0399122406a..44269883d1b 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -12,18 +12,6 @@ node = cluster.add_instance( stay_alive=True, ) -system_logs = [ - # enabled by default - ("system.text_log", 1), - ("system.query_log", 1), - ("system.query_thread_log", 1), - ("system.part_log", 1), - ("system.trace_log", 1), - ("system.metric_log", 1), - ("system.error_log", 1), -] - - @pytest.fixture(scope="module", autouse=True) def start_cluster(): try: @@ -33,22 +21,29 @@ def start_cluster(): cluster.shutdown() -@pytest.fixture(scope="function") -def flush_logs(): +def test_system_logs_exists(): + system_logs = [ + # disabled by default + ("system.text_log", 0), + ("system.query_log", 1), + ("system.query_thread_log", 1), + ("system.part_log", 1), + ("system.trace_log", 1), + ("system.metric_log", 1), + ("system.error_log", 1), + ] + node.query("SYSTEM FLUSH LOGS") - - -@pytest.mark.parametrize("table,exists", system_logs) -def test_system_logs(flush_logs, table, exists): - q = "SELECT * FROM {}".format(table) - if exists: - node.query(q) - else: - response = node.query_and_get_error(q) - assert ( - "Table {} does not exist".format(table) in response - or "Unknown table expression identifier '{}'".format(table) in response - ) + for table, exists in system_logs: + q = "SELECT * FROM {}".format(table) + if exists: + node.query(q) + else: + response = node.query_and_get_error(q) + assert ( + "Table {} does not exist".format(table) in response + or "Unknown table expression identifier '{}'".format(table) in response + ) # Logic is tricky, let's check that there is no hang in case of message queue @@ -67,11 +62,14 @@ def test_system_logs_non_empty_queue(): def test_system_suspend(): - node.query("CREATE TABLE t (x DateTime) ENGINE=Memory;") - node.query("INSERT INTO t VALUES (now());") - node.query("SYSTEM SUSPEND FOR 1 SECOND;") - node.query("INSERT INTO t VALUES (now());") - assert "1\n" == node.query("SELECT max(x) - min(x) >= 1 FROM t;") + try: + node.query("CREATE TABLE t (x DateTime) ENGINE=Memory;") + node.query("INSERT INTO t VALUES (now());") + node.query("SYSTEM SUSPEND FOR 1 SECOND;") + node.query("INSERT INTO t VALUES (now());") + assert "1\n" == node.query("SELECT max(x) - min(x) >= 1 FROM t;") + finally: + node.query("DROP TABLE IF EXISTS t;") def test_log_max_size(start_cluster): @@ -95,6 +93,7 @@ def test_log_max_size(start_cluster): ] ) + node.query("SYSTEM FLUSH LOGS") node.query(f"TRUNCATE TABLE IF EXISTS system.query_log") node.restart_clickhouse() diff --git a/tests/integration/test_system_logs_recreate/test.py b/tests/integration/test_system_logs_recreate/test.py index d5347ae7dea..c6d5861904c 100644 --- a/tests/integration/test_system_logs_recreate/test.py +++ b/tests/integration/test_system_logs_recreate/test.py @@ -33,124 +33,129 @@ def test_system_logs_recreate(): "error_log", ] - node.query("SYSTEM FLUSH LOGS") - for table in system_logs: - assert "ENGINE = MergeTree" in node.query(f"SHOW CREATE TABLE system.{table}") - assert "ENGINE = Null" not in node.query(f"SHOW CREATE TABLE system.{table}") - assert ( - len( - node.query(f"SHOW TABLES FROM system LIKE '{table}%'") - .strip() - .split("\n") + try: + node.query("SYSTEM FLUSH LOGS") + for table in system_logs: + assert "ENGINE = MergeTree" in node.query(f"SHOW CREATE TABLE system.{table}") + assert "ENGINE = Null" not in node.query(f"SHOW CREATE TABLE system.{table}") + assert ( + len( + node.query(f"SHOW TABLES FROM system LIKE '{table}%'") + .strip() + .split("\n") + ) + == 1 ) - == 1 - ) - # NOTE: we use zzz- prefix to make it the last file, - # so that it will be applied last. - for table in system_logs: - node.exec_in_container( - [ - "bash", - "-c", - f"""echo " - - <{table}> - ENGINE = Null - - - - " > /etc/clickhouse-server/config.d/zzz-override-{table}.xml - """, - ] - ) - - node.restart_clickhouse() - node.query("SYSTEM FLUSH LOGS") - for table in system_logs: - assert "ENGINE = MergeTree" not in node.query( - f"SHOW CREATE TABLE system.{table}" - ) - assert "ENGINE = Null" in node.query(f"SHOW CREATE TABLE system.{table}") - assert ( - len( - node.query(f"SHOW TABLES FROM system LIKE '{table}%'") - .strip() - .split("\n") + # NOTE: we use zzz- prefix to make it the last file, + # so that it will be applied last. + for table in system_logs: + node.exec_in_container( + [ + "bash", + "-c", + f"""echo " + + <{table}> + ENGINE = Null + + + + " > /etc/clickhouse-server/config.d/zzz-override-{table}.xml + """, + ] ) - == 2 - ) - # apply only storage_policy for all system tables - for table in system_logs: - node.exec_in_container( - [ - "bash", - "-c", - f"""echo " - - <{table}> - system_tables - - - " > /etc/clickhouse-server/config.d/zzz-override-{table}.xml - """, - ] - ) - node.restart_clickhouse() - node.query("SYSTEM FLUSH LOGS") - import logging - - for table in system_logs: - create_table_sql = node.query(f"SHOW CREATE TABLE system.{table} FORMAT TSVRaw") - logging.debug( - "With storage policy, SHOW CREATE TABLE system.%s is: %s", - table, - create_table_sql, - ) - assert "ENGINE = MergeTree" in create_table_sql - assert "ENGINE = Null" not in create_table_sql - assert "SETTINGS storage_policy = 'system_tables'" in create_table_sql - assert ( - len( - node.query(f"SHOW TABLES FROM system LIKE '{table}%'") - .strip() - .split("\n") + node.restart_clickhouse() + node.query("SYSTEM FLUSH LOGS") + for table in system_logs: + assert "ENGINE = MergeTree" not in node.query( + f"SHOW CREATE TABLE system.{table}" ) - == 3 - ) - - for table in system_logs: - node.exec_in_container( - ["rm", f"/etc/clickhouse-server/config.d/zzz-override-{table}.xml"] - ) - - node.restart_clickhouse() - node.query("SYSTEM FLUSH LOGS") - for table in system_logs: - assert "ENGINE = MergeTree" in node.query(f"SHOW CREATE TABLE system.{table}") - assert "ENGINE = Null" not in node.query(f"SHOW CREATE TABLE system.{table}") - assert ( - len( - node.query(f"SHOW TABLES FROM system LIKE '{table}%'") - .strip() - .split("\n") + assert "ENGINE = Null" in node.query(f"SHOW CREATE TABLE system.{table}") + assert ( + len( + node.query(f"SHOW TABLES FROM system LIKE '{table}%'") + .strip() + .split("\n") + ) + == 2 ) - == 4 - ) - node.query("SYSTEM FLUSH LOGS") - # Ensure that there was no superfluous RENAME's - # IOW that the table created only when the structure is indeed different. - for table in system_logs: - assert ( - len( - node.query(f"SHOW TABLES FROM system LIKE '{table}%'") - .strip() - .split("\n") + # apply only storage_policy for all system tables + for table in system_logs: + node.exec_in_container( + [ + "bash", + "-c", + f"""echo " + + <{table}> + system_tables + + + " > /etc/clickhouse-server/config.d/zzz-override-{table}.xml + """, + ] ) - == 4 - ) + node.restart_clickhouse() + node.query("SYSTEM FLUSH LOGS") + import logging + + for table in system_logs: + create_table_sql = node.query(f"SHOW CREATE TABLE system.{table} FORMAT TSVRaw") + logging.debug( + "With storage policy, SHOW CREATE TABLE system.%s is: %s", + table, + create_table_sql, + ) + assert "ENGINE = MergeTree" in create_table_sql + assert "ENGINE = Null" not in create_table_sql + assert "SETTINGS storage_policy = 'system_tables'" in create_table_sql + assert ( + len( + node.query(f"SHOW TABLES FROM system LIKE '{table}%'") + .strip() + .split("\n") + ) + == 3 + ) + + for table in system_logs: + node.exec_in_container( + ["rm", f"/etc/clickhouse-server/config.d/zzz-override-{table}.xml"] + ) + + node.restart_clickhouse() + node.query("SYSTEM FLUSH LOGS") + for table in system_logs: + assert "ENGINE = MergeTree" in node.query(f"SHOW CREATE TABLE system.{table}") + assert "ENGINE = Null" not in node.query(f"SHOW CREATE TABLE system.{table}") + assert ( + len( + node.query(f"SHOW TABLES FROM system LIKE '{table}%'") + .strip() + .split("\n") + ) + == 4 + ) + + node.query("SYSTEM FLUSH LOGS") + # Ensure that there was no superfluous RENAME's + # IOW that the table created only when the structure is indeed different. + for table in system_logs: + assert ( + len( + node.query(f"SHOW TABLES FROM system LIKE '{table}%'") + .strip() + .split("\n") + ) + == 4 + ) + finally: + for table in system_logs: + for syffix in range(3): + node.query(f"DROP TABLE IF EXISTS system.{table}_{syffix} sync") def test_drop_system_log(): From 08f8d94856841254957e1e746685d32de2074dc6 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 2 Aug 2024 12:16:44 +0200 Subject: [PATCH 386/644] no flush backup logs at shutdown, flush all logs --- src/Backups/BackupsWorker.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 106aa89082d..0b93ae6d547 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -1171,8 +1171,6 @@ void BackupsWorker::waitAll() for (const auto & id : current_operations) wait(id, /* rethrow_exception= */ false); - backup_log->flush(backup_log->getLastLogIndex(), /* should_prepare_tables_anyway */ false); - LOG_INFO(log, "Backups and restores finished"); } @@ -1223,8 +1221,6 @@ void BackupsWorker::cancelAll(bool wait_) for (const auto & id : current_operations) wait(id, /* rethrow_exception= */ false); - backup_log->flush(backup_log->getLastLogIndex(), /* should_prepare_tables_anyway */ false); - LOG_INFO(log, "Backups and restores finished or stopped"); } From 3cd3b2857c8104933eda67f901d8b098082c1049 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Aug 2024 12:33:04 +0200 Subject: [PATCH 387/644] 00965_shard_unresolvable_addresses is still slow --- .../queries/0_stateless/00965_shard_unresolvable_addresses.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql b/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql index 41bf4d261f6..f2afb974a06 100644 --- a/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql +++ b/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql @@ -1,4 +1,5 @@ --- Tags: shard +-- Tags: shard, no-fasttest +-- no-fasttest: Slow timeouts SET prefer_localhost_replica = 1; SET connections_with_failover_max_tries=1; From b416764585e7e03382f66c99636b4bf0b51bd79f Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 7 Aug 2024 13:02:38 +0200 Subject: [PATCH 388/644] put description of system log table in one place --- src/Interpreters/SystemLog.cpp | 35 ++++------------------ src/Interpreters/SystemLog.h | 54 ++++++++++++++++------------------ 2 files changed, 30 insertions(+), 59 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 9b58da3f545..d4403b72583 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -284,34 +284,11 @@ ASTPtr getCreateTableQueryClean(const StorageID & table_id, ContextPtr context) SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConfiguration & config) { - query_log = createSystemLog(global_context, "system", "query_log", config, "query_log", "Contains information about executed queries, for example, start time, duration of processing, error messages."); - query_thread_log = createSystemLog(global_context, "system", "query_thread_log", config, "query_thread_log", "Contains information about threads that execute queries, for example, thread name, thread start time, duration of query processing."); - part_log = createSystemLog(global_context, "system", "part_log", config, "part_log", "This table contains information about events that occurred with data parts in the MergeTree family tables, such as adding or merging data."); - trace_log = createSystemLog(global_context, "system", "trace_log", config, "trace_log", "Contains stack traces collected by the sampling query profiler."); - crash_log = createSystemLog(global_context, "system", "crash_log", config, "crash_log", "Contains information about stack traces for fatal errors. The table does not exist in the database by default, it is created only when fatal errors occur."); - text_log = createSystemLog(global_context, "system", "text_log", config, "text_log", "Contains logging entries which are normally written to a log file or to stdout."); - metric_log = createSystemLog(global_context, "system", "metric_log", config, "metric_log", "Contains history of metrics values from tables system.metrics and system.events, periodically flushed to disk."); - error_log = createSystemLog(global_context, "system", "error_log", config, "error_log", "Contains history of error values from table system.errors, periodically flushed to disk."); - filesystem_cache_log = createSystemLog(global_context, "system", "filesystem_cache_log", config, "filesystem_cache_log", "Contains a history of all events occurred with filesystem cache for objects on a remote filesystem."); - filesystem_read_prefetches_log = createSystemLog( - global_context, "system", "filesystem_read_prefetches_log", config, "filesystem_read_prefetches_log", "Contains a history of all prefetches done during reading from MergeTables backed by a remote filesystem."); - asynchronous_metric_log = createSystemLog( - global_context, "system", "asynchronous_metric_log", config, - "asynchronous_metric_log", "Contains the historical values for system.asynchronous_metrics, once per time interval (one second by default)."); - opentelemetry_span_log = createSystemLog( - global_context, "system", "opentelemetry_span_log", config, - "opentelemetry_span_log", "Contains information about trace spans for executed queries."); - query_views_log = createSystemLog(global_context, "system", "query_views_log", config, "query_views_log", "Contains information about the dependent views executed when running a query, for example, the view type or the execution time."); - zookeeper_log = createSystemLog(global_context, "system", "zookeeper_log", config, "zookeeper_log", "This table contains information about the parameters of the request to the ZooKeeper server and the response from it."); - session_log = createSystemLog(global_context, "system", "session_log", config, "session_log", "Contains information about all successful and failed login and logout events."); - transactions_info_log = createSystemLog( - global_context, "system", "transactions_info_log", config, "transactions_info_log", "Contains information about all transactions executed on a current server."); - processors_profile_log = createSystemLog(global_context, "system", "processors_profile_log", config, "processors_profile_log", "Contains profiling information on processors level (building blocks for a pipeline for query execution."); - asynchronous_insert_log = createSystemLog(global_context, "system", "asynchronous_insert_log", config, "asynchronous_insert_log", "Contains a history for all asynchronous inserts executed on current server."); - backup_log = createSystemLog(global_context, "system", "backup_log", config, "backup_log", "Contains logging entries with the information about BACKUP and RESTORE operations."); - s3_queue_log = createSystemLog(global_context, "system", "s3queue_log", config, "s3queue_log", "Contains logging entries with the information files processes by S3Queue engine."); - azure_queue_log = createSystemLog(global_context, "system", "azure_queue_log", config, "azure_queue_log", "Contains logging entries with the information files processes by S3Queue engine."); - blob_storage_log = createSystemLog(global_context, "system", "blob_storage_log", config, "blob_storage_log", "Contains logging entries with information about various blob storage operations such as uploads and deletes."); +#define CREATE_PUBLIC_MEMBERS(log_type, member, descr) \ + member = createSystemLog(global_context, "system", #member, config, #member, descr); \ + + LIST_OF_ALL_SYSTEM_LOGS(CREATE_PUBLIC_MEMBERS) +#undef CREATE_PUBLIC_MEMBERS if (session_log) global_context->addWarningMessage("Table system.session_log is enabled. It's unreliable and may contain garbage. Do not use it for any kind of security monitoring."); @@ -355,7 +332,6 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf std::vector SystemLogs::getAllLogs() const { -/// NOLINTBEGIN(bugprone-macro-parentheses) #define GET_RAW_POINTERS(log_type, member, descr) \ member.get(), \ @@ -363,7 +339,6 @@ std::vector SystemLogs::getAllLogs() const LIST_OF_ALL_SYSTEM_LOGS(GET_RAW_POINTERS) }; #undef GET_RAW_POINTERS -/// NOLINTEND(bugprone-macro-parentheses) auto last_it = std::remove(result.begin(), result.end(), nullptr); result.erase(last_it, result.end()); diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 093be203282..6682829c0c6 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -8,28 +8,28 @@ #include #define LIST_OF_ALL_SYSTEM_LOGS(M) \ - M(QueryLog, query_log, "Used to log queries.") \ - M(QueryThreadLog, query_thread_log, "Used to log query threads.") \ - M(PartLog, part_log, "Used to log operations with parts.") \ - M(TraceLog, trace_log, "Used to log traces from query profiler.") \ - M(CrashLog, crash_log, "Used to log server crashes.") \ - M(TextLog, text_log, "Used to log all text messages.") \ - M(MetricLog, metric_log, "Used to log all metrics.") \ - M(ErrorLog, error_log, "Used to log errors.") \ - M(FilesystemCacheLog, filesystem_cache_log, "") \ - M(FilesystemReadPrefetchesLog, filesystem_read_prefetches_log, "") \ - M(ObjectStorageQueueLog, s3_queue_log, "") \ - M(ObjectStorageQueueLog, azure_queue_log, "") \ - M(AsynchronousMetricLog, asynchronous_metric_log, "Metrics from system.asynchronous_metrics") \ - M(OpenTelemetrySpanLog, opentelemetry_span_log, "OpenTelemetry trace spans.") \ - M(QueryViewsLog, query_views_log, "Used to log queries of materialized and live views.") \ - M(ZooKeeperLog, zookeeper_log, "Used to log all actions of ZooKeeper client.") \ - M(SessionLog, session_log, "Login, LogOut and Login failure events.") \ - M(TransactionsInfoLog, transactions_info_log, "Events related to transactions.") \ - M(ProcessorsProfileLog, processors_profile_log, "Used to log processors profiling") \ - M(AsynchronousInsertLog, asynchronous_insert_log, "") \ - M(BackupLog, backup_log, "Backup and restore events") \ - M(BlobStorageLog, blob_storage_log, "Log blob storage operations") \ + M(QueryLog, query_log, "Contains information about executed queries, for example, start time, duration of processing, error messages.") \ + M(QueryThreadLog, query_thread_log, "Contains information about threads that execute queries, for example, thread name, thread start time, duration of query processing.") \ + M(PartLog, part_log, "This table contains information about events that occurred with data parts in the MergeTree family tables, such as adding or merging data.") \ + M(TraceLog, trace_log, "Contains stack traces collected by the sampling query profiler.") \ + M(CrashLog, crash_log, "Contains information about stack traces for fatal errors. The table does not exist in the database by default, it is created only when fatal errors occur.") \ + M(TextLog, text_log, "Contains logging entries which are normally written to a log file or to stdout.") \ + M(MetricLog, metric_log, "Contains history of metrics values from tables system.metrics and system.events, periodically flushed to disk.") \ + M(ErrorLog, error_log, "Contains history of error values from table system.errors, periodically flushed to disk.") \ + M(FilesystemCacheLog, filesystem_cache_log, "Contains a history of all events occurred with filesystem cache for objects on a remote filesystem.") \ + M(FilesystemReadPrefetchesLog, filesystem_read_prefetches_log, "Contains a history of all prefetches done during reading from MergeTables backed by a remote filesystem.") \ + M(ObjectStorageQueueLog, s3_queue_log, "Contains logging entries with the information files processes by S3Queue engine.") \ + M(ObjectStorageQueueLog, azure_queue_log, "Contains logging entries with the information files processes by S3Queue engine.") \ + M(AsynchronousMetricLog, asynchronous_metric_log, "Contains the historical values for system.asynchronous_metrics, once per time interval (one second by default).") \ + M(OpenTelemetrySpanLog, opentelemetry_span_log, "Contains information about trace spans for executed queries.") \ + M(QueryViewsLog, query_views_log, "Contains information about the dependent views executed when running a query, for example, the view type or the execution time.") \ + M(ZooKeeperLog, zookeeper_log, "This table contains information about the parameters of the request to the ZooKeeper server and the response from it.") \ + M(SessionLog, session_log, "Contains information about all successful and failed login and logout events.") \ + M(TransactionsInfoLog, transactions_info_log, "Contains information about all transactions executed on a current server.") \ + M(ProcessorsProfileLog, processors_profile_log, "Contains profiling information on processors level (building blocks for a pipeline for query execution.") \ + M(AsynchronousInsertLog, asynchronous_insert_log, "Contains a history for all asynchronous inserts executed on current server.") \ + M(BackupLog, backup_log, "Contains logging entries with the information about BACKUP and RESTORE operations.") \ + M(BlobStorageLog, blob_storage_log, "Contains logging entries with information about various blob storage operations such as uploads and deletes.") \ namespace DB @@ -60,13 +60,11 @@ namespace DB }; */ -/// NOLINTBEGIN(bugprone-macro-parentheses) #define FORWARD_DECLARATION(log_type, member, descr) \ class log_type; \ LIST_OF_ALL_SYSTEM_LOGS(FORWARD_DECLARATION) #undef FORWARD_DECLARATION -/// NOLINTEND(bugprone-macro-parentheses) /// System logs should be destroyed in destructor of the last Context and before tables, @@ -83,13 +81,11 @@ public: void shutdown(); void handleCrash(); -/// NOLINTBEGIN(bugprone-macro-parentheses) -#define PUBLIC_MEMBERS(log_type, member, descr) \ +#define DECLARE_PUBLIC_MEMBERS(log_type, member, descr) \ std::shared_ptr member; \ - LIST_OF_ALL_SYSTEM_LOGS(PUBLIC_MEMBERS) -#undef PUBLIC_MEMBERS -/// NOLINTEND(bugprone-macro-parentheses) + LIST_OF_ALL_SYSTEM_LOGS(DECLARE_PUBLIC_MEMBERS) +#undef DECLARE_PUBLIC_MEMBERS private: std::vector getAllLogs() const; From b9f564f6f4fa98cc385acd6ce3c8e9f736bc55e0 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 7 Aug 2024 11:12:11 +0000 Subject: [PATCH 389/644] Automatic style fix --- .../test_system_flush_logs/test.py | 1 + .../test_system_logs_recreate/test.py | 20 ++++++++++++++----- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index 44269883d1b..dd48ef055f5 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -12,6 +12,7 @@ node = cluster.add_instance( stay_alive=True, ) + @pytest.fixture(scope="module", autouse=True) def start_cluster(): try: diff --git a/tests/integration/test_system_logs_recreate/test.py b/tests/integration/test_system_logs_recreate/test.py index c6d5861904c..8b84734ed02 100644 --- a/tests/integration/test_system_logs_recreate/test.py +++ b/tests/integration/test_system_logs_recreate/test.py @@ -36,8 +36,12 @@ def test_system_logs_recreate(): try: node.query("SYSTEM FLUSH LOGS") for table in system_logs: - assert "ENGINE = MergeTree" in node.query(f"SHOW CREATE TABLE system.{table}") - assert "ENGINE = Null" not in node.query(f"SHOW CREATE TABLE system.{table}") + assert "ENGINE = MergeTree" in node.query( + f"SHOW CREATE TABLE system.{table}" + ) + assert "ENGINE = Null" not in node.query( + f"SHOW CREATE TABLE system.{table}" + ) assert ( len( node.query(f"SHOW TABLES FROM system LIKE '{table}%'") @@ -103,7 +107,9 @@ def test_system_logs_recreate(): import logging for table in system_logs: - create_table_sql = node.query(f"SHOW CREATE TABLE system.{table} FORMAT TSVRaw") + create_table_sql = node.query( + f"SHOW CREATE TABLE system.{table} FORMAT TSVRaw" + ) logging.debug( "With storage policy, SHOW CREATE TABLE system.%s is: %s", table, @@ -129,8 +135,12 @@ def test_system_logs_recreate(): node.restart_clickhouse() node.query("SYSTEM FLUSH LOGS") for table in system_logs: - assert "ENGINE = MergeTree" in node.query(f"SHOW CREATE TABLE system.{table}") - assert "ENGINE = Null" not in node.query(f"SHOW CREATE TABLE system.{table}") + assert "ENGINE = MergeTree" in node.query( + f"SHOW CREATE TABLE system.{table}" + ) + assert "ENGINE = Null" not in node.query( + f"SHOW CREATE TABLE system.{table}" + ) assert ( len( node.query(f"SHOW TABLES FROM system LIKE '{table}%'") From 49871bacc1d56fb82b78c70dbfc92d52003e2e99 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 7 Aug 2024 12:37:39 +0100 Subject: [PATCH 390/644] fix test --- .../poco/Net/include/Poco/Net/HTTPServerSession.h | 1 - base/poco/Net/src/HTTPServerSession.cpp | 1 - tests/integration/test_server_keep_alive/test.py | 15 ++++++++++++--- 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPServerSession.h b/base/poco/Net/include/Poco/Net/HTTPServerSession.h index 54e7f2c8c50..b0659ca405c 100644 --- a/base/poco/Net/include/Poco/Net/HTTPServerSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPServerSession.h @@ -66,7 +66,6 @@ namespace Net bool _firstRequest; Poco::Timespan _keepAliveTimeout; int _maxKeepAliveRequests; - HTTPServerParams::Ptr _params; }; diff --git a/base/poco/Net/src/HTTPServerSession.cpp b/base/poco/Net/src/HTTPServerSession.cpp index 3093f215952..8eec3e14872 100644 --- a/base/poco/Net/src/HTTPServerSession.cpp +++ b/base/poco/Net/src/HTTPServerSession.cpp @@ -24,7 +24,6 @@ HTTPServerSession::HTTPServerSession(const StreamSocket & socket, HTTPServerPara , _firstRequest(true) , _keepAliveTimeout(pParams->getKeepAliveTimeout()) , _maxKeepAliveRequests(pParams->getMaxKeepAliveRequests()) - , _params(pParams) { setTimeout(pParams->getTimeout()); } diff --git a/tests/integration/test_server_keep_alive/test.py b/tests/integration/test_server_keep_alive/test.py index 96f08a37adb..e550319b6df 100644 --- a/tests/integration/test_server_keep_alive/test.py +++ b/tests/integration/test_server_keep_alive/test.py @@ -1,5 +1,6 @@ import logging import pytest +import random import requests from helpers.cluster import ClickHouseCluster @@ -24,19 +25,27 @@ def test_max_keep_alive_requests_on_user_side(start_cluster): # In this test we have `keep_alive_timeout` set to one hour to never trigger connection reset by timeout, `max_keep_alive_requests` is set to 5. # We expect server to close connection after each 5 requests. We detect connection reset by change in src port. # So the first 5 requests should come from the same port, the following 5 requests should come from another port. + + log_comments = [] + for _ in range(10): + rand_id = random.randint(0, 1000000) + log_comment = f"test_requests_with_keep_alive_{rand_id}" + log_comments.append(log_comment) + log_comments = sorted(log_comments) + session = requests.Session() for i in range(10): session.get( - f"http://{node.ip_address}:8123/?query=select%201&log_comment=test_requests_with_keep_alive_{i}" + f"http://{node.ip_address}:8123/?query=select%201&log_comment={log_comments[i]}" ) ports = node.query( - """ + f""" SYSTEM FLUSH LOGS; SELECT port FROM system.query_log - WHERE log_comment like 'test_requests_with_keep_alive_%' AND type = 'QueryFinish' + WHERE log_comment IN ({", ".join(f"'{comment}'" for comment in log_comments)}) AND type = 'QueryFinish' ORDER BY log_comment """ ).split("\n")[:-1] From 016d1fea6d72c25179633f72f8ca8338dc59338f Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 7 Aug 2024 13:58:03 +0200 Subject: [PATCH 391/644] CI: Integration tests uncover some logging --- tests/ci/integration_tests_runner.py | 22 ++++++++-------------- 1 file changed, 8 insertions(+), 14 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 2b348be8b51..6165b1b9aaa 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -20,6 +20,7 @@ from typing import Any, Dict from env_helper import IS_CI from integration_test_images import IMAGES +from tee_popen import TeePopen MAX_RETRY = 1 NUM_WORKERS = 5 @@ -356,20 +357,13 @@ class ClickhouseIntegrationTestsRunner: logging.info("Package found in %s", full_path) log_name = "install_" + f + ".log" log_path = os.path.join(str(self.path()), log_name) - with open(log_path, "w", encoding="utf-8") as log: - cmd = f"dpkg -x {full_path} ." - logging.info("Executing installation cmd %s", cmd) - with subprocess.Popen( - cmd, shell=True, stderr=log, stdout=log - ) as proc: - if proc.wait() == 0: - logging.info( - "Installation of %s successfull", full_path - ) - else: - raise RuntimeError( - f"Installation of {full_path} failed" - ) + cmd = f"dpkg -x {full_path} ." + logging.info("Executing installation cmd %s", cmd) + with TeePopen(cmd, log_file=log_path) as proc: + if proc.wait() == 0: + logging.info("Installation of %s successfull", full_path) + else: + raise RuntimeError(f"Installation of {full_path} failed") break else: raise FileNotFoundError(f"Package with {package} not found") From ddc058aa6ff6780fa67bc5c59d9d7ff9a71d4ee1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 7 Aug 2024 12:51:06 +0200 Subject: [PATCH 392/644] Update minio in stateless tests --- docker/test/stateless/Dockerfile | 4 ++-- docker/test/stateless/setup_minio.sh | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index a0e5513a3a2..d8eb072328f 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -69,8 +69,8 @@ ENV MAX_RUN_TIME=0 # Unrelated to vars in setup_minio.sh, but should be the same there # to have the same binaries for local running scenario -ARG MINIO_SERVER_VERSION=2022-01-03T18-22-58Z -ARG MINIO_CLIENT_VERSION=2022-01-05T23-52-51Z +ARG MINIO_SERVER_VERSION=2024-08-03T04-33-23Z +ARG MINIO_CLIENT_VERSION=2024-07-31T15-58-33Z ARG TARGETARCH # Download Minio-related binaries diff --git a/docker/test/stateless/setup_minio.sh b/docker/test/stateless/setup_minio.sh index 2b9433edd20..d8310d072b8 100755 --- a/docker/test/stateless/setup_minio.sh +++ b/docker/test/stateless/setup_minio.sh @@ -59,8 +59,8 @@ find_os() { download_minio() { local os local arch - local minio_server_version=${MINIO_SERVER_VERSION:-2022-09-07T22-25-02Z} - local minio_client_version=${MINIO_CLIENT_VERSION:-2022-08-28T20-08-11Z} + local minio_server_version=${MINIO_SERVER_VERSION:-2024-08-03T04-33-23Z} + local minio_client_version=${MINIO_CLIENT_VERSION:-2024-07-31T15-58-33Z} os=$(find_os) arch=$(find_arch) @@ -82,10 +82,10 @@ setup_minio() { local test_type=$1 ./mc alias set clickminio http://localhost:11111 clickhouse clickhouse ./mc admin user add clickminio test testtest - ./mc admin policy set clickminio readwrite user=test + ./mc admin policy attach clickminio readwrite --user=test ./mc mb --ignore-existing clickminio/test if [ "$test_type" = "stateless" ]; then - ./mc policy set public clickminio/test + ./mc anonymous set public clickminio/test fi } @@ -148,4 +148,4 @@ main() { setup_aws_credentials } -main "$@" \ No newline at end of file +main "$@" From 6afff5824e3bd3e4eca2c50d7cce10dda5678433 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 7 Aug 2024 12:14:05 +0000 Subject: [PATCH 393/644] Fix 03130_convert_outer_join_to_inner_join --- .../03130_convert_outer_join_to_inner_join.sql | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql index 28362f1f469..168177a06d7 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql @@ -6,14 +6,18 @@ CREATE TABLE test_table_1 ( id UInt64, value String -) ENGINE=MergeTree ORDER BY id; +) ENGINE=MergeTree ORDER BY id +SETTINGS index_granularity = 16 # We have number of granules in the `EXPLAIN` output in reference file +; DROP TABLE IF EXISTS test_table_2; CREATE TABLE test_table_2 ( id UInt64, value String -) ENGINE=MergeTree ORDER BY id; +) ENGINE=MergeTree ORDER BY id +SETTINGS index_granularity = 16 +; INSERT INTO test_table_1 VALUES (1, 'Value_1'), (2, 'Value_2'); INSERT INTO test_table_2 VALUES (2, 'Value_2'), (3, 'Value_3'); From 4a537874cad725227c847850b2da42d0ed86ccaf Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 7 Aug 2024 14:35:05 +0200 Subject: [PATCH 394/644] adjust tests --- src/Disks/DiskFomAST.cpp | 2 +- src/Storages/MergeTree/MergeTreeSettings.cpp | 22 +++++++++---------- ...2808_custom_disk_with_user_defined_name.sh | 2 +- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index 2a5e7368de9..35cb124acfd 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -144,7 +144,7 @@ std::string DiskFomAST::getConfigDefinedDisk(const std::string &disk_name, Conte if (result->isCustomDisk()) throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Disk name `{}` is a custom disk that is used in other table." + "Disk name `{}` is a custom disk that is used in other table. " "That disk could not be used by a reference by other tables. The custom disk should be fully specified with a disk function.", disk_name); diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index f72b24e3270..e11af43ed23 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -59,19 +59,19 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr conte CustomType custom; if (name == "disk") { + ASTPtr value_as_custom_ast = nullptr; if (value.tryGet(custom) && 0 == strcmp(custom.getTypeName(), "AST")) + value_as_custom_ast = dynamic_cast(custom.getImpl()).ast; + + if (value_as_custom_ast && isDiskFunction(value_as_custom_ast)) { - auto ast = dynamic_cast(custom.getImpl()).ast; - if (ast && isDiskFunction(ast)) - { - auto disk_name = DiskFomAST::createCustomDisk(ast, context, is_attach); - LOG_DEBUG(getLogger("MergeTreeSettings"), "Created custom disk {}", disk_name); - value = disk_name; - } - else - { - value = DiskFomAST::getConfigDefinedDisk(value.safeGet(), context); - } + auto disk_name = DiskFomAST::createCustomDisk(value_as_custom_ast, context, is_attach); + LOG_DEBUG(getLogger("MergeTreeSettings"), "Created custom disk {}", disk_name); + value = disk_name; + } + else + { + value = DiskFomAST::getConfigDefinedDisk(value.safeGet(), context); } if (has("storage_policy")) diff --git a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh index 333bc1bc25d..b62adea5683 100755 --- a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh +++ b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh @@ -13,7 +13,7 @@ DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) ENGINE = MergeTree() ORDER BY tuple() SETTINGS disk = disk(name = 's3_disk', type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); -""" 2>&1 | grep -q "Disk with name \`s3_disk\` already exist" && echo 'OK' || echo 'FAIL' +""" 2>&1 | grep -q "The disk \`s3_disk\` is already exist and described by the config" && echo 'OK' || echo 'FAIL' disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}" From 139fe624c2cd8e7e0af1e779f81113f1b132e9f7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Aug 2024 13:16:47 +0000 Subject: [PATCH 395/644] Fix typos --- tests/integration/test_storage_kafka/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index b1d4f1f26b3..c1a66934f43 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2242,16 +2242,16 @@ def test_kafka_virtual_columns_with_materialized_view( ) -def insert_with_retry(instance, values, table_name="kafka", max_try_couunt=5): +def insert_with_retry(instance, values, table_name="kafka", max_try_count=5): try_count = 0 while True: logging.debug(f"Inserting, try_count is {try_count}") try: try_count += 1 - instance.query("INSERT INTO test.kafka VALUES {}".format(values)) + instance.query(f"INSERT INTO test.{table_name} VALUES {values}") break except QueryRuntimeException as e: - if "Local: Timed out." in str(e) and try_count < max_try_couunt: + if "Local: Timed out." in str(e) and try_count < max_try_count: continue else: raise From a9b22a454d42ce81da8e711e8abd1e9908443ebc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 15:11:21 +0200 Subject: [PATCH 396/644] Make 03172_system_detached_tables parallelizable --- .../03172_system_detached_tables.reference | 14 ++-- .../03172_system_detached_tables.sh | 68 +++++++++++++++++++ .../03172_system_detached_tables.sql | 53 --------------- 3 files changed, 75 insertions(+), 60 deletions(-) create mode 100755 tests/queries/0_stateless/03172_system_detached_tables.sh delete mode 100644 tests/queries/0_stateless/03172_system_detached_tables.sql diff --git a/tests/queries/0_stateless/03172_system_detached_tables.reference b/tests/queries/0_stateless/03172_system_detached_tables.reference index 83d1ff13942..2fce89e5980 100644 --- a/tests/queries/0_stateless/03172_system_detached_tables.reference +++ b/tests/queries/0_stateless/03172_system_detached_tables.reference @@ -1,11 +1,11 @@ database atomic tests -test03172_system_detached_tables test_table 0 -test03172_system_detached_tables test_table_perm 1 -test03172_system_detached_tables test_table 0 -test03172_system_detached_tables test_table_perm 1 -test03172_system_detached_tables test_table 0 +default_atomic test_table 0 +default_atomic test_table_perm 1 +default_atomic test_table 0 +default_atomic test_table_perm 1 +default_atomic test_table 0 ----------------------- database lazy tests -before attach test03172_system_detached_tables_lazy test_table 0 -before attach test03172_system_detached_tables_lazy test_table_perm 1 +before attach default_lazy test_table 0 +before attach default_lazy test_table_perm 1 DROP TABLE diff --git a/tests/queries/0_stateless/03172_system_detached_tables.sh b/tests/queries/0_stateless/03172_system_detached_tables.sh new file mode 100755 index 00000000000..47775abcc45 --- /dev/null +++ b/tests/queries/0_stateless/03172_system_detached_tables.sh @@ -0,0 +1,68 @@ +#!/bin/bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +DATABASE_ATOMIC="${CLICKHOUSE_DATABASE}_atomic" +DATABASE_LAZY="${CLICKHOUSE_DATABASE}_lazy" + +$CLICKHOUSE_CLIENT --multiquery " + +SELECT 'database atomic tests'; +DROP DATABASE IF EXISTS ${DATABASE_ATOMIC}; +CREATE DATABASE IF NOT EXISTS ${DATABASE_ATOMIC} ENGINE=Atomic; + +CREATE TABLE ${DATABASE_ATOMIC}.test_table (n Int64) ENGINE=MergeTree ORDER BY n; +SELECT * FROM system.detached_tables WHERE database='${DATABASE_ATOMIC}'; + +DETACH TABLE ${DATABASE_ATOMIC}.test_table; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='${DATABASE_ATOMIC}'; + +ATTACH TABLE ${DATABASE_ATOMIC}.test_table; + +CREATE TABLE ${DATABASE_ATOMIC}.test_table_perm (n Int64) ENGINE=MergeTree ORDER BY n; +SELECT * FROM system.detached_tables WHERE database='${DATABASE_ATOMIC}'; + +DETACH TABLE ${DATABASE_ATOMIC}.test_table_perm PERMANENTLY; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='${DATABASE_ATOMIC}'; + +DETACH TABLE ${DATABASE_ATOMIC}.test_table SYNC; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='${DATABASE_ATOMIC}'; + +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='${DATABASE_ATOMIC}' AND table='test_table'; + +DROP DATABASE ${DATABASE_ATOMIC} SYNC; + +" + +$CLICKHOUSE_CLIENT --multiquery " + +SELECT '-----------------------'; +SELECT 'database lazy tests'; + +DROP DATABASE IF EXISTS ${DATABASE_LAZY}; +CREATE DATABASE ${DATABASE_LAZY} Engine=Lazy(10); + +CREATE TABLE ${DATABASE_LAZY}.test_table (number UInt64) engine=Log; +INSERT INTO ${DATABASE_LAZY}.test_table SELECT * FROM numbers(100); +DETACH TABLE ${DATABASE_LAZY}.test_table; + +CREATE TABLE ${DATABASE_LAZY}.test_table_perm (number UInt64) engine=Log; +INSERT INTO ${DATABASE_LAZY}.test_table_perm SELECT * FROM numbers(100); +DETACH table ${DATABASE_LAZY}.test_table_perm PERMANENTLY; + +SELECT 'before attach', database, table, is_permanently FROM system.detached_tables WHERE database='${DATABASE_LAZY}'; + +ATTACH TABLE ${DATABASE_LAZY}.test_table; +ATTACH TABLE ${DATABASE_LAZY}.test_table_perm; + +SELECT 'after attach', database, table, is_permanently FROM system.detached_tables WHERE database='${DATABASE_LAZY}'; + +SELECT 'DROP TABLE'; +DROP TABLE ${DATABASE_LAZY}.test_table SYNC; +DROP TABLE ${DATABASE_LAZY}.test_table_perm SYNC; + +DROP DATABASE ${DATABASE_LAZY} SYNC; + +" diff --git a/tests/queries/0_stateless/03172_system_detached_tables.sql b/tests/queries/0_stateless/03172_system_detached_tables.sql deleted file mode 100644 index 1a3c2d7cc0f..00000000000 --- a/tests/queries/0_stateless/03172_system_detached_tables.sql +++ /dev/null @@ -1,53 +0,0 @@ --- Tags: no-parallel - -SELECT 'database atomic tests'; -DROP DATABASE IF EXISTS test03172_system_detached_tables; -CREATE DATABASE IF NOT EXISTS test03172_system_detached_tables ENGINE=Atomic; - -CREATE TABLE test03172_system_detached_tables.test_table (n Int64) ENGINE=MergeTree ORDER BY n; -SELECT * FROM system.detached_tables WHERE database='test03172_system_detached_tables'; - -DETACH TABLE test03172_system_detached_tables.test_table; -SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables'; - -ATTACH TABLE test03172_system_detached_tables.test_table; - -CREATE TABLE test03172_system_detached_tables.test_table_perm (n Int64) ENGINE=MergeTree ORDER BY n; -SELECT * FROM system.detached_tables WHERE database='test03172_system_detached_tables'; - -DETACH TABLE test03172_system_detached_tables.test_table_perm PERMANENTLY; -SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables'; - -DETACH TABLE test03172_system_detached_tables.test_table SYNC; -SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables'; - -SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables' AND table='test_table'; - -DROP DATABASE test03172_system_detached_tables SYNC; - -SELECT '-----------------------'; -SELECT 'database lazy tests'; - -DROP DATABASE IF EXISTS test03172_system_detached_tables_lazy; -CREATE DATABASE test03172_system_detached_tables_lazy Engine=Lazy(10); - -CREATE TABLE test03172_system_detached_tables_lazy.test_table (number UInt64) engine=Log; -INSERT INTO test03172_system_detached_tables_lazy.test_table SELECT * FROM numbers(100); -DETACH TABLE test03172_system_detached_tables_lazy.test_table; - -CREATE TABLE test03172_system_detached_tables_lazy.test_table_perm (number UInt64) engine=Log; -INSERT INTO test03172_system_detached_tables_lazy.test_table_perm SELECT * FROM numbers(100); -DETACH table test03172_system_detached_tables_lazy.test_table_perm PERMANENTLY; - -SELECT 'before attach', database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables_lazy'; - -ATTACH TABLE test03172_system_detached_tables_lazy.test_table; -ATTACH TABLE test03172_system_detached_tables_lazy.test_table_perm; - -SELECT 'after attach', database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables_lazy'; - -SELECT 'DROP TABLE'; -DROP TABLE test03172_system_detached_tables_lazy.test_table SYNC; -DROP TABLE test03172_system_detached_tables_lazy.test_table_perm SYNC; - -DROP DATABASE test03172_system_detached_tables_lazy SYNC; From 3a564255de9f6f827ff033392cc6b15ca260c6fa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 15:16:15 +0200 Subject: [PATCH 397/644] Make 01764_table_function_dictionary parallelizable --- tests/queries/0_stateless/01764_table_function_dictionary.sql | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01764_table_function_dictionary.sql b/tests/queries/0_stateless/01764_table_function_dictionary.sql index 76e7213b367..e37f8d2a290 100644 --- a/tests/queries/0_stateless/01764_table_function_dictionary.sql +++ b/tests/queries/0_stateless/01764_table_function_dictionary.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel - DROP TABLE IF EXISTS table_function_dictionary_source_table; CREATE TABLE table_function_dictionary_source_table ( @@ -18,7 +16,7 @@ CREATE DICTIONARY table_function_dictionary_test_dictionary value UInt64 DEFAULT 0 ) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_function_dictionary_source_table')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' DATABASE currentDatabase() TABLE 'table_function_dictionary_source_table')) LAYOUT(DIRECT()); SELECT * FROM dictionary('table_function_dictionary_test_dictionary'); From 1dd5af578871d00602b427c3eec9fe8308e7cbd1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 15:17:31 +0200 Subject: [PATCH 398/644] Make 01760_ddl_dictionary_use_current_database_name parallelizable --- .../01760_ddl_dictionary_use_current_database_name.sql | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/01760_ddl_dictionary_use_current_database_name.sql b/tests/queries/0_stateless/01760_ddl_dictionary_use_current_database_name.sql index a7f04921f1f..c6bccde8590 100644 --- a/tests/queries/0_stateless/01760_ddl_dictionary_use_current_database_name.sql +++ b/tests/queries/0_stateless/01760_ddl_dictionary_use_current_database_name.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel - DROP TABLE IF EXISTS ddl_dictonary_test_source; CREATE TABLE ddl_dictonary_test_source ( From 76ce1fc1ee82be806a3472108ff104cb440a1aed Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 15:20:22 +0200 Subject: [PATCH 399/644] Make 01945_system_warnings parallelizable --- tests/queries/0_stateless/01945_system_warnings.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/01945_system_warnings.sh b/tests/queries/0_stateless/01945_system_warnings.sh index 249c3218bcc..63403ce2893 100755 --- a/tests/queries/0_stateless/01945_system_warnings.sh +++ b/tests/queries/0_stateless/01945_system_warnings.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From b3c30b05d6ea7c436ddc21f31016122c10de1ec7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Aug 2024 13:22:22 +0000 Subject: [PATCH 400/644] Make test_kafka_insert repeatable --- tests/integration/test_storage_kafka/test.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index c1a66934f43..3f71866c913 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2280,11 +2280,12 @@ def test_kafka_insert(kafka_cluster, create_query_generator): values.append("({i}, {i})".format(i=i)) values = ",".join(values) - insert_with_retry(instance, values) + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): + insert_with_retry(instance, values) - messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count) - result = "\n".join(messages) - kafka_check_result(result, True) + messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count) + result = "\n".join(messages) + kafka_check_result(result, True) @pytest.mark.parametrize( From ac5aab17584ab66e81a0124b9b87590e8d6ad43d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Aug 2024 13:30:28 +0000 Subject: [PATCH 401/644] Handle kafka null messages --- src/Storages/Kafka/KafkaConsumer2.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index 8581398aa90..f6ef85da317 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -358,8 +358,9 @@ ReadBufferPtr KafkaConsumer2::getNextMessage() size_t size = current->get_payload().get_size(); ++current; - chassert(data != nullptr); - return std::make_shared(data, size); + // `data` can be nullptr on case of the Kafka message has empty payload + if (data) + return std::make_shared(data, size); } return nullptr; From 28c8d158635ef2564a60c25a6f971aa324030dcc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Aug 2024 13:37:56 +0000 Subject: [PATCH 402/644] Fix style --- tests/queries/0_stateless/03217_filtering_in_system_tables.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql index bbc755e478d..72ca7c8684d 100644 --- a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql @@ -16,10 +16,12 @@ SYSTEM FLUSH LOGS; -- StorageSystemTables SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 + AND current_database = currentDatabase() AND query LIKE '%SELECT database, table FROM system.tables WHERE database = \'information_schema\' AND table = \'tables\';' AND type = 'QueryFinish'; -- StorageSystemReplicas SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 + AND current_database = currentDatabase() AND query LIKE '%SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = \'test_03217_system_tables_replica_1\' AND replica_name = \'r1\';' AND type = 'QueryFinish'; From 3485e87d8ac635ec42e0f55f0d11dbb07ae03dba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Aug 2024 15:18:55 +0000 Subject: [PATCH 403/644] Really handle null messages --- src/Storages/Kafka/KafkaConsumer2.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index f6ef85da317..dc71086db3b 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -352,8 +352,7 @@ void KafkaConsumer2::subscribeIfNotSubscribedYet() ReadBufferPtr KafkaConsumer2::getNextMessage() { - if (current != messages.end()) - { + while (current != messages.end()) { const auto * data = current->get_payload().get_data(); size_t size = current->get_payload().get_size(); ++current; From d2e9833dba4038999e7aeb540ea6b56cc1c59449 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 7 Aug 2024 17:00:41 +0200 Subject: [PATCH 404/644] Add minio audit logs --- docker/test/stateless/run.sh | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ea32df23af0..b33c261dacc 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -176,6 +176,25 @@ done setup_logs_replication attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01 +# create minio log webhooks for both audit and server logs +clickhouse-client --query "CREATE TABLE minio_audit_logs +( + log String, + event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(substring(JSONExtractRaw(log, 'time'), 2, 29), 9, 'UTC') +) +ENGINE = MergeTree +ORDER BY tuple()" +./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" + +clickhouse-client --query "CREATE TABLE minio_server_logs +( + log String, + event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(substring(JSONExtractRaw(log, 'time'), 2, 29), 9, 'UTC') +) +ENGINE = MergeTree +ORDER BY tuple()" +./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" + function fn_exists() { declare -F "$1" > /dev/null; } @@ -328,6 +347,11 @@ do fi done + +# collect minio audit and server logs +clickhouse-client -q "SELECT log FROM minio_audit_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_audit_logs.jsonl.zst' FORMAT JSONEachRow" +clickhouse-client -q "SELECT log FROM minio_server_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_server_logs.jsonl.zst' FORMAT JSONEachRow" + # Stop server so we can safely read data with clickhouse-local. # Why do we read data with clickhouse-local? # Because it's the simplest way to read it when server has crashed. From bf111b65fdcaa333680cf64ad9d32fe9493b182e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Aug 2024 17:48:46 +0200 Subject: [PATCH 405/644] 03201_avro_negative_block_size_arrays is parallelizable --- .../0_stateless/03201_avro_negative_block_size_arrays.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03201_avro_negative_block_size_arrays.sh b/tests/queries/0_stateless/03201_avro_negative_block_size_arrays.sh index dcecd7b3bea..f7101989377 100755 --- a/tests/queries/0_stateless/03201_avro_negative_block_size_arrays.sh +++ b/tests/queries/0_stateless/03201_avro_negative_block_size_arrays.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest +# no-fasttest: Requires libraries set -e From ff2e8b65bd5b45112e4aeaf03de089d68019a90a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Aug 2024 17:58:44 +0200 Subject: [PATCH 406/644] 03171_hashed_dictionary_short_circuit_bug_fix is parallelizable --- .../03171_hashed_dictionary_short_circuit_bug_fix.sql | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.sql b/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.sql index e1b5531a442..6d3a63dbadb 100644 --- a/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.sql +++ b/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel - CREATE TABLE x ( hash_id UInt64, user_result Decimal(3, 2) ) ENGINE = Memory(); CREATE TABLE y ( hash_id UInt64, user_result DECIMAL(18, 6) ) ENGINE = Memory(); From db9ba0188622ddd0677a99ab9d8e0765cb5a6c55 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Aug 2024 17:58:54 +0200 Subject: [PATCH 407/644] Leave some notes --- tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql index bbe701f022b..7bbe2d3c533 100644 --- a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql +++ b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql @@ -1,4 +1,5 @@ -- Tags: no-random-settings, no-object-storage, no-parallel +-- no-parallel: Running `DROP MARK CACHE` can have a big impact on other concurrent tests -- Tag no-object-storage: this test relies on the number of opened files in MergeTree that can differ in object storages SET allow_experimental_dynamic_type = 1; From 09964cee569eebea42e4efd52b2228062f8e8331 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Aug 2024 18:09:06 +0200 Subject: [PATCH 408/644] Fix 03168_query_log_privileges_not_empty --- ...8_query_log_privileges_not_empty.reference | 1 - .../03168_query_log_privileges_not_empty.sh | 36 +++++++++---------- 2 files changed, 16 insertions(+), 21 deletions(-) diff --git a/tests/queries/0_stateless/03168_query_log_privileges_not_empty.reference b/tests/queries/0_stateless/03168_query_log_privileges_not_empty.reference index e3ac97f9945..f4ada41d77d 100644 --- a/tests/queries/0_stateless/03168_query_log_privileges_not_empty.reference +++ b/tests/queries/0_stateless/03168_query_log_privileges_not_empty.reference @@ -1,4 +1,3 @@ -1 3168 8613 [] ['SELECT(a, b) ON default.d_03168_query_log'] [] [] diff --git a/tests/queries/0_stateless/03168_query_log_privileges_not_empty.sh b/tests/queries/0_stateless/03168_query_log_privileges_not_empty.sh index 9abc635a874..b817052e355 100755 --- a/tests/queries/0_stateless/03168_query_log_privileges_not_empty.sh +++ b/tests/queries/0_stateless/03168_query_log_privileges_not_empty.sh @@ -1,32 +1,28 @@ #!/usr/bin/env bash -# Tags: no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -user_name="u_03168_query_log" -table_name="default.d_03168_query_log" +user_name="u_03168_query_log_${CLICKHOUSE_DATABASE}" +table_name="d_03168_query_log" test_query="select a, b from ${table_name}" -${CLICKHOUSE_CLIENT_BINARY} --query "drop user if exists ${user_name}" -${CLICKHOUSE_CLIENT_BINARY} --query "create user ${user_name}" -${CLICKHOUSE_CLIENT_BINARY} --query "drop table if exists ${table_name}" -${CLICKHOUSE_CLIENT_BINARY} --query "create table ${table_name} (a UInt64, b UInt64) order by a" +${CLICKHOUSE_CLIENT} --query "drop user if exists ${user_name}" +${CLICKHOUSE_CLIENT} --query "create user ${user_name}" +${CLICKHOUSE_CLIENT} --query "drop table if exists ${table_name}" +${CLICKHOUSE_CLIENT} --query "create table ${table_name} (a UInt64, b UInt64) order by a" +${CLICKHOUSE_CLIENT} --query "insert into table ${table_name} values (3168, 8613)" -${CLICKHOUSE_CLIENT_BINARY} --query "insert into table ${table_name} values (3168, 8613)" +${CLICKHOUSE_CLIENT} --user ${user_name} --query "${test_query}" 2>&1 >/dev/null | (grep -q "ACCESS_DENIED" || echo "Expected ACCESS_DENIED error not found") -error="$(${CLICKHOUSE_CLIENT_BINARY} --user ${user_name} --query "${test_query}" 2>&1 >/dev/null)" -echo "${error}" | grep -Fc "ACCESS_DENIED" +${CLICKHOUSE_CLIENT} --query "grant select(a, b) on ${table_name} to ${user_name}" +${CLICKHOUSE_CLIENT} --user ${user_name} --query "${test_query}" -${CLICKHOUSE_CLIENT_BINARY} --query "grant select(a, b) on ${table_name} to ${user_name}" +${CLICKHOUSE_CLIENT} --query "system flush logs" +${CLICKHOUSE_CLIENT} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'ExceptionBeforeStart' and current_database = currentDatabase() order by event_time desc limit 1" +${CLICKHOUSE_CLIENT} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'QueryStart' and current_database = currentDatabase() order by event_time desc limit 1" +${CLICKHOUSE_CLIENT} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'QueryFinish' and current_database = currentDatabase() order by event_time desc limit 1" -${CLICKHOUSE_CLIENT_BINARY} --user ${user_name} --query "${test_query}" - -${CLICKHOUSE_CLIENT_BINARY} --query "system flush logs" -${CLICKHOUSE_CLIENT_BINARY} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'ExceptionBeforeStart' and current_database = currentDatabase() order by event_time desc limit 1" -${CLICKHOUSE_CLIENT_BINARY} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'QueryStart' and current_database = currentDatabase() order by event_time desc limit 1" -${CLICKHOUSE_CLIENT_BINARY} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'QueryFinish' and current_database = currentDatabase() order by event_time desc limit 1" - -${CLICKHOUSE_CLIENT_BINARY} --query "drop table ${table_name}" -${CLICKHOUSE_CLIENT_BINARY} --query "drop user ${user_name}" +${CLICKHOUSE_CLIENT} --query "drop table ${table_name}" +${CLICKHOUSE_CLIENT} --query "drop user ${user_name}" From c7b94ca43a4f5051e0b84be5cb0e2bc94a6137e2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Aug 2024 18:11:51 +0200 Subject: [PATCH 409/644] 03164_adapting_parquet_reader_output_size is parallelizable --- .../0_stateless/03164_adapting_parquet_reader_output_size.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql index fa098b64702..e6b13510301 100644 --- a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql +++ b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-parallel, no-random-settings +-- Tags: no-fasttest, no-random-settings set max_insert_threads=1; @@ -22,4 +22,4 @@ CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, c INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() FROM numbers(100000); SELECT max(blockSize()) FROM test_parquet; -DROP TABLE IF EXISTS test_parquet; \ No newline at end of file +DROP TABLE IF EXISTS test_parquet; From 58b15c71d9811315666e0b563870aa404682294e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Aug 2024 18:15:19 +0200 Subject: [PATCH 410/644] 03156_default_multiquery_split is parallelizable --- tests/queries/0_stateless/03156_default_multiquery_split.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03156_default_multiquery_split.sh b/tests/queries/0_stateless/03156_default_multiquery_split.sh index ac64c2d093d..8ba2f46b786 100755 --- a/tests/queries/0_stateless/03156_default_multiquery_split.sh +++ b/tests/queries/0_stateless/03156_default_multiquery_split.sh @@ -1,11 +1,10 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-ordinary-database CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -SQL_FILE_NAME=$"03156_default_multiquery_split_$$.sql" +SQL_FILE_NAME=$"03156_default_multiquery_split_${CLICKHOUSE_DATABASE}.sql" # The old multiquery implementation uses '\n' to split INSERT query segmentation # this case is mainly to test the following situations From a109e78776fd435b9dfb464bf8d229ef319ec0e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Aug 2024 18:17:08 +0200 Subject: [PATCH 411/644] 03148_async_queries_in_query_log_errors is parallelizable --- .../0_stateless/03148_async_queries_in_query_log_errors.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.sh b/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.sh index 2b4b96a9cbf..9c290133bf9 100755 --- a/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.sh +++ b/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 35f827e55a7b1583f3424ac4692203aa851a1818 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Aug 2024 18:19:15 +0200 Subject: [PATCH 412/644] Make 03147_table_function_loop parallelizable --- .../queries/0_stateless/03147_table_function_loop.sql | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/03147_table_function_loop.sql b/tests/queries/0_stateless/03147_table_function_loop.sql index aa3c8e2def5..e10155fe8b3 100644 --- a/tests/queries/0_stateless/03147_table_function_loop.sql +++ b/tests/queries/0_stateless/03147_table_function_loop.sql @@ -3,14 +3,11 @@ SELECT * FROM loop(numbers(3)) LIMIT 10; SELECT * FROM loop (numbers(3)) LIMIT 10 settings max_block_size = 1; -DROP DATABASE IF EXISTS 03147_db; -CREATE DATABASE IF NOT EXISTS 03147_db; -CREATE TABLE 03147_db.t (n Int8) ENGINE=MergeTree ORDER BY n; -INSERT INTO 03147_db.t SELECT * FROM numbers(10); -USE 03147_db; +CREATE TABLE t (n Int8) ENGINE=MergeTree ORDER BY n; +INSERT INTO t SELECT * FROM numbers(10); -SELECT * FROM loop(03147_db.t) LIMIT 15; +SELECT * FROM loop({CLICKHOUSE_DATABASE:Identifier}.t) LIMIT 15; SELECT * FROM loop(t) LIMIT 15; -SELECT * FROM loop(03147_db, t) LIMIT 15; +SELECT * FROM loop({CLICKHOUSE_DATABASE:Identifier}, t) LIMIT 15; SELECT * FROM loop('', '') -- { serverError UNKNOWN_TABLE } From ad678cb5a8d4533a88273b8244dc0844c83e641c Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 7 Aug 2024 18:24:03 +0200 Subject: [PATCH 413/644] Ignore disappeared projections on start --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 33 +++++++++++- .../MergeTree/MergeTreeDataPartChecksum.cpp | 6 --- .../MergeTree/MergeTreeDataPartChecksum.h | 3 -- .../test_broken_projections/test.py | 50 +++++++++++++++++++ 4 files changed, 81 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 3a44359b537..918a4cda714 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -72,6 +72,7 @@ namespace ErrorCodes extern const int BAD_TTL_FILE; extern const int NOT_IMPLEMENTED; extern const int NO_SUCH_COLUMN_IN_TABLE; + extern const int FILE_DOESNT_EXIST; } @@ -749,8 +750,16 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks /// Probably there is something wrong with files of this part. /// So it can be helpful to add to the error message some information about those files. String files_in_part; + for (auto it = getDataPartStorage().iterate(); it->isValid(); it->next()) - files_in_part += fmt::format("{}{} ({} bytes)", (files_in_part.empty() ? "" : ", "), it->name(), getDataPartStorage().getFileSize(it->name())); + { + std::string file_info; + if (!getDataPartStorage().isDirectory(it->name())) + file_info = fmt::format(" ({} bytes)", getDataPartStorage().getFileSize(it->name())); + + files_in_part += fmt::format("{}{}{}", (files_in_part.empty() ? "" : ", "), it->name(), file_info); + + } if (!files_in_part.empty()) e->addMessage("Part contains files: {}", files_in_part); if (isEmpty()) @@ -2141,7 +2150,27 @@ void IMergeTreeDataPart::checkConsistencyBase() const } } - checksums.checkSizes(getDataPartStorage()); + const auto & data_part_storage = getDataPartStorage(); + for (const auto & [filename, checksum] : checksums.files) + { + try + { + checksum.checkSize(data_part_storage, filename); + } + catch (const Exception & ex) + { + /// For projection parts check will mark them broken in loadProjections + if (!parent_part && filename.ends_with(".proj")) + { + std::string projection_name = fs::path(filename).stem(); + LOG_INFO(storage.log, "Projection {} doesn't exist on start for part {}, marking it as broken", projection_name, name); + if (hasProjection(projection_name)) + markProjectionPartAsBroken(projection_name, ex.message(), ex.code()); + } + else + throw; + } + } } else { diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index b327480fa92..3ef36ce364c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -100,12 +100,6 @@ void MergeTreeDataPartChecksums::checkEqual(const MergeTreeDataPartChecksums & r } } -void MergeTreeDataPartChecksums::checkSizes(const IDataPartStorage & storage) const -{ - for (const auto & [name, checksum] : files) - checksum.checkSize(storage, name); -} - UInt64 MergeTreeDataPartChecksums::getTotalSizeOnDisk() const { UInt64 res = 0; diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.h b/src/Storages/MergeTree/MergeTreeDataPartChecksum.h index 05178dc3a60..dc52f1ada2b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.h +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.h @@ -65,9 +65,6 @@ struct MergeTreeDataPartChecksums static bool isBadChecksumsErrorCode(int code); - /// Checks that the directory contains all the needed files of the correct size. Does not check the checksum. - void checkSizes(const IDataPartStorage & storage) const; - /// Returns false if the checksum is too old. bool read(ReadBuffer & in); /// Assume that header with version (the first line) is read diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index 162c0dbaa2f..578ff42369c 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -4,6 +4,7 @@ import logging import string import random from helpers.cluster import ClickHouseCluster +from multiprocessing.dummy import Pool cluster = ClickHouseCluster(__file__) @@ -18,6 +19,12 @@ def cluster(): stay_alive=True, with_zookeeper=True, ) + cluster.add_instance( + "node_restart", + main_configs=["config.d/dont_start_broken.xml"], + stay_alive=True, + with_zookeeper=True, + ) logging.info("Starting cluster...") cluster.start() @@ -632,6 +639,49 @@ def test_broken_on_start(cluster): check(node, table_name, 0) +def test_disappeared_projection_on_start(cluster): + node = cluster.instances["node_restart"] + + table_name = "test_disapperead_projection" + create_table(node, table_name, 1) + + node.query(f"SYSTEM STOP MERGES {table_name}") + + insert(node, table_name, 0, 5) + insert(node, table_name, 5, 5) + insert(node, table_name, 10, 5) + insert(node, table_name, 15, 5) + + assert ["all_0_0_0", "all_1_1_0", "all_2_2_0", "all_3_3_0"] == get_parts( + node, table_name + ) + + def drop_projection(): + node.query( + f"ALTER TABLE {table_name} DROP PROJECTION proj2", + settings={"mutations_sync": "0"}, + ) + + p = Pool(2) + p.apply_async(drop_projection) + + for i in range(30): + create_query = node.query(f"SHOW CREATE TABLE {table_name}") + if "proj2" not in create_query: + break + time.sleep(0.5) + + assert "proj2" not in create_query + + # Remove 'proj2' for part all_2_2_0 + break_projection(node, table_name, "proj2", "all_2_2_0", "part") + + node.restart_clickhouse() + + # proj2 is not broken, it doesn't exist, but ok + check(node, table_name, 0, expect_broken_part="proj2", do_check_command=0) + + def test_mutation_with_broken_projection(cluster): node = cluster.instances["node"] From 0dc4d773edd530494c1ab514d104a45665bdd16a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 7 Aug 2024 18:46:34 +0200 Subject: [PATCH 414/644] Fxi style --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 918a4cda714..93904c1a838 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -72,7 +72,6 @@ namespace ErrorCodes extern const int BAD_TTL_FILE; extern const int NOT_IMPLEMENTED; extern const int NO_SUCH_COLUMN_IN_TABLE; - extern const int FILE_DOESNT_EXIST; } From 06d154055f9e233180f13585e43e2992ae5ccfdf Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 7 Aug 2024 19:19:33 +0200 Subject: [PATCH 415/644] adjust tests --- tests/integration/test_disk_configuration/test.py | 2 +- tests/queries/0_stateless/03008_s3_plain_rewritable.sh | 7 ++++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_disk_configuration/test.py b/tests/integration/test_disk_configuration/test.py index afc5303298c..f297c665dc5 100644 --- a/tests/integration/test_disk_configuration/test.py +++ b/tests/integration/test_disk_configuration/test.py @@ -373,7 +373,7 @@ def test_merge_tree_setting_override(start_cluster): CREATE TABLE {TABLE_NAME} (a Int32) ENGINE = MergeTree() ORDER BY tuple() - SETTINGS disk = 'kek', storage_policy = 's3'; + SETTINGS disk = 's3', storage_policy = 's3'; """ ) ) diff --git a/tests/queries/0_stateless/03008_s3_plain_rewritable.sh b/tests/queries/0_stateless/03008_s3_plain_rewritable.sh index 4d5989f6f12..8eea7940774 100755 --- a/tests/queries/0_stateless/03008_s3_plain_rewritable.sh +++ b/tests/queries/0_stateless/03008_s3_plain_rewritable.sh @@ -46,7 +46,12 @@ ${CLICKHOUSE_CLIENT} --query "drop table if exists test_s3_mt_dst" ${CLICKHOUSE_CLIENT} -m --query " create table test_s3_mt_dst (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b) -settings disk = '03008_s3_plain_rewritable' +settings disk = disk( + name = 03008_s3_plain_rewritable, + type = s3_plain_rewritable, + endpoint = 'http://localhost:11111/test/03008_test_s3_mt/', + access_key_id = clickhouse, + secret_access_key = clickhouse); " ${CLICKHOUSE_CLIENT} -m --query " From d992431f96f4374d7bb7d15be51c3569dedcb4d4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Aug 2024 17:27:24 +0000 Subject: [PATCH 416/644] Make tests more sturdier for flaky test check --- tests/integration/test_storage_kafka/test.py | 33 +++++++++++++++----- 1 file changed, 26 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 3f71866c913..39796fa3ba5 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1161,8 +1161,27 @@ def kafka_cluster(): @pytest.fixture(autouse=True) def kafka_setup_teardown(): - instance.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;") - # logging.debug("kafka is available - running test") + instance.query("DROP DATABASE IF EXISTS test SYNC; CREATE DATABASE test;") + admin_client = get_admin_client(cluster) + def get_topics_to_delete(): + return [t for t in admin_client.list_topics() if not t.startswith("_")] + topics = get_topics_to_delete() + logging.debug(f"Deleting topics: {topics}") + result = admin_client.delete_topics(topics) + for topic, error in result.topic_error_codes: + if error != 0: + logging.warning(f"Received error {error} while deleting topic {topic}") + else: + logging.info(f"Deleted topic {topic}") + + retries = 0 + topics = get_topics_to_delete() + while (len(topics) != 0): + logging.info(f"Existing topics: {topics}") + if retries >= 5: + raise Exception(f"Failed to delete topics {topics}") + retries += 1 + time.sleep(0.5) yield # run test @@ -3286,7 +3305,7 @@ def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster, create_query_gen ) instance.query( f""" - DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.view SYNC; DROP TABLE IF EXISTS test.consumer; {create_query}; @@ -3329,7 +3348,7 @@ def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster, create_query_gen instance.query( """ DROP TABLE test.consumer; - DROP TABLE test.view; + DROP TABLE test.view SYNC; """ ) @@ -5381,7 +5400,7 @@ def test_multiple_read_in_materialized_views(kafka_cluster, create_query_generat ) instance.query( f""" - DROP TABLE IF EXISTS test.kafka_multiple_read_input; + DROP TABLE IF EXISTS test.kafka_multiple_read_input SYNC; DROP TABLE IF EXISTS test.kafka_multiple_read_table; DROP TABLE IF EXISTS test.kafka_multiple_read_mv; @@ -5496,9 +5515,9 @@ def test_kafka_null_message(kafka_cluster, create_query_generator): instance.query( """ - DROP TABLE test.null_message_consumer; + DROP TABLE test.null_message_consumer SYNC; DROP TABLE test.null_message_view; - DROP TABLE test.null_message_kafka; + DROP TABLE test.null_message_kafka SYNC; """ ) From ec3a248e70e16d2cb4db1ec5ff17e95cc11dedae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Aug 2024 17:37:54 +0000 Subject: [PATCH 417/644] Fix clang-tidy --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b24d7968b61..5ee0bd328e0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1146,7 +1146,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( auto metadata_snapshot = getInMemoryMetadataPtr(); auto virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, {parts[0]}); - auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag.getOutputs().at(0), nullptr, /*allow_non_deterministic_functions=*/ false); + auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag.getOutputs().at(0), nullptr, /*allow_partial_result=*/ false); if (!filter_dag) return {}; From 47270449dcdbd056432aa4d837ccb33917fc1a9f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Aug 2024 17:39:09 +0000 Subject: [PATCH 418/644] Style fix --- src/Storages/Kafka/KafkaConsumer2.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp index dc71086db3b..60626dfa402 100644 --- a/src/Storages/Kafka/KafkaConsumer2.cpp +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -352,7 +352,8 @@ void KafkaConsumer2::subscribeIfNotSubscribedYet() ReadBufferPtr KafkaConsumer2::getNextMessage() { - while (current != messages.end()) { + while (current != messages.end()) + { const auto * data = current->get_payload().get_data(); size_t size = current->get_payload().get_size(); ++current; From 94398996b2a002f5f08e3f97c2544e7a0f712f59 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 7 Aug 2024 17:47:15 +0000 Subject: [PATCH 419/644] Automatic style fix --- tests/integration/test_storage_kafka/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 39796fa3ba5..4b6c9922d74 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1163,8 +1163,10 @@ def kafka_cluster(): def kafka_setup_teardown(): instance.query("DROP DATABASE IF EXISTS test SYNC; CREATE DATABASE test;") admin_client = get_admin_client(cluster) + def get_topics_to_delete(): return [t for t in admin_client.list_topics() if not t.startswith("_")] + topics = get_topics_to_delete() logging.debug(f"Deleting topics: {topics}") result = admin_client.delete_topics(topics) @@ -1176,7 +1178,7 @@ def kafka_setup_teardown(): retries = 0 topics = get_topics_to_delete() - while (len(topics) != 0): + while len(topics) != 0: logging.info(f"Existing topics: {topics}") if retries >= 5: raise Exception(f"Failed to delete topics {topics}") From cf8ddbc15e0bb1143ff1737aec80171b518b24bf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Mi=D1=81hael=20Stetsyuk?= <59827607+mstetsyuk@users.noreply.github.com> Date: Wed, 7 Aug 2024 19:03:20 +0100 Subject: [PATCH 420/644] Update src/Databases/DatabaseReplicated.cpp Co-authored-by: Alexander Tokmakov --- src/Databases/DatabaseReplicated.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 213c94d4d94..09dd2065b19 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -390,7 +390,8 @@ ReplicasInfo DatabaseReplicated::tryGetReplicasInfo(const ClusterPtr & cluster_) } return replicas_info; - } catch (...) + } + catch (...) { tryLogCurrentException(log); return {}; From 7341dcefd44a1ee38a68e11604db7dfae8a32882 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Wed, 7 Aug 2024 20:36:23 +0200 Subject: [PATCH 421/644] Follow up for #67843 --- tests/integration/test_access_for_functions/test.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/integration/test_access_for_functions/test.py b/tests/integration/test_access_for_functions/test.py index 52777c60729..82125b35b49 100644 --- a/tests/integration/test_access_for_functions/test.py +++ b/tests/integration/test_access_for_functions/test.py @@ -80,5 +80,3 @@ EOF""", instance.query(f"SHOW GRANTS FOR `{user_id}`") == f"GRANT SELECT ON mydb.* TO `{user_id}`\n" ) - instance.stop_clickhouse() - instance.start_clickhouse() From 466944683bf9c5e1d9dcb3d91c24ab9bf896a791 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 7 Aug 2024 18:50:19 +0000 Subject: [PATCH 422/644] fix for multiple WITH --- src/Interpreters/AddDefaultDatabaseVisitor.h | 7 +++-- .../03215_view_with_recursive.reference | 1 + .../0_stateless/03215_view_with_recursive.sql | 28 +++++++++++++++++++ 3 files changed, 33 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/AddDefaultDatabaseVisitor.h b/src/Interpreters/AddDefaultDatabaseVisitor.h index 5e46a653efa..a28c7c1bff3 100644 --- a/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -101,7 +101,7 @@ private: const String database_name; std::set external_tables; - mutable String with_alias; + mutable std::unordered_set with_aliases; bool only_replace_current_database_function = false; bool only_replace_in_join = false; @@ -120,7 +120,8 @@ private: void visit(ASTSelectQuery & select, ASTPtr &) const { if (select.recursive_with) - with_alias = select.with()->children[0]->as()->name; + for (const auto & child : select.with()->children) + with_aliases.insert(child->as()->name); if (select.tables()) tryVisit(select.refTables()); @@ -171,7 +172,7 @@ private: if (external_tables.contains(identifier.shortName())) return; /// This is WITH RECURSIVE alias. - if (!with_alias.empty() && identifier.name() == with_alias) + if (with_aliases.contains(identifier.name())) return; auto qualified_identifier = std::make_shared(database_name, identifier.name()); diff --git a/tests/queries/0_stateless/03215_view_with_recursive.reference b/tests/queries/0_stateless/03215_view_with_recursive.reference index c3ac783e702..c3ca8065a70 100644 --- a/tests/queries/0_stateless/03215_view_with_recursive.reference +++ b/tests/queries/0_stateless/03215_view_with_recursive.reference @@ -1 +1,2 @@ 5050 +8 diff --git a/tests/queries/0_stateless/03215_view_with_recursive.sql b/tests/queries/0_stateless/03215_view_with_recursive.sql index ef7908612af..5d93ccc5438 100644 --- a/tests/queries/0_stateless/03215_view_with_recursive.sql +++ b/tests/queries/0_stateless/03215_view_with_recursive.sql @@ -13,3 +13,31 @@ SELECT sum(number) FROM test_table; SELECT * FROM 03215_test_v; + +CREATE VIEW 03215_multi_v +AS WITH RECURSIVE + task AS + ( + SELECT + number AS task_id, + number - 1 AS parent_id + FROM numbers(10) + ), + rtq AS + ( + SELECT + task_id, + parent_id + FROM task AS t + WHERE t.parent_id = 1 + UNION ALL + SELECT + t.task_id, + t.parent_id + FROM task AS t, rtq AS r + WHERE t.parent_id = r.task_id + ) +SELECT count() +FROM rtq; + +SELECT * FROM 03215_multi_v; From 1082792950ca7b962c1288ab49bb8ff3ca855bbe Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 7 Aug 2024 20:21:50 +0100 Subject: [PATCH 423/644] fix test --- .../test_async_metrics_in_cgroup/test.py | 98 +++++++++---------- 1 file changed, 45 insertions(+), 53 deletions(-) diff --git a/tests/integration/test_async_metrics_in_cgroup/test.py b/tests/integration/test_async_metrics_in_cgroup/test.py index 00951c95a0e..d9f2e3aaaed 100644 --- a/tests/integration/test_async_metrics_in_cgroup/test.py +++ b/tests/integration/test_async_metrics_in_cgroup/test.py @@ -1,11 +1,10 @@ import pytest -import subprocess -import time from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node") +node1 = cluster.add_instance("node1", stay_alive=True) +node2 = cluster.add_instance("node2", stay_alive=True) @pytest.fixture(scope="module") @@ -17,61 +16,54 @@ def start_cluster(): cluster.shutdown() -def test_user_cpu_accounting(start_cluster): - if node.is_built_with_sanitizer(): - pytest.skip("Disabled for sanitizers") - - # check that our metrics sources actually exist - assert ( - subprocess.Popen("test -f /sys/fs/cgroup/cpu.stat".split(" ")).wait() == 0 - or subprocess.Popen( - "test -f /sys/fs/cgroup/cpuacct/cpuacct.stat".split(" ") - ).wait() - == 0 - ) - - # first let's spawn some cpu-intensive process outside of the container and check that it doesn't accounted by ClickHouse server - proc = subprocess.Popen( - "openssl speed -multi 8".split(" "), - stdout=subprocess.DEVNULL, - stderr=subprocess.DEVNULL, - ) - - time.sleep(5) - - metric = node.query( - """ - SELECT max(value) - FROM ( - SELECT toStartOfInterval(event_time, toIntervalSecond(1)) AS t, avg(value) AS value - FROM system.asynchronous_metric_log - WHERE event_time >= now() - 60 AND metric = 'OSUserTime' - GROUP BY t - ) - """ - ).strip("\n") - - assert float(metric) < 2 - - proc.kill() - - # then let's test that we will account cpu time spent by the server itself +def run_cpu_intensive_task(node): node.query( - "SELECT cityHash64(*) FROM system.numbers_mt FORMAT Null SETTINGS max_execution_time=10", + "SELECT sum(*) FROM system.numbers_mt FORMAT Null SETTINGS max_execution_time=10", ignore_error=True, ) - metric = node.query( + +def get_async_metric(node, metric): + node.query("SYSTEM FLUSH LOGS") + return node.query( + f""" + SELECT max(value) + FROM ( + SELECT toStartOfInterval(event_time, toIntervalSecond(1)) AS t, avg(value) AS value + FROM system.asynchronous_metric_log + WHERE event_time >= now() - 60 AND metric = '{metric}' + GROUP BY t + ) + SETTINGS max_threads = 1 """ - SELECT max(value) - FROM ( - SELECT toStartOfInterval(event_time, toIntervalSecond(1)) AS t, avg(value) AS value - FROM system.asynchronous_metric_log - WHERE event_time >= now() - 60 AND metric = 'OSUserTime' - GROUP BY t - ) - """ ).strip("\n") + +def test_user_cpu_accounting(start_cluster): + if node1.is_built_with_sanitizer(): + pytest.skip("Disabled for sanitizers") + + # run query on the other node, its usage shouldn't be accounted by node1 + run_cpu_intensive_task(node2) + + node1_cpu_time = get_async_metric(node1, "OSUserTime") + assert float(node1_cpu_time) < 2 + + # then let's test that we will account cpu time spent by the server itself + node2_cpu_time = get_async_metric(node2, "OSUserTime") # this check is really weak, but CI is tough place and we cannot guarantee that test process will get many cpu time - assert float(metric) > 1 + assert float(node2_cpu_time) > 2 + + +def test_normalized_user_cpu(start_cluster): + if node1.is_built_with_sanitizer(): + pytest.skip("Disabled for sanitizers") + + # run query on the other node, its usage shouldn't be accounted by node1 + run_cpu_intensive_task(node2) + + node1_cpu_time = get_async_metric(node1, "OSUserTimeNormalized") + assert float(node1_cpu_time) < 1.01 + + node2_cpu_time = get_async_metric(node2, "OSUserTimeNormalized") + assert float(node2_cpu_time) < 1.01 From d81b5239debaf01b74521511db44d6cb4cd419c4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 7 Aug 2024 21:37:01 +0200 Subject: [PATCH 424/644] Remove unused CLI option --- src/Client/ClientApplicationBase.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp index 9f133616d2e..71d13ad4f53 100644 --- a/src/Client/ClientApplicationBase.cpp +++ b/src/Client/ClientApplicationBase.cpp @@ -200,8 +200,6 @@ void ClientApplicationBase::init(int argc, char ** argv) ("pager", po::value(), "Pipe all output into this command (less or similar)") ("max_memory_usage_in_client", po::value(), "Set memory limit in client/local server") - ("fuzzer-args", po::value(), "Command line arguments for the LLVM's libFuzzer driver. Only relevant if the application is compiled with libFuzzer.") - ("client_logs_file", po::value(), "Path to a file for writing client logs. Currently we only have fatal logs (when the client crashes)") ; From f08cb90fe3f017d44e4dd58c7e696396d6bf5ac0 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Wed, 7 Aug 2024 19:54:55 +0000 Subject: [PATCH 425/644] fxs --- src/Databases/DatabaseReplicated.cpp | 9 ++++++++- src/Storages/System/StorageSystemClusters.cpp | 6 +++--- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 09dd2065b19..fe00c1c60aa 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -379,10 +379,17 @@ ReplicasInfo DatabaseReplicated::tryGetReplicasInfo(const ClusterPtr & cluster_) auto replica_active = zk_res[2 * global_replica_index + 1]; auto replica_log_ptr = zk_res[2 * global_replica_index + 2]; + UInt64 recovery_time = 0; + { + std::lock_guard lock(ddl_worker_mutex); + if (replica.is_local && ddl_worker) + recovery_time = ddl_worker->getCurrentInitializationDurationMs(); + } + replicas_info[global_replica_index] = ReplicaInfo{ .is_active = replica_active.error == Coordination::Error::ZOK, .replication_lag = replica_log_ptr.error != Coordination::Error::ZNONODE ? std::optional(max_log_ptr - parse(replica_log_ptr.data)) : std::nullopt, - .recovery_time = replica.is_local && ddl_worker ? ddl_worker->getCurrentInitializationDurationMs() : 0, + .recovery_time = recovery_time, }; ++global_replica_index; diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index db1955c2e99..9493d2c97ab 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -70,8 +70,9 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const std const auto & shards_info = cluster->getShardsInfo(); const auto & addresses_with_failover = cluster->getShardsAddresses(); + size_t recovery_time_column_idx = columns_mask.size() - 1, replication_lag_column_idx = columns_mask.size() - 2, is_active_column_idx = columns_mask.size() - 3; ReplicasInfo replicas_info; - if (replicated) + if (replicated && (columns_mask[recovery_time_column_idx] || columns_mask[replication_lag_column_idx] || columns_mask[is_active_column_idx])) replicas_info = replicated->tryGetReplicasInfo(name_and_cluster.second); size_t replica_idx = 0; @@ -122,6 +123,7 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const std if (columns_mask[src_index++]) res_columns[res_index++]->insert(address.database_replica_name); + /// make sure these three columns remain the last ones if (columns_mask[src_index++]) { if (replicas_info.empty()) @@ -132,7 +134,6 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const std res_columns[res_index++]->insert(replica_info.is_active); } } - if (columns_mask[src_index++]) { if (replicas_info.empty()) @@ -146,7 +147,6 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const std res_columns[res_index++]->insertDefault(); } } - if (columns_mask[src_index++]) { if (replicas_info.empty()) From 9fdc746c44499e819649a0298755cae7b02c23e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 7 Aug 2024 22:23:21 +0200 Subject: [PATCH 426/644] Fix test `02845_threads_count_in_distributed_queries` --- .../02845_threads_count_in_distributed_queries.sql.j2 | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 index ffdd4e3400e..7d751eb8f17 100644 --- a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 +++ b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 @@ -1,3 +1,7 @@ +-- Tags: no-parallel, no-fasttest +-- ^ because query_thread_log is not guaranteed to be written under high load +-- (when the queue is full, events are silently dropped) + -- enforce some defaults to be sure that the env settings will not affect the test SET max_threads=5, async_socket_for_remote=1, prefer_localhost_replica=1, optimize_read_in_order=1, load_marks_asynchronously=0, local_filesystem_read_method='pread', remote_filesystem_read_method='read'; From b2722d883282eaea7f5d57d962b8f9acc884ce05 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 7 Aug 2024 20:54:40 +0000 Subject: [PATCH 427/644] Disallow LowCardinality type for external tables --- src/Storages/StorageTimeSeries.cpp | 13 ++++++++++++- tests/integration/test_prometheus_protocols/test.py | 10 +++++++--- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageTimeSeries.cpp b/src/Storages/StorageTimeSeries.cpp index f5a2c0c59a2..3ff57aaf3e5 100644 --- a/src/Storages/StorageTimeSeries.cpp +++ b/src/Storages/StorageTimeSeries.cpp @@ -155,7 +155,18 @@ StorageTimeSeries::StorageTimeSeries( auto & target = targets.emplace_back(); target.kind = target_kind; target.table_id = initTarget(target_kind, target_info, local_context, getStorageID(), columns, *storage_settings, mode); - target.is_inner_table = target_info->table_id.empty(); + target.is_inner_table = target_info && target_info->table_id.empty(); + + if (target_kind == ViewTarget::Metrics && !target.is_inner_table) + { + auto table = DatabaseCatalog::instance().tryGetTable(target.table_id, getContext()); + auto metadata = table->getInMemoryMetadataPtr(); + + for (const auto & column : metadata->columns) + if (column.type->lowCardinality()) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "External metrics table cannot have LowCardnality columns for now."); + } + has_inner_tables |= target.is_inner_table; } } diff --git a/tests/integration/test_prometheus_protocols/test.py b/tests/integration/test_prometheus_protocols/test.py index 6adb3da56c3..0c75a8194c7 100644 --- a/tests/integration/test_prometheus_protocols/test.py +++ b/tests/integration/test_prometheus_protocols/test.py @@ -60,19 +60,19 @@ def show_query_result(query): def compare_query(query): - timeout = 30 + timeout = 60 start_time = time.time() evaluation_time = start_time print(f"Evaluating query: {query}") print(f"Evaluation time: {evaluation_time}") while time.time() < start_time + timeout: result_from_writer = execute_query_on_prometheus_writer(query, evaluation_time) + time.sleep(1) result_from_reader = execute_query_on_prometheus_reader(query, evaluation_time) print(f"Result from prometheus_writer: {result_from_writer}") print(f"Result from prometheus_reader: {result_from_reader}") if result_from_writer == result_from_reader: return - time.sleep(1) raise Exception( f"Got different results from prometheus_writer and prometheus_reader" ) @@ -162,8 +162,12 @@ def test_external_tables(): "max_time SimpleAggregateFunction(max, Nullable(DateTime64(3)))) " "ENGINE=AggregatingMergeTree ORDER BY (metric_name, id)" ) + + # FIXME: The table structure should be: + # "CREATE TABLE mymetrics (metric_family_name String, type LowCardinality(String), unit LowCardinality(String), help String)" + # Renamed it because of the bug and potential type mismatch. node.query( - "CREATE TABLE mymetrics (metric_family_name String, type LowCardinality(String), unit LowCardinality(String), help String) " + "CREATE TABLE mymetrics (metric_family_name String, type String, unit String, help String) " "ENGINE=ReplacingMergeTree ORDER BY metric_family_name" ) node.query( From 85427030aaac223559c0ce6ca06dfa01d4345c21 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 7 Aug 2024 10:01:41 +0200 Subject: [PATCH 428/644] tests: fix 03002_part_log_rmt_fetch_merge_error flakiness CI found [1]: --- /usr/share/clickhouse-test/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.reference 2024-08-07 05:09:42.058643403 +0200 +++ /tmp/clickhouse-test/0_stateless/03002_part_log_rmt_fetch_merge_error.stdout 2024-08-07 05:54:45.514083455 +0200 @@ -6,5 +6,7 @@ after rmt_master NewPart 0 1 rmt_master MergeParts 0 1 +rmt_master RemovePart 0 1 rmt_slave MergeParts 1 0 rmt_slave DownloadPart 0 2 +rmt_slave RemovePart 0 1 MergeTree settings used in test: --ratio_of_defaults_for_sparse_serialization 1.0 --prefer_fetch_merged_part_size_threshold 3517855074 --vertical_merge_algorithm_min_rows_to_activate 1000000 --vertical_merge_algorithm_min_columns_to_activate 100 --allow_vertical_merges_from_compact_to_wide_parts 0 --min_merge_bytes_to_use_direct_io 10737418240 --index_granularity_bytes 7659983 --merge_max_block_size 17667 --index_granularity 48465 --min_bytes_for_wide_part 1073741824 --marks_compress_block_size 58048 --primary_key_compress_block_size 18342 --replace_long_file_name_to_hash 0 --max_file_name_length 36 --min_bytes_for_full_part_storage 536870912 --compact_parts_max_bytes_to_buffer 148846831 --compact_parts_max_granules_to_buffer 140 --compact_parts_merge_max_bytes_to_prefetch_part 4513530 --cache_populated_by_fetch 1 --concurrent_part_removal_threshold 8 --old_parts_lifetime 10 The reason is old_parts_lifetime=10 [1]: https://s3.amazonaws.com/clickhouse-test-reports/67511/881d57a7644057b586e4cdb95ebb8785d912d4c5/stateless_tests__msan__%5B3_4%5D.html Signed-off-by: Azat Khuzhin --- .../0_stateless/03002_part_log_rmt_fetch_merge_error.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh index 25d946b325d..e58c542b8ac 100755 --- a/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh +++ b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh @@ -21,9 +21,9 @@ $CLICKHOUSE_CLIENT -nm -q " drop table if exists rmt_master; drop table if exists rmt_slave; - create table rmt_master (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', 'master') order by key settings always_fetch_merged_part=0; + create table rmt_master (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', 'master') order by key settings always_fetch_merged_part=0, old_parts_lifetime=600; -- always_fetch_merged_part=1, consider this table as a 'slave' - create table rmt_slave (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', 'slave') order by key settings always_fetch_merged_part=1; + create table rmt_slave (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', 'slave') order by key settings always_fetch_merged_part=1, old_parts_lifetime=600; insert into rmt_master values (1); From 1dece979fe317a04f98d2b8008619c47fb72edb1 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 7 Aug 2024 23:10:40 +0200 Subject: [PATCH 429/644] CI: pass job timout into tests --- tests/ci/ci.py | 1 + tests/ci/ci_definitions.py | 2 +- tests/ci/functional_test_check.py | 3 +++ 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 8d0414ce7a8..49b597333dc 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -985,6 +985,7 @@ def _run_test(job_name: str, run_command: str) -> int: else: print("Use run command from the workflow") env["CHECK_NAME"] = job_name + env["MAX_RUN_TIME"] = str(timeout or 0) print(f"Going to start run command [{run_command}]") stopwatch = Stopwatch() job_log = Path(TEMP_PATH) / "job_log.txt" diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 48847b0d7a6..592cb2f4879 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -420,7 +420,7 @@ class CommonJobConfigs: ), run_command='functional_test_check.py "$CHECK_NAME"', runner_type=Runners.FUNC_TESTER, - timeout=9000, + timeout=1000, # test ) STATEFUL_TEST = JobConfig( job_name_keyword="stateful", diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 52970404d2d..3aff97643c3 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -114,6 +114,9 @@ def get_run_command( if flaky_check: envs.append("-e NUM_TRIES=50") envs.append("-e MAX_RUN_TIME=2800") + else: + max_run_time = os.getenv("MAX_RUN_TIME", 0) + envs.append(f"-e MAX_RUN_TIME={max_run_time}") envs += [f"-e {e}" for e in additional_envs] From 086b3d240dd696c483f136d79db4587a83bb0a14 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 7 Aug 2024 23:34:36 +0200 Subject: [PATCH 430/644] CI: push CI --- tests/ci/ci_definitions.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 592cb2f4879..b62d2e0aa8e 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -420,7 +420,7 @@ class CommonJobConfigs: ), run_command='functional_test_check.py "$CHECK_NAME"', runner_type=Runners.FUNC_TESTER, - timeout=1000, # test + timeout=1001, # test ) STATEFUL_TEST = JobConfig( job_name_keyword="stateful", From c6c0a44b93c382b384eb3ef83cf9da5102629de8 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 7 Aug 2024 23:57:19 +0200 Subject: [PATCH 431/644] fix flaky tests --- .../test_storage_azure_blob_storage/test.py | 2 +- tests/integration/test_storage_hdfs/test.py | 51 +++++++------------ 2 files changed, 19 insertions(+), 34 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 092c124855c..fbdc7f29f98 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -1272,7 +1272,7 @@ def test_filtering_by_file_or_path(cluster): node.query("SYSTEM FLUSH LOGS") result = node.query( - f"SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query ilike '%select%azure%test_filter%' AND type='QueryFinish'" + f"SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query ilike '%select%azure%test_filter%' AND type='QueryFinish' ORDER BY event_time_microseconds DESC LIMIT 1" ) assert int(result) == 1 diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 77921b885b0..c52e99b800e 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -41,7 +41,6 @@ def test_read_write_storage(started_cluster): node1.query("insert into SimpleHDFSStorage values (1, 'Mark', 72.53)") assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" assert node1.query("select * from SimpleHDFSStorage") == "1\tMark\t72.53\n" - node1.query("drop table if exists SimpleHDFSStorage") def test_read_write_storage_with_globs(started_cluster): @@ -95,11 +94,6 @@ def test_read_write_storage_with_globs(started_cluster): print(ex) assert "in readonly mode" in str(ex) - node1.query("DROP TABLE HDFSStorageWithRange") - node1.query("DROP TABLE HDFSStorageWithEnum") - node1.query("DROP TABLE HDFSStorageWithQuestionMark") - node1.query("DROP TABLE HDFSStorageWithAsterisk") - def test_storage_with_multidirectory_glob(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -341,7 +335,6 @@ def test_virtual_columns(started_cluster): ) == expected ) - node1.query("DROP TABLE virtual_cols") def test_read_files_with_spaces(started_cluster): @@ -363,7 +356,6 @@ def test_read_files_with_spaces(started_cluster): ) assert node1.query("select * from test order by id") == "1\n2\n3\n" fs.delete(dir, recursive=True) - node1.query("DROP TABLE test") def test_truncate_table(started_cluster): @@ -435,7 +427,7 @@ def test_seekable_formats(started_cluster): f"hdfs('hdfs://hdfs1:9000/parquet', 'Parquet', 'a Int32, b String')" ) node1.query( - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000) SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000)" ) result = node1.query(f"SELECT count() FROM {table_function}") @@ -443,7 +435,7 @@ def test_seekable_formats(started_cluster): table_function = f"hdfs('hdfs://hdfs1:9000/orc', 'ORC', 'a Int32, b String')" node1.query( - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000) SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000)" ) result = node1.query(f"SELECT count() FROM {table_function}") assert int(result) == 5000000 @@ -467,7 +459,7 @@ def test_read_table_with_default(started_cluster): def test_schema_inference(started_cluster): node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/native', 'Native', 'a Int32, b String') SELECT number, randomString(100) FROM numbers(5000000) SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function hdfs('hdfs://hdfs1:9000/native', 'Native', 'a Int32, b String') SELECT number, randomString(100) FROM numbers(5000000)" ) result = node1.query(f"desc hdfs('hdfs://hdfs1:9000/native', 'Native')") @@ -520,7 +512,6 @@ def test_hdfs_directory_not_exist(started_cluster): assert "" == node1.query( "select * from HDFSStorageWithNotExistDir settings hdfs_ignore_file_doesnt_exist=1" ) - node1.query("DROP TABLE HDFSStorageWithNotExistDir") def test_overwrite(started_cluster): @@ -540,7 +531,6 @@ def test_overwrite(started_cluster): result = node1.query(f"select count() from test_overwrite") assert int(result) == 10 - node1.query(f"DROP TABLE test_overwrite") def test_multiple_inserts(started_cluster): @@ -577,7 +567,6 @@ def test_multiple_inserts(started_cluster): result = node1.query(f"select count() from test_multiple_inserts") assert int(result) == 60 - node1.query(f"DROP TABLE test_multiple_inserts") def test_format_detection(started_cluster): @@ -591,10 +580,10 @@ def test_format_detection(started_cluster): def test_schema_inference_with_globs(started_cluster): node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data1.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function hdfs('hdfs://hdfs1:9000/data1.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL" ) node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data2.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select 0 SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function hdfs('hdfs://hdfs1:9000/data2.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select 0" ) result = node1.query( @@ -608,7 +597,7 @@ def test_schema_inference_with_globs(started_cluster): assert sorted(result.split()) == ["0", "\\N"] node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data3.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function hdfs('hdfs://hdfs1:9000/data3.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL" ) filename = "data{1,3}.jsoncompacteachrow" @@ -620,7 +609,7 @@ def test_schema_inference_with_globs(started_cluster): assert "All attempts to extract table structure from files failed" in result node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data0.jsoncompacteachrow', 'TSV', 'x String') select '[123;]' SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function hdfs('hdfs://hdfs1:9000/data0.jsoncompacteachrow', 'TSV', 'x String') select '[123;]'" ) result = node1.query_and_get_error( @@ -632,7 +621,7 @@ def test_schema_inference_with_globs(started_cluster): def test_insert_select_schema_inference(started_cluster): node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/test.native.zst') select toUInt64(1) as x SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function hdfs('hdfs://hdfs1:9000/test.native.zst') select toUInt64(1) as x" ) result = node1.query(f"desc hdfs('hdfs://hdfs1:9000/test.native.zst')") @@ -675,9 +664,7 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_2', 'Parquet', 'a Int32, b String')" ) - node1.query( - f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" - ) + node1.query(f"insert into table function {table_function} SELECT 1, 'kek'") result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "parquet_2" @@ -685,9 +672,7 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_3', 'Parquet', 'a Int32, _path String')" ) - node1.query( - f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" - ) + node1.query(f"insert into table function {table_function} SELECT 1, 'kek'") result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "kek" @@ -984,11 +969,11 @@ def test_read_subcolumns(started_cluster): node = started_cluster.instances["node1"] node.query( - f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3) SETTINGS hdfs_truncate_on_insert=1" + f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" ) node.query( - f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3) SETTINGS hdfs_truncate_on_insert=1" + f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" ) res = node.query( @@ -1034,11 +1019,11 @@ def test_union_schema_inference_mode(started_cluster): node = started_cluster.instances["node1"] node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference1.jsonl') select 1 as a SETTINGS hdfs_truncate_on_insert=1" + "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference1.jsonl') select 1 as a" ) node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference2.jsonl') select 2 as b SETTINGS hdfs_truncate_on_insert=1" + "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference2.jsonl') select 2 as b" ) node.query("system drop schema cache for hdfs") @@ -1070,7 +1055,7 @@ def test_union_schema_inference_mode(started_cluster): ) assert result == "a\tNullable(Int64)\n" "b\tNullable(Int64)\n" node.query( - f"insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference3.jsonl', TSV) select 'Error' SETTINGS hdfs_truncate_on_insert=1" + f"insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference3.jsonl', TSV) select 'Error'" ) error = node.query_and_get_error( @@ -1083,11 +1068,11 @@ def test_format_detection(started_cluster): node = started_cluster.instances["node1"] node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection0', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(0) SETTINGS hdfs_truncate_on_insert=1" + "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection0', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(0)" ) node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection1', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(10) SETTINGS hdfs_truncate_on_insert=1" + "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection1', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(10)" ) expected_desc_result = node.query( @@ -1151,7 +1136,7 @@ def test_write_to_globbed_partitioned_path(started_cluster): node = started_cluster.instances["node1"] error = node.query_and_get_error( - "insert into function hdfs('hdfs://hdfs1:9000/test_data_*_{_partition_id}.csv') partition by 42 select 42 SETTINGS hdfs_truncate_on_insert=1" + "insert into function hdfs('hdfs://hdfs1:9000/test_data_*_{_partition_id}.csv') partition by 42 select 42" ) assert "DATABASE_ACCESS_DENIED" in error From 55ad7d30946d609159fe5ae9156f02f5b160585a Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 8 Aug 2024 00:08:12 +0200 Subject: [PATCH 432/644] Fix stylelint --- tests/ci/functional_test_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 3aff97643c3..b7391eff01b 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -115,7 +115,7 @@ def get_run_command( envs.append("-e NUM_TRIES=50") envs.append("-e MAX_RUN_TIME=2800") else: - max_run_time = os.getenv("MAX_RUN_TIME", 0) + max_run_time = os.getenv("MAX_RUN_TIME", "0") envs.append(f"-e MAX_RUN_TIME={max_run_time}") envs += [f"-e {e}" for e in additional_envs] From c19ee360d1a4cf0bc7607923505ba1e2a3848132 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 8 Aug 2024 00:44:42 +0200 Subject: [PATCH 433/644] Update StorageObjectStorageSource.cpp --- src/Storages/ObjectStorage/StorageObjectStorageSource.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 810bad4788b..d8e26977e75 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -208,7 +208,6 @@ Chunk StorageObjectStorageSource::generate() .filename = &filename, .last_modified = object_info->metadata->last_modified, .etag = &(object_info->metadata->etag) - .last_modified = object_info->metadata->last_modified, }, getContext(), read_from_format_info.columns_description); const auto & partition_columns = configuration->getPartitionColumns(); From 8426e0d5e5d7f102fd57a45c82ae6acccda65369 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 7 Aug 2024 16:44:10 +0800 Subject: [PATCH 434/644] fix crash --- src/Interpreters/HashJoin/HashJoinMethodsImpl.h | 3 +-- src/Interpreters/HashJoin/JoinFeatures.h | 2 +- .../03006_join_on_inequal_expression_fast.reference | 1 - 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 2bf5f6aef4a..5fefe53d145 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -361,7 +361,6 @@ size_t HashJoinMethods::joinRightColumns( } bool right_row_found = false; - KnownRowsHolder known_rows; for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx) { @@ -693,7 +692,7 @@ size_t HashJoinMethods::joinRightColumnsWithAddti any_matched = true; if constexpr (join_features.is_semi_join || join_features.is_any_join) { - auto used_once = used_flags.template setUsedOnce( + auto used_once = used_flags.template setUsedOnce( selected_right_row_it->block, selected_right_row_it->row_num, 0); if (used_once) { diff --git a/src/Interpreters/HashJoin/JoinFeatures.h b/src/Interpreters/HashJoin/JoinFeatures.h index a530179f0b4..b8de606c51e 100644 --- a/src/Interpreters/HashJoin/JoinFeatures.h +++ b/src/Interpreters/HashJoin/JoinFeatures.h @@ -22,7 +22,7 @@ struct JoinFeatures static constexpr bool need_filter = !need_replication && (inner || right || (is_semi_join && left) || (is_anti_join && left)); static constexpr bool add_missing = (left || full) && !is_semi_join; - static constexpr bool need_flags = MapGetter, HashJoin::MapsOne>>::flagged; + static constexpr bool need_flags = MapGetter, HashJoin::MapsAll>>::flagged; static constexpr bool is_maps_all = std::is_same_v, HashJoin::MapsAll>; }; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference index 11ac01d24d5..aa8d4103db2 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference @@ -1,4 +1,3 @@ -03006_join_on_inequal_expression_fast.sql -- { echoOn } SET join_algorithm='hash'; SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); From daf62e16824bb3af1137dba181a72ada11b367ad Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 6 Aug 2024 18:07:42 +0800 Subject: [PATCH 435/644] update --- .../window-functions/percent_rank.md | 6 ++-- src/Planner/PlannerActionsVisitor.cpp | 36 ++++++------------- src/Planner/PlannerActionsVisitor.h | 9 ++--- src/Planner/PlannerWindowFunctions.cpp | 14 ++++---- src/Planner/Utils.cpp | 19 ++++++++++ src/Planner/Utils.h | 7 ++++ 6 files changed, 48 insertions(+), 43 deletions(-) diff --git a/docs/en/sql-reference/window-functions/percent_rank.md b/docs/en/sql-reference/window-functions/percent_rank.md index 4b260f667b9..2e348f2a333 100644 --- a/docs/en/sql-reference/window-functions/percent_rank.md +++ b/docs/en/sql-reference/window-functions/percent_rank.md @@ -15,12 +15,12 @@ Alias: `percentRank` (case-sensitive) ```sql percent_rank (column_name) OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] - [RANGE RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] | [window_name]) + [RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] | [window_name]) FROM table_name -WINDOW window_name as ([PARTITION BY grouping_column] [ORDER BY sorting_column] RANGE RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +WINDOW window_name as ([PARTITION BY grouping_column] [ORDER BY sorting_column] RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) ``` -The default and required window frame definition is `RANGE RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING`. +The default and required window frame definition is `RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING`. For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 288669e7050..43177fc73c0 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -237,9 +237,8 @@ public: if (function_node.isWindowFunction()) { - auto get_window_frame = [&]() { return extractWindowFrame(function_node); }; buffer << " OVER ("; - buffer << calculateWindowNodeActionName(function_node.getWindowNode(), get_window_frame); + buffer << calculateWindowNodeActionName(node, function_node.getWindowNode()); buffer << ')'; } @@ -300,21 +299,22 @@ public: return calculateConstantActionNodeName(constant_literal, applyVisitor(FieldToDataType(), constant_literal)); } - String calculateWindowNodeActionName(const QueryTreeNodePtr & node, std::function()> get_window_frame) + String calculateWindowNodeActionName(const QueryTreeNodePtr & function_nodew_node_, const QueryTreeNodePtr & window_node_) { - auto & window_node = node->as(); + const auto & function_node = function_nodew_node_->as(); + const auto & window_node = window_node_->as(); WriteBufferFromOwnString buffer; if (window_node.hasPartitionBy()) { buffer << "PARTITION BY "; - auto & partition_by_nodes = window_node.getPartitionBy().getNodes(); + const auto & partition_by_nodes = window_node.getPartitionBy().getNodes(); size_t partition_by_nodes_size = partition_by_nodes.size(); for (size_t i = 0; i < partition_by_nodes_size; ++i) { - auto & partition_by_node = partition_by_nodes[i]; + const auto & partition_by_node = partition_by_nodes[i]; buffer << calculateActionNodeName(partition_by_node); if (i + 1 != partition_by_nodes_size) buffer << ", "; @@ -328,7 +328,7 @@ public: buffer << "ORDER BY "; - auto & order_by_nodes = window_node.getOrderBy().getNodes(); + const auto & order_by_nodes = window_node.getOrderBy().getNodes(); size_t order_by_nodes_size = order_by_nodes.size(); for (size_t i = 0; i < order_by_nodes_size; ++i) @@ -366,7 +366,7 @@ public: } } - auto window_frame_opt = get_window_frame(); + auto window_frame_opt = extractWindowFrame(function_node); if (window_frame_opt) { auto & window_frame = *window_frame_opt; @@ -1028,27 +1028,11 @@ String calculateConstantActionNodeName(const Field & constant_literal) return ActionNodeNameHelper::calculateConstantActionNodeName(constant_literal); } -std::optional extractWindowFrame(const FunctionNode & node) -{ - if (!node.isWindowFunction()) - return {}; - auto & window_node = node.getWindowNode()->as(); - const auto & window_frame = window_node.getWindowFrame(); - if (!window_frame.is_default) - return window_frame; - auto aggregate_function = node.getAggregateFunction(); - if (const auto * win_func = dynamic_cast(aggregate_function.get())) - { - return win_func->getDefaultFrame(); - } - return {}; -} - -String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, std::function()> get_window_frame, bool use_column_identifier_as_action_node_name) +String calculateWindowNodeActionName(const QueryTreeNodePtr & function_node, const QueryTreeNodePtr & window_node, const PlannerContext & planner_context, bool use_column_identifier_as_action_node_name) { QueryTreeNodeToName empty_map; ActionNodeNameHelper helper(empty_map, planner_context, use_column_identifier_as_action_node_name); - return helper.calculateWindowNodeActionName(node, get_window_frame); + return helper.calculateWindowNodeActionName(function_node, window_node); } } diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 17cce39f2a0..4f608ad3f7b 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -71,18 +71,13 @@ String calculateConstantActionNodeName(const Field & constant_literal, const Dat /// Calculate action node name for constant, data type will be derived from constant literal value String calculateConstantActionNodeName(const Field & constant_literal); -/// If the window frame is not set in sql, try to use the default frame from window function -/// if it have any one. Otherwise return empty. -/// If the window frame is set in sql, use it anyway. -std::optional extractWindowFrame(const FunctionNode & node); - /** Calculate action node name for window node. * Window node action name can only be part of window function action name. * For column node column node identifier from planner context is used, if use_column_identifier_as_action_node_name = true. */ -String calculateWindowNodeActionName(const QueryTreeNodePtr & node, +String calculateWindowNodeActionName(const QueryTreeNodePtr & function_node, + const QueryTreeNodePtr & window_node, const PlannerContext & planner_context, - std::function()> get_window_frame, bool use_column_identifier_as_action_node_name = true); } diff --git a/src/Planner/PlannerWindowFunctions.cpp b/src/Planner/PlannerWindowFunctions.cpp index a69dd95a650..f91cf644cf0 100644 --- a/src/Planner/PlannerWindowFunctions.cpp +++ b/src/Planner/PlannerWindowFunctions.cpp @@ -12,27 +12,27 @@ #include #include +#include namespace DB { namespace ErrorCodes { -extern const int NOT_IMPLEMENTED; + extern const int NOT_IMPLEMENTED; } namespace { -WindowDescription extractWindowDescriptionFromWindowNode(const FunctionNode & func_node, const PlannerContext & planner_context) +WindowDescription extractWindowDescriptionFromWindowNode(const QueryTreeNodePtr & func_node_, const PlannerContext & planner_context) { + const auto & func_node = func_node_->as(); auto node = func_node.getWindowNode(); auto & window_node = node->as(); - auto get_window_frame = [&]() { return extractWindowFrame(func_node); }; - WindowDescription window_description; - window_description.window_name = calculateWindowNodeActionName(node, planner_context, get_window_frame); + window_description.window_name = calculateWindowNodeActionName(func_node_, node, planner_context); for (const auto & partition_by_node : window_node.getPartitionBy().getNodes()) { @@ -49,7 +49,7 @@ WindowDescription extractWindowDescriptionFromWindowNode(const FunctionNode & fu window_description.full_sort_description.end(), window_description.order_by.begin(), window_description.order_by.end()); /// WINDOW frame is validated during query analysis stage - auto window_frame = get_window_frame(); + auto window_frame = extractWindowFrame(func_node); window_description.frame = window_frame ? *window_frame : window_node.getWindowFrame(); auto node_frame = window_node.getWindowFrame(); @@ -82,7 +82,7 @@ extractWindowDescriptions(const QueryTreeNodes & window_function_nodes, const Pl { auto & window_function_node_typed = window_function_node->as(); - auto function_window_description = extractWindowDescriptionFromWindowNode(window_function_node_typed, planner_context); + auto function_window_description = extractWindowDescriptionFromWindowNode(window_function_node, planner_context); auto frame_type = function_window_description.frame.type; if (frame_type != WindowFrame::FrameType::ROWS && frame_type != WindowFrame::FrameType::RANGE) diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index a6e94a124e6..822a3e9465e 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -22,6 +22,8 @@ #include +#include + #include #include #include @@ -34,6 +36,7 @@ #include #include #include +#include #include @@ -507,4 +510,20 @@ void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets) } } +std::optional extractWindowFrame(const FunctionNode & node) +{ + if (!node.isWindowFunction()) + return {}; + auto & window_node = node.getWindowNode()->as(); + const auto & window_frame = window_node.getWindowFrame(); + if (!window_frame.is_default) + return window_frame; + auto aggregate_function = node.getAggregateFunction(); + if (const auto * win_func = dynamic_cast(aggregate_function.get())) + { + return win_func->getDefaultFrame(); + } + return {}; +} + } diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index ae60976a8d6..254b8f4eae1 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -19,6 +19,8 @@ #include +#include + namespace DB { @@ -91,4 +93,9 @@ ASTPtr parseAdditionalResultFilter(const Settings & settings); using UsefulSets = std::unordered_set; void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets); +/// If the window frame is not set in sql, try to use the default frame from window function +/// if it have any one. Otherwise return empty. +/// If the window frame is set in sql, use it anyway. +std::optional extractWindowFrame(const FunctionNode & node); + } From c42725331d2a8bcafad804afc4a9f610630abbc4 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 5 Aug 2024 21:41:41 +0200 Subject: [PATCH 436/644] Fix --- src/Common/FailPoint.cpp | 1 + src/Functions/sleep.h | 8 ++++++++ ...date_tricky_expression_and_replication.reference} | 0 ...umn_update_tricky_expression_and_replication.sql} | 12 ++++++------ 4 files changed, 15 insertions(+), 6 deletions(-) rename tests/queries/0_stateless/{02597_column_update_tricy_expression_and_replication.reference => 02597_column_update_tricky_expression_and_replication.reference} (100%) rename tests/queries/0_stateless/{02597_column_update_tricy_expression_and_replication.sql => 02597_column_update_tricky_expression_and_replication.sql} (74%) diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index f5ec8cf0356..0b1ec552d43 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -60,6 +60,7 @@ static struct InitFiu ONCE(receive_timeout_on_table_status_response) \ REGULAR(keepermap_fail_drop_data) \ REGULAR(lazy_pipe_fds_fail_close) \ + PAUSEABLE(infinite_sleep) \ namespace FailPoints diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index 62ee19fa904..b6e4b36ee64 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -32,6 +33,11 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace FailPoints +{ + extern const char infinite_sleep[]; +} + /** sleep(seconds) - the specified number of seconds sleeps each columns. */ @@ -107,6 +113,8 @@ public: { /// When sleeping, the query cannot be cancelled. For ability to cancel query, we limit sleep time. UInt64 microseconds = static_cast(seconds * 1e6); + FailPointInjection::pauseFailPoint(FailPoints::infinite_sleep); + if (max_microseconds && microseconds > max_microseconds) throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is {} microseconds. Requested: {} microseconds", max_microseconds, microseconds); diff --git a/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.reference b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.reference similarity index 100% rename from tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.reference rename to tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.reference diff --git a/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.sql b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.sql similarity index 74% rename from tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.sql rename to tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.sql index 34f88b19b7e..12901c1e33d 100644 --- a/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.sql +++ b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.sql @@ -6,23 +6,23 @@ CREATE TABLE test ( ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '1') ORDER BY (c_id, p_id); -INSERT INTO test SELECT '1', '11', '111' FROM numbers(3); +INSERT INTO test SELECT '1', '11', '111' FROM numbers(5); -INSERT INTO test SELECT '2', '22', '22' FROM numbers(3); - -set mutations_sync=0; +SET mutations_sync=0; +SYSTEM ENABLE FAILPOINT infinite_sleep; ALTER TABLE test UPDATE d = d + sleepEachRow(0.3) where 1; ALTER TABLE test ADD COLUMN x UInt32 default 0; ALTER TABLE test UPDATE d = x + 1 where 1; + +SYSTEM DISABLE FAILPOINT infinite_sleep; ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; --{serverError BAD_ARGUMENTS} ALTER TABLE test UPDATE x = x + 1 where 1 SETTINGS mutations_sync = 2; - ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; -select * from test format Null; +SELECT * from test format Null; DROP TABLE test; From 5329fba9a25409e427f08ec7dc1e8e6d134e21d8 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 6 Aug 2024 20:21:31 +0200 Subject: [PATCH 437/644] Fix harder --- ...e_tricky_expression_and_replication.python | 61 +++++++++++++++++++ ...pdate_tricky_expression_and_replication.sh | 9 +++ ...date_tricky_expression_and_replication.sql | 28 --------- .../0_stateless/helpers/pure_http_client.py | 7 ++- 4 files changed, 74 insertions(+), 31 deletions(-) create mode 100644 tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.python create mode 100755 tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.sh delete mode 100644 tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.sql diff --git a/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.python b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.python new file mode 100644 index 00000000000..835cc7bfa51 --- /dev/null +++ b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.python @@ -0,0 +1,61 @@ +#!/usr/bin/env python3 + +import os +import sys +from threading import Thread +from queue import Queue + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) + +from pure_http_client import ClickHouseClient + + +client = ClickHouseClient() + + +client.query("DROP TABLE IF EXISTS test SYNC") +client.query(""" +CREATE TABLE test +( + c_id String, + p_id String, + d UInt32, +) +Engine = ReplicatedMergeTree() +ORDER BY (c_id, p_id) +""") + + +def attempt_mutation(q): + try: + client.query( + "ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync=2", + with_retries=False, + ) + except ValueError as e: + assert "BAD_ARGUMENTS" in str(e) + q.put("OK") + + +client.query("INSERT INTO test SELECT '1', '11', '111' FROM numbers(5)") +client.query("SYSTEM ENABLE FAILPOINT infinite_sleep") +client.query( + "ALTER TABLE test UPDATE d = d + sleepEachRow(0.3) where 1 SETTINGS mutations_sync=0" +) +client.query("ALTER TABLE test ADD COLUMN x UInt32 default 0 SETTINGS mutations_sync=0") +client.query("ALTER TABLE test UPDATE d = x + 1 where 1 SETTINGS mutations_sync=0") + +q = Queue() +t = Thread(target=attempt_mutation, args=(q,)) +t.start() +t.join() +assert not q.empty() +assert q.get() == "OK" + +client.query("SYSTEM DISABLE FAILPOINT infinite_sleep") + +client.query("ALTER TABLE test UPDATE x = x + 1 where 1 SETTINGS mutations_sync=2") +client.query("ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync=2") +client.query("SELECT * from test format Null") +client.query("DROP TABLE test") diff --git a/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.sh b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.sh new file mode 100755 index 00000000000..5be04d99204 --- /dev/null +++ b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: zookeeper, no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# We should have correct env vars from shell_config.sh to run this test +python3 "$CURDIR"/02597_column_update_tricky_expression_and_replication.python diff --git a/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.sql b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.sql deleted file mode 100644 index 12901c1e33d..00000000000 --- a/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.sql +++ /dev/null @@ -1,28 +0,0 @@ -CREATE TABLE test ( - `c_id` String, - `p_id` String, - `d` UInt32 -) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '1') -ORDER BY (c_id, p_id); - -INSERT INTO test SELECT '1', '11', '111' FROM numbers(5); - -SET mutations_sync=0; -SYSTEM ENABLE FAILPOINT infinite_sleep; - -ALTER TABLE test UPDATE d = d + sleepEachRow(0.3) where 1; - -ALTER TABLE test ADD COLUMN x UInt32 default 0; -ALTER TABLE test UPDATE d = x + 1 where 1; - -SYSTEM DISABLE FAILPOINT infinite_sleep; -ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; --{serverError BAD_ARGUMENTS} - -ALTER TABLE test UPDATE x = x + 1 where 1 SETTINGS mutations_sync = 2; -ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; - -SELECT * from test format Null; - -DROP TABLE test; - diff --git a/tests/queries/0_stateless/helpers/pure_http_client.py b/tests/queries/0_stateless/helpers/pure_http_client.py index a31a91e0550..360a1eaa486 100644 --- a/tests/queries/0_stateless/helpers/pure_http_client.py +++ b/tests/queries/0_stateless/helpers/pure_http_client.py @@ -19,9 +19,9 @@ class ClickHouseClient: self.host = host def query( - self, query, connection_timeout=500, settings=dict(), binary_result=False + self, query, connection_timeout=500, settings=dict(), binary_result=False, with_retries=True ): - NUMBER_OF_TRIES = 30 + NUMBER_OF_TRIES = 30 if with_retries else 1 DELAY = 10 params = { @@ -40,7 +40,8 @@ class ClickHouseClient: if r.status_code == 200: return r.content if binary_result else r.text else: - print("ATTENTION: try #%d failed" % i) + if with_retries: + print("ATTENTION: try #%d failed" % i) if i != (NUMBER_OF_TRIES - 1): print(query) print(r.text) From f4e1bde9b6c1b4e4477368408f955efc9bb10df5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 6 Aug 2024 18:37:21 +0000 Subject: [PATCH 438/644] Automatic style fix --- ..._column_update_tricky_expression_and_replication.python | 6 ++++-- tests/queries/0_stateless/helpers/pure_http_client.py | 7 ++++++- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.python b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.python index 835cc7bfa51..8bd5783c011 100644 --- a/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.python +++ b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.python @@ -15,7 +15,8 @@ client = ClickHouseClient() client.query("DROP TABLE IF EXISTS test SYNC") -client.query(""" +client.query( + """ CREATE TABLE test ( c_id String, @@ -24,7 +25,8 @@ CREATE TABLE test ) Engine = ReplicatedMergeTree() ORDER BY (c_id, p_id) -""") +""" +) def attempt_mutation(q): diff --git a/tests/queries/0_stateless/helpers/pure_http_client.py b/tests/queries/0_stateless/helpers/pure_http_client.py index 360a1eaa486..c3c4109ce5b 100644 --- a/tests/queries/0_stateless/helpers/pure_http_client.py +++ b/tests/queries/0_stateless/helpers/pure_http_client.py @@ -19,7 +19,12 @@ class ClickHouseClient: self.host = host def query( - self, query, connection_timeout=500, settings=dict(), binary_result=False, with_retries=True + self, + query, + connection_timeout=500, + settings=dict(), + binary_result=False, + with_retries=True, ): NUMBER_OF_TRIES = 30 if with_retries else 1 DELAY = 10 From b4cb5dcdd3c4d572e6ad73836569c09a7a69f526 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 7 Aug 2024 18:46:11 +0200 Subject: [PATCH 439/644] Typo --- ...02597_column_update_tricky_expression_and_replication.python | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.python b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.python index 8bd5783c011..eb0cab9d56f 100644 --- a/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.python +++ b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.python @@ -23,7 +23,7 @@ CREATE TABLE test p_id String, d UInt32, ) -Engine = ReplicatedMergeTree() +Engine = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '1') ORDER BY (c_id, p_id) """ ) From 142e9528f0f1f7daa68f296181f1c0e5f7b6108b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Aug 2024 03:10:45 +0200 Subject: [PATCH 440/644] Add a test for #57420 --- .../03218_materialize_msan.reference | 1 + .../0_stateless/03218_materialize_msan.sql | 21 +++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/03218_materialize_msan.reference create mode 100644 tests/queries/0_stateless/03218_materialize_msan.sql diff --git a/tests/queries/0_stateless/03218_materialize_msan.reference b/tests/queries/0_stateless/03218_materialize_msan.reference new file mode 100644 index 00000000000..eb84f35f9f4 --- /dev/null +++ b/tests/queries/0_stateless/03218_materialize_msan.reference @@ -0,0 +1 @@ +[(NULL,'11\01111111\011111','1111')] -2147483648 \N diff --git a/tests/queries/0_stateless/03218_materialize_msan.sql b/tests/queries/0_stateless/03218_materialize_msan.sql new file mode 100644 index 00000000000..b41300ea1e3 --- /dev/null +++ b/tests/queries/0_stateless/03218_materialize_msan.sql @@ -0,0 +1,21 @@ +SELECT + materialize([(NULL, '11\01111111\011111', '1111')]) AS t, + (t[1048576]).2, + materialize(-2147483648), + (t[-2147483648]).1 +GROUP BY + materialize([(NULL, '1')]), + '', + (materialize((t[1023]).2), (materialize(''), (t[2147483647]).1, materialize(9223372036854775807)), (materialize(''), materialize(NULL, 2147483647, t[65535], 256)), materialize(NULL)) +; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} + +SELECT + materialize([(NULL, '11\01111111\011111', '1111')]) AS t, + (t[1048576]).2, + materialize(-2147483648), + (t[-2147483648]).1 +GROUP BY + materialize([(NULL, '1')]), + '', + (materialize((t[1023]).2), (materialize(''), (t[2147483647]).1, materialize(9223372036854775807)), (materialize(''), materialize(NULL), materialize(2147483647), materialize(t[65535]), materialize(256)), materialize(NULL)) +; From 013680e397dcd74f807c0e90efa9a643f9bae1a1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 8 Aug 2024 03:00:51 +0000 Subject: [PATCH 441/644] empty commit From 301ac5dab7222035cdcc4fa32d061eb9f2294c05 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 9 Jul 2024 17:51:48 +0200 Subject: [PATCH 442/644] Fix possible data-race StorageKafka with statistics_interval_ms>0 The problem here is that ignorelist did not work by some reason, if I will look at the ignored functions it should not contain any TSan interseption code, while it does: $ lldb-13 clickhouse (lldb) target create "clickhouse" disas -n rd_avg_rollover Current executable set to '/home/azat/ch/tmp/tsan-test/clickhouse' (x86_64). (lldb) disas -n rd_avg_rollover clickhouse`rd_kafka_stats_emit_avg: clickhouse[0x1cbf84a7] <+39>: leaq 0x30(%r15), %r12 clickhouse[0x1cbf84ab] <+43>: movq %r12, %rdi clickhouse[0x1cbf84ae] <+46>: callq 0x1ccdad40 ; rdk_thread_mutex_lock at tinycthread.c:111 clickhouse[0x1cbf84b3] <+51>: leaq 0x58(%r15), %rdi clickhouse[0x1cbf84b7] <+55>: callq 0x71b5390 ; __tsan_read4 clickhouse[0x1cbf84bc] <+60>: cmpl $0x0, 0x58(%r15) clickhouse[0x1cbf84c1] <+65>: je 0x1cbf8595 ; <+277> [inlined] rd_avg_rollover + 238 at rdavg.h clickhouse[0x1cbf84c7] <+71>: leaq -0xc8(%rbp), %rdi clickhouse[0x1cbf84ce] <+78>: xorl %esi, %esi clickhouse[0x1cbf84d0] <+80>: callq 0x1ccdac80 ; rdk_thread_mutex_init at tinycthread.c:62 clickhouse[0x1cbf84d5] <+85>: leaq 0x5c(%r15), %rdi clickhouse[0x1cbf84d9] <+89>: callq 0x71b5390 ; __tsan_read4 (lldb) disas -n rd_avg_calc clickhouse`rd_kafka_broker_ops_io_serve: clickhouse[0x1cbdf086] <+1990>: leaq 0x5a4(%rbx), %rdi clickhouse[0x1cbdf08d] <+1997>: callq 0x71b5390 ; __tsan_read4 clickhouse[0x1cbdf092] <+2002>: cmpl $0x0, 0x5a4(%rbx) clickhouse[0x1cbdf099] <+2009>: je 0x1cbdf12b ; <+2155> [inlined] rd_kafka_broker_timeout_scan + 719 at rdkafka_broker.c I guess the reason is that they had been inlined So now rd_avg_calc() guarded with a mutex. Refs: https://github.com/ClickHouse/librdkafka/pull/11 Fixes: https://github.com/ClickHouse/ClickHouse/issues/60939 Signed-off-by: Azat Khuzhin --- contrib/librdkafka | 2 +- tests/tsan_ignorelist.txt | 4 +--- tests/ubsan_ignorelist.txt | 1 + 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/contrib/librdkafka b/contrib/librdkafka index 2d2aab6f5b7..39d4ed49ccf 160000 --- a/contrib/librdkafka +++ b/contrib/librdkafka @@ -1 +1 @@ -Subproject commit 2d2aab6f5b79db1cfca15d7bf0dee75d00d82082 +Subproject commit 39d4ed49ccf3406e2bf825d5d7b0903b5a290782 diff --git a/tests/tsan_ignorelist.txt b/tests/tsan_ignorelist.txt index 96bf6e4251f..2a31fc9bc15 100644 --- a/tests/tsan_ignorelist.txt +++ b/tests/tsan_ignorelist.txt @@ -5,11 +5,9 @@ # # Caveats for generic entry "fun": # - does not work for __attribute__((__always_inline__)) +# - and may not work for functions that had been inlined # - requires asterisk at the beginning *and* end for static functions # [thread] # https://github.com/ClickHouse/ClickHouse/issues/55629 fun:rd_kafka_broker_set_nodename -# https://github.com/ClickHouse/ClickHouse/issues/60443 -fun:*rd_avg_calc* -fun:*rd_avg_rollover* diff --git a/tests/ubsan_ignorelist.txt b/tests/ubsan_ignorelist.txt index 57d6598afa6..b75819b3f4b 100644 --- a/tests/ubsan_ignorelist.txt +++ b/tests/ubsan_ignorelist.txt @@ -9,6 +9,7 @@ # # Caveats for generic entry "fun": # - does not work for __attribute__((__always_inline__)) +# - and may not work for functions that had been inlined # - requires asterisk at the beginning *and* end for static functions # [undefined] From d6ecabb41dff64a1fb8ac5a77ffc1a8bed15162b Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 5 Aug 2024 00:40:23 +0000 Subject: [PATCH 443/644] Fix flaky test_storage_s3_queue/test.py::test_multiple_tables_streaming_sync_distributed Disable parallel processing for the Ordered mode for the test_storage_s3_queue/test.py::test_multiple_tables_streaming_sync_distributed test. The reason for this is that the load between the processing nodes is too uneven when s3queue_processing_threads_num != 1, e.g.: ``` $ grep res1 pytest.log 2024-08-07 07:15:58 [ 575 ] DEBUG : res1 size: 13300, res2 size: 1700, total_rows: 15000 (test.py:813, test_multiple_tables_streaming_sync_distributed) ``` In CIs environment, there are rare cases when one of the processors handles all the workload, while the other is busy-waiting, and the test fails on assert: When s3queue_processing_threads_num == 1, the workload is evenly distributed: ``` $ grep res1 pytest.log 2024-08-07 07:26:52 [ 586 ] DEBUG : res1 size: 7200, res2 size: 7800, total_rows: 15000 (test.py:813, test_multiple_tables_streaming_sync_distributed) ``` This change only fixes test flakiness. Further investigation of the Order mode parallelism is required. --- tests/integration/test_storage_s3_queue/test.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 92d6f181464..8f197e09e61 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -771,7 +771,11 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): table_name, mode, files_path, - additional_settings={"keeper_path": keeper_path, "s3queue_buckets": 2}, + additional_settings={ + "keeper_path": keeper_path, + "s3queue_buckets": 2, + **({"s3queue_processing_threads_num": 1} if mode == "ordered" else {}), + }, ) for instance in [node, node_2]: @@ -806,6 +810,10 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): list(map(int, l.split())) for l in run_query(node_2, get_query).splitlines() ] + logging.debug( + f"res1 size: {len(res1)}, res2 size: {len(res2)}, total_rows: {total_rows}" + ) + assert len(res1) + len(res2) == total_rows # Checking that all engines have made progress From bd3674e6e982b66f571248a717628649b1482fe5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 7 Aug 2024 20:13:50 +0200 Subject: [PATCH 444/644] Add restart --- docker/test/stateless/run.sh | 34 ++++++++++++++++++++++++++++++---- 1 file changed, 30 insertions(+), 4 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index b33c261dacc..19f2cfca43f 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -54,8 +54,6 @@ source /utils.lib /usr/share/clickhouse-test/config/install.sh ./setup_minio.sh stateless -./mc admin trace clickminio > /test_output/minio.log & -MC_ADMIN_PID=$! ./setup_hdfs_minicluster.sh @@ -176,7 +174,7 @@ done setup_logs_replication attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01 -# create minio log webhooks for both audit and server logs +# create tables for minio log webhooks clickhouse-client --query "CREATE TABLE minio_audit_logs ( log String, @@ -184,7 +182,6 @@ clickhouse-client --query "CREATE TABLE minio_audit_logs ) ENGINE = MergeTree ORDER BY tuple()" -./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" clickhouse-client --query "CREATE TABLE minio_server_logs ( @@ -193,7 +190,36 @@ clickhouse-client --query "CREATE TABLE minio_server_logs ) ENGINE = MergeTree ORDER BY tuple()" + +# create minio log webhooks for both audit and server logs ./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" +./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" +max_retries=100 +retry=1 + +while [ $retry -le $max_retries ]; do + echo "clickminio restart attempt $retry:" + + output=$(mc admin service restart clickminio 2>&1) + echo "$output" + + if echo "$output" | grep -q "Restarted \`clickminio\` successfully in 1 seconds"; then + echo "Restarted clickminio successfully." + break + fi + + sleep 1 + + retry=$((retry + 1)) +done + +if [ $retry -gt $max_retries ]; then + echo "Failed to restart clickminio after $max_retries attempts." +fi + +./mc admin service restart clickminio +./mc admin trace clickminio > /test_output/minio.log & +MC_ADMIN_PID=$! function fn_exists() { declare -F "$1" > /dev/null; From 5c97205742ff12f28cab2b853a9473cbb59edfe3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 07:23:10 +0000 Subject: [PATCH 445/644] Reapply "Bump rocksdb from v8.10 to v9.4 + enable jemalloc and liburing" This reverts commit ff8ce505d752eff1c867d73b47e39a03f0f13622. --- contrib/CMakeLists.txt | 2 +- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 44 ++++++++++++++++++---------- 3 files changed, 30 insertions(+), 18 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 977efda15ff..eb3afe0ccdf 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -71,7 +71,6 @@ add_contrib (zlib-ng-cmake zlib-ng) add_contrib (bzip2-cmake bzip2) add_contrib (minizip-ng-cmake minizip-ng) add_contrib (snappy-cmake snappy) -add_contrib (rocksdb-cmake rocksdb) add_contrib (thrift-cmake thrift) # parquet/arrow/orc add_contrib (arrow-cmake arrow) # requires: snappy, thrift, double-conversion @@ -148,6 +147,7 @@ add_contrib (hive-metastore-cmake hive-metastore) # requires: thrift, avro, arro add_contrib (cppkafka-cmake cppkafka) add_contrib (libpqxx-cmake libpqxx) add_contrib (libpq-cmake libpq) +add_contrib (rocksdb-cmake rocksdb) # requires: jemalloc, snappy, zlib, lz4, zstd, liburing add_contrib (nuraft-cmake NuRaft) add_contrib (fast_float-cmake fast_float) add_contrib (idna-cmake idna) diff --git a/contrib/rocksdb b/contrib/rocksdb index 49ce8a1064d..5f003e4a22d 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 49ce8a1064dd1ad89117899839bf136365e49e79 +Subproject commit 5f003e4a22d2e48e37c98d9620241237cd30dd24 diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 57c056532c6..7e5e9a28d0f 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -5,36 +5,38 @@ if (NOT ENABLE_ROCKSDB OR NO_SSE3_OR_HIGHER) # assumes SSE4.2 and PCLMUL return() endif() -# not in original build system, otherwise xxHash.cc fails to compile with ClickHouse C++23 default -set (CMAKE_CXX_STANDARD 20) - -# Always disable jemalloc for rocksdb by default because it introduces non-standard jemalloc APIs -option(WITH_JEMALLOC "build with JeMalloc" OFF) - -option(WITH_LIBURING "build with liburing" OFF) # TODO could try to enable this conditionally, depending on ClickHouse's ENABLE_LIBURING - # ClickHouse cannot be compiled without snappy, lz4, zlib, zstd option(WITH_SNAPPY "build with SNAPPY" ON) option(WITH_LZ4 "build with lz4" ON) option(WITH_ZLIB "build with zlib" ON) option(WITH_ZSTD "build with zstd" ON) -if(WITH_SNAPPY) +if (ENABLE_JEMALLOC) + add_definitions(-DROCKSDB_JEMALLOC -DJEMALLOC_NO_DEMANGLE) + list (APPEND THIRDPARTY_LIBS ch_contrib::jemalloc) +endif () + +if (ENABLE_LIBURING) + add_definitions(-DROCKSDB_IOURING_PRESENT) + list (APPEND THIRDPARTY_LIBS ch_contrib::liburing) +endif () + +if (WITH_SNAPPY) add_definitions(-DSNAPPY) list(APPEND THIRDPARTY_LIBS ch_contrib::snappy) endif() -if(WITH_ZLIB) +if (WITH_ZLIB) add_definitions(-DZLIB) list(APPEND THIRDPARTY_LIBS ch_contrib::zlib) endif() -if(WITH_LZ4) +if (WITH_LZ4) add_definitions(-DLZ4) list(APPEND THIRDPARTY_LIBS ch_contrib::lz4) endif() -if(WITH_ZSTD) +if (WITH_ZSTD) add_definitions(-DZSTD) list(APPEND THIRDPARTY_LIBS ch_contrib::zstd) endif() @@ -88,6 +90,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/cache/sharded_cache.cc ${ROCKSDB_SOURCE_DIR}/cache/tiered_secondary_cache.cc ${ROCKSDB_SOURCE_DIR}/db/arena_wrapped_db_iter.cc + ${ROCKSDB_SOURCE_DIR}/db/attribute_group_iterator_impl.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_contents.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_fetcher.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_addition.cc @@ -104,6 +107,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/blob/prefetch_buffer_collection.cc ${ROCKSDB_SOURCE_DIR}/db/builder.cc ${ROCKSDB_SOURCE_DIR}/db/c.cc + ${ROCKSDB_SOURCE_DIR}/db/coalescing_iterator.cc ${ROCKSDB_SOURCE_DIR}/db/column_family.cc ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction.cc ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_iterator.cc @@ -124,6 +128,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_write.cc ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_compaction_flush.cc ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_files.cc + ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_follower.cc ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_open.cc ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_debug.cc ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_experimental.cc @@ -181,6 +186,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/env/env_encryption.cc ${ROCKSDB_SOURCE_DIR}/env/file_system.cc ${ROCKSDB_SOURCE_DIR}/env/file_system_tracer.cc + ${ROCKSDB_SOURCE_DIR}/env/fs_on_demand.cc ${ROCKSDB_SOURCE_DIR}/env/fs_remap.cc ${ROCKSDB_SOURCE_DIR}/env/mock_env.cc ${ROCKSDB_SOURCE_DIR}/env/unique_id_gen.cc @@ -368,6 +374,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/volatile_tier_impl.cc ${ROCKSDB_SOURCE_DIR}/utilities/simulator_cache/cache_simulator.cc ${ROCKSDB_SOURCE_DIR}/utilities/simulator_cache/sim_cache.cc + ${ROCKSDB_SOURCE_DIR}/utilities/table_properties_collectors/compact_for_tiering_collector.cc ${ROCKSDB_SOURCE_DIR}/utilities/table_properties_collectors/compact_on_deletion_collector.cc ${ROCKSDB_SOURCE_DIR}/utilities/trace/file_trace_reader_writer.cc ${ROCKSDB_SOURCE_DIR}/utilities/trace/replayer_impl.cc @@ -388,6 +395,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_prepared_txn_db.cc ${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn.cc ${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn_db.cc + ${ROCKSDB_SOURCE_DIR}/utilities/types_util.cc ${ROCKSDB_SOURCE_DIR}/utilities/ttl/db_ttl_impl.cc ${ROCKSDB_SOURCE_DIR}/utilities/wal_filter.cc ${ROCKSDB_SOURCE_DIR}/utilities/write_batch_with_index/write_batch_with_index.cc @@ -418,14 +426,18 @@ if(HAS_ARMV8_CRC) endif(HAS_ARMV8_CRC) list(APPEND SOURCES - "${ROCKSDB_SOURCE_DIR}/port/port_posix.cc" - "${ROCKSDB_SOURCE_DIR}/env/env_posix.cc" - "${ROCKSDB_SOURCE_DIR}/env/fs_posix.cc" - "${ROCKSDB_SOURCE_DIR}/env/io_posix.cc") + ${ROCKSDB_SOURCE_DIR}/port/port_posix.cc + ${ROCKSDB_SOURCE_DIR}/env/env_posix.cc + ${ROCKSDB_SOURCE_DIR}/env/fs_posix.cc + ${ROCKSDB_SOURCE_DIR}/env/io_posix.cc) add_library(_rocksdb ${SOURCES}) add_library(ch_contrib::rocksdb ALIAS _rocksdb) target_link_libraries(_rocksdb PRIVATE ${THIRDPARTY_LIBS} ${SYSTEM_LIBS}) +# Not in the native build system but useful anyways: +# Make all functions in xxHash.h inline. Beneficial for performance: https://github.com/Cyan4973/xxHash/tree/v0.8.2#build-modifiers +target_compile_definitions (_rocksdb PRIVATE XXH_INLINE_ALL) + # SYSTEM is required to overcome some issues target_include_directories(_rocksdb SYSTEM BEFORE INTERFACE "${ROCKSDB_SOURCE_DIR}/include") From 5a17d93bf3968c9d7b19935ca4270d29b8eebef9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 07:29:20 +0000 Subject: [PATCH 446/644] Fix freebsd build --- contrib/rocksdb-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 7e5e9a28d0f..44aa7494607 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -11,7 +11,7 @@ option(WITH_LZ4 "build with lz4" ON) option(WITH_ZLIB "build with zlib" ON) option(WITH_ZSTD "build with zstd" ON) -if (ENABLE_JEMALLOC) +if (ENABLE_JEMALLOC AND OS_LINUX) # gives compile errors with jemalloc enabled for rocksdb on non-Linux add_definitions(-DROCKSDB_JEMALLOC -DJEMALLOC_NO_DEMANGLE) list (APPEND THIRDPARTY_LIBS ch_contrib::jemalloc) endif () From 9d30b45dbeddec977e2b5ef1ca8286c35c012129 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Aug 2024 08:43:49 +0100 Subject: [PATCH 447/644] Fix --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 19f2cfca43f..4d86afa4bac 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -200,7 +200,7 @@ retry=1 while [ $retry -le $max_retries ]; do echo "clickminio restart attempt $retry:" - output=$(mc admin service restart clickminio 2>&1) + output=$(./mc admin service restart clickminio 2>&1) echo "$output" if echo "$output" | grep -q "Restarted \`clickminio\` successfully in 1 seconds"; then From 71a761232c36f5ec3c30df7f3c4c3294641e414f Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Thu, 8 Aug 2024 08:45:42 +0100 Subject: [PATCH 448/644] empty From 29d701aea7f429b884715e08c9e2f60851156963 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Thu, 8 Aug 2024 10:51:57 +0300 Subject: [PATCH 449/644] Forgot to drop table in test --- tests/integration/test_storage_hdfs/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 035fe45797d..aef5ddb3675 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -118,6 +118,7 @@ def test_read_write_storage_with_globs(started_cluster): node1.query("drop table HDFSStorageWithEnum") node1.query("drop table HDFSStorageWithQuestionMark") node1.query("drop table HDFSStorageWithAsterisk") + node1.query("drop table HDFSStorageWithDoubleAsterisk") def test_storage_with_multidirectory_glob(started_cluster): From 59a63cd110d42ac313384481af1c6bf836dfef06 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 8 Aug 2024 09:48:12 +0200 Subject: [PATCH 450/644] CI: Fix for filtering jobs in PRs --- tests/ci/ci_cache.py | 3 ++- tests/ci/test_ci_config.py | 43 ++++++++++++++++++++++++++++++++++++-- 2 files changed, 43 insertions(+), 3 deletions(-) diff --git a/tests/ci/ci_cache.py b/tests/ci/ci_cache.py index a59fd3e5a29..9eeda7161ee 100644 --- a/tests/ci/ci_cache.py +++ b/tests/ci/ci_cache.py @@ -731,7 +731,8 @@ class CiCache: job_config=reference_config, ): remove_from_workflow.append(job_name) - has_test_jobs_to_skip = True + if job_name != CI.JobNames.DOCS_CHECK: + has_test_jobs_to_skip = True else: required_builds += ( job_config.required_builds if job_config.required_builds else [] diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 6ffedfdecd4..525b3bf367b 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -642,7 +642,7 @@ class TestCIConfig(unittest.TestCase): release_branch=True, ) for record_t_, records_ in ci_cache.records.items(): - if record_t_.value == CiCache.RecordType.FAILED.value: + if record_t_.value == record.record_type.value: records_[record.to_str_key()] = record ci_cache.filter_out_not_affected_jobs() @@ -716,7 +716,7 @@ class TestCIConfig(unittest.TestCase): release_branch=True, ) for record_t_, records_ in ci_cache.records.items(): - if record_t_.value == CiCache.RecordType.FAILED.value: + if record_t_.value == record.record_type.value: records_[record.to_str_key()] = record ci_cache.filter_out_not_affected_jobs() @@ -726,3 +726,42 @@ class TestCIConfig(unittest.TestCase): MOCK_REQUIRED_BUILDS, ) self.assertCountEqual(list(ci_cache.jobs_to_do), expected_to_do) + + def test_ci_py_filters_not_affected_jobs_in_prs_docs_check(self): + """ + checks ci.py filters not affected jobs in PRs, + Docs Check is special from ci_cache perspective - + check it ci pr pipline is filtered properly when only docs check is to be skipped + """ + settings = CiSettings() + settings.no_ci_cache = True + pr_info = PRInfo(github_event=_TEST_EVENT_JSON) + pr_info.event_type = EventType.PULL_REQUEST + pr_info.number = 123 + assert pr_info.is_pr + ci_cache = CIPY._configure_jobs( + S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True + ) + self.assertTrue(not ci_cache.jobs_to_skip, "Must be no jobs in skip list") + assert not ci_cache.jobs_to_wait + assert not ci_cache.jobs_to_skip + + job_config = ci_cache.jobs_to_do[CI.JobNames.DOCS_CHECK] + for batch in range(job_config.num_batches): + # add any record into cache + record = CiCache.Record( + record_type=CiCache.RecordType.PENDING, + job_name=CI.JobNames.DOCS_CHECK, + job_digest=ci_cache.job_digests[CI.JobNames.DOCS_CHECK], + batch=batch, + num_batches=job_config.num_batches, + release_branch=True, + ) + for record_t_, records_ in ci_cache.records.items(): + if record_t_.value == record.record_type.value: + records_[record.to_str_key()] = record + + expected_jobs = list(ci_cache.jobs_to_do) + expected_jobs.remove(CI.JobNames.DOCS_CHECK) + ci_cache.filter_out_not_affected_jobs() + self.assertCountEqual(list(ci_cache.jobs_to_do), expected_jobs) From 6a2ebfc95b29af7b9df5166f04b794a2db8fdff0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Aug 2024 10:07:14 +0200 Subject: [PATCH 451/644] Revert "Use `Atomic` database by default in `clickhouse-local`" --- programs/local/LocalServer.cpp | 21 ++++----- src/Databases/DatabaseAtomic.cpp | 24 ++-------- src/Databases/DatabaseAtomic.h | 3 -- src/Databases/DatabaseLazy.cpp | 3 +- src/Databases/DatabaseLazy.h | 2 +- src/Databases/DatabaseOnDisk.cpp | 28 +++-------- src/Databases/DatabaseOnDisk.h | 7 +-- src/Databases/DatabaseOrdinary.cpp | 4 +- src/Databases/DatabasesOverlay.cpp | 47 ------------------- src/Databases/DatabasesOverlay.h | 9 ---- src/Databases/IDatabase.h | 1 - .../MySQL/DatabaseMaterializedMySQL.cpp | 1 - src/Interpreters/StorageID.h | 1 + .../0_stateless/01191_rename_dictionary.sql | 1 - ...ickhouse_local_interactive_table.reference | 4 +- ...2141_clickhouse_local_interactive_table.sh | 4 +- .../03199_atomic_clickhouse_local.reference | 6 --- .../03199_atomic_clickhouse_local.sh | 24 ---------- 18 files changed, 29 insertions(+), 161 deletions(-) delete mode 100644 tests/queries/0_stateless/03199_atomic_clickhouse_local.reference delete mode 100755 tests/queries/0_stateless/03199_atomic_clickhouse_local.sh diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 0d731ed0e14..6b0b8fc5b50 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include @@ -51,6 +50,7 @@ #include #include #include +#include #include #include #include @@ -216,12 +216,12 @@ static DatabasePtr createMemoryDatabaseIfNotExists(ContextPtr context, const Str return system_database; } -static DatabasePtr createClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context) +static DatabasePtr createClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context_) { - auto overlay = std::make_shared(name_, context); - overlay->registerNextDatabase(std::make_shared(name_, fs::weakly_canonical(context->getPath()), UUIDHelpers::generateV4(), context)); - overlay->registerNextDatabase(std::make_shared(name_, "", context)); - return overlay; + auto databaseCombiner = std::make_shared(name_, context_); + databaseCombiner->registerNextDatabase(std::make_shared(name_, "", context_)); + databaseCombiner->registerNextDatabase(std::make_shared(name_, context_)); + return databaseCombiner; } /// If path is specified and not empty, will try to setup server environment and load existing metadata @@ -367,7 +367,7 @@ std::string LocalServer::getInitialCreateTableQuery() else table_structure = "(" + table_structure + ")"; - return fmt::format("CREATE TEMPORARY TABLE {} {} ENGINE = File({}, {});", + return fmt::format("CREATE TABLE {} {} ENGINE = File({}, {});", table_name, table_structure, data_format, table_file); } @@ -761,12 +761,7 @@ void LocalServer::processConfig() DatabaseCatalog::instance().initializeAndLoadTemporaryDatabase(); std::string default_database = server_settings.default_database; - { - DatabasePtr database = createClickHouseLocalDatabaseOverlay(default_database, global_context); - if (UUID uuid = database->getUUID(); uuid != UUIDHelpers::Nil) - DatabaseCatalog::instance().addUUIDMapping(uuid); - DatabaseCatalog::instance().attachDatabase(default_database, database); - } + DatabaseCatalog::instance().attachDatabase(default_database, createClickHouseLocalDatabaseOverlay(default_database, global_context)); global_context->setCurrentDatabase(default_database); if (getClientConfiguration().has("path")) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 83b82976e4f..d86e29ca915 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -53,6 +53,9 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, c , db_uuid(uuid) { assert(db_uuid != UUIDHelpers::Nil); + fs::create_directories(fs::path(getContext()->getPath()) / "metadata"); + fs::create_directories(path_to_table_symlinks); + tryCreateMetadataSymlink(); } DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, ContextPtr context_) @@ -60,16 +63,6 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, C { } -void DatabaseAtomic::createDirectories() -{ - if (database_atomic_directories_created.test_and_set()) - return; - DatabaseOnDisk::createDirectories(); - fs::create_directories(fs::path(getContext()->getPath()) / "metadata"); - fs::create_directories(path_to_table_symlinks); - tryCreateMetadataSymlink(); -} - String DatabaseAtomic::getTableDataPath(const String & table_name) const { std::lock_guard lock(mutex); @@ -106,7 +99,6 @@ void DatabaseAtomic::drop(ContextPtr) void DatabaseAtomic::attachTable(ContextPtr /* context_ */, const String & name, const StoragePtr & table, const String & relative_table_path) { assert(relative_table_path != data_path && !relative_table_path.empty()); - createDirectories(); DetachedTables not_in_use; std::lock_guard lock(mutex); not_in_use = cleanupDetachedTables(); @@ -208,15 +200,11 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_ if (exchange && !supportsAtomicRename()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "RENAME EXCHANGE is not supported"); - createDirectories(); waitDatabaseStarted(); auto & other_db = dynamic_cast(to_database); bool inside_database = this == &other_db; - if (!inside_database) - other_db.createDirectories(); - String old_metadata_path = getObjectMetadataPath(table_name); String new_metadata_path = to_database.getObjectMetadataPath(to_table_name); @@ -337,7 +325,6 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora const String & table_metadata_tmp_path, const String & table_metadata_path, ContextPtr query_context) { - createDirectories(); DetachedTables not_in_use; auto table_data_path = getTableDataPath(query); try @@ -474,9 +461,6 @@ void DatabaseAtomic::beforeLoadingMetadata(ContextMutablePtr /*context*/, Loadin if (mode < LoadingStrictnessLevel::FORCE_RESTORE) return; - if (!fs::exists(path_to_table_symlinks)) - return; - /// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken for (const auto & table_path : fs::directory_iterator(path_to_table_symlinks)) { @@ -604,7 +588,6 @@ void DatabaseAtomic::renameDatabase(ContextPtr query_context, const String & new { /// CREATE, ATTACH, DROP, DETACH and RENAME DATABASE must hold DDLGuard - createDirectories(); waitDatabaseStarted(); bool check_ref_deps = query_context->getSettingsRef().check_referential_table_dependencies; @@ -696,5 +679,4 @@ void registerDatabaseAtomic(DatabaseFactory & factory) }; factory.registerDatabase("Atomic", create_fn); } - } diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index ca24494f600..4a4ccfa2573 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -76,9 +76,6 @@ protected: using DetachedTables = std::unordered_map; [[nodiscard]] DetachedTables cleanupDetachedTables() TSA_REQUIRES(mutex); - std::atomic_flag database_atomic_directories_created = ATOMIC_FLAG_INIT; - void createDirectories(); - void tryCreateMetadataSymlink(); virtual bool allowMoveTableToOtherDatabaseEngine(IDatabase & /*to_database*/) const { return false; } diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index e43adfc5d37..3fb6d30fcb8 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -47,13 +47,12 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, : DatabaseOnDisk(name_, metadata_path_, std::filesystem::path("data") / escapeForFileName(name_) / "", "DatabaseLazy (" + name_ + ")", context_) , expiration_time(expiration_time_) { - createDirectories(); } void DatabaseLazy::loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel /*mode*/) { - iterateMetadataFiles([this, &local_context](const String & file_name) + iterateMetadataFiles(local_context, [this, &local_context](const String & file_name) { const std::string table_name = unescapeForFileName(file_name.substr(0, file_name.size() - 4)); diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index aeac130594f..41cfb751141 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -12,7 +12,7 @@ class DatabaseLazyIterator; class Context; /** Lazy engine of databases. - * Works like DatabaseOrdinary, but stores only recently accessed tables in memory. + * Works like DatabaseOrdinary, but stores in memory only the cache. * Can be used only with *Log engines. */ class DatabaseLazy final : public DatabaseOnDisk diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 82a81b0b32d..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -172,14 +172,7 @@ DatabaseOnDisk::DatabaseOnDisk( , metadata_path(metadata_path_) , data_path(data_path_) { -} - - -void DatabaseOnDisk::createDirectories() -{ - if (directories_created.test_and_set()) - return; - fs::create_directories(std::filesystem::path(getContext()->getPath()) / data_path); + fs::create_directories(local_context->getPath() + data_path); fs::create_directories(metadata_path); } @@ -197,8 +190,6 @@ void DatabaseOnDisk::createTable( const StoragePtr & table, const ASTPtr & query) { - createDirectories(); - const auto & settings = local_context->getSettingsRef(); const auto & create = query->as(); assert(table_name == create.getTable()); @@ -266,6 +257,7 @@ void DatabaseOnDisk::createTable( } commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path, local_context); + removeDetachedPermanentlyFlag(local_context, table_name, table_metadata_path, false); } @@ -293,8 +285,6 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora { try { - createDirectories(); - /// Add a table to the map of known tables. attachTable(query_context, query.getTable(), table, getTableDataPath(query)); @@ -430,7 +420,6 @@ void DatabaseOnDisk::renameTable( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Moving tables between databases of different engines is not supported"); } - createDirectories(); waitDatabaseStarted(); auto table_data_relative_path = getTableDataPath(table_name); @@ -579,14 +568,14 @@ void DatabaseOnDisk::drop(ContextPtr local_context) assert(TSA_SUPPRESS_WARNING_FOR_READ(tables).empty()); if (local_context->getSettingsRef().force_remove_data_recursively_on_drop) { - (void)fs::remove_all(std::filesystem::path(getContext()->getPath()) / data_path); + (void)fs::remove_all(local_context->getPath() + getDataPath()); (void)fs::remove_all(getMetadataPath()); } else { try { - (void)fs::remove(std::filesystem::path(getContext()->getPath()) / data_path); + (void)fs::remove(local_context->getPath() + getDataPath()); (void)fs::remove(getMetadataPath()); } catch (const fs::filesystem_error & e) @@ -624,18 +613,15 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n } } -void DatabaseOnDisk::iterateMetadataFiles(const IteratingFunction & process_metadata_file) const +void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const IteratingFunction & process_metadata_file) const { - if (!fs::exists(metadata_path)) - return; - auto process_tmp_drop_metadata_file = [&](const String & file_name) { assert(getUUID() == UUIDHelpers::Nil); static const char * tmp_drop_ext = ".sql.tmp_drop"; const std::string object_name = file_name.substr(0, file_name.size() - strlen(tmp_drop_ext)); - if (fs::exists(std::filesystem::path(getContext()->getPath()) / data_path / object_name)) + if (fs::exists(local_context->getPath() + getDataPath() + '/' + object_name)) { fs::rename(getMetadataPath() + file_name, getMetadataPath() + object_name + ".sql"); LOG_WARNING(log, "Object {} was not dropped previously and will be restored", backQuote(object_name)); @@ -652,7 +638,7 @@ void DatabaseOnDisk::iterateMetadataFiles(const IteratingFunction & process_meta std::vector> metadata_files; fs::directory_iterator dir_end; - for (fs::directory_iterator dir_it(metadata_path); dir_it != dir_end; ++dir_it) + for (fs::directory_iterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it) { String file_name = dir_it->path().filename(); /// For '.svn', '.gitignore' directory and similar. diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index 0c0ecf76a26..12656068643 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -64,7 +64,7 @@ public: time_t getObjectMetadataModificationTime(const String & object_name) const override; String getDataPath() const override { return data_path; } - String getTableDataPath(const String & table_name) const override { return std::filesystem::path(data_path) / escapeForFileName(table_name) / ""; } + String getTableDataPath(const String & table_name) const override { return data_path + escapeForFileName(table_name) + "/"; } String getTableDataPath(const ASTCreateQuery & query) const override { return getTableDataPath(query.getTable()); } String getMetadataPath() const override { return metadata_path; } @@ -83,7 +83,7 @@ protected: using IteratingFunction = std::function; - void iterateMetadataFiles(const IteratingFunction & process_metadata_file) const; + void iterateMetadataFiles(ContextPtr context, const IteratingFunction & process_metadata_file) const; ASTPtr getCreateTableQueryImpl( const String & table_name, @@ -99,9 +99,6 @@ protected: virtual void removeDetachedPermanentlyFlag(ContextPtr context, const String & table_name, const String & table_metadata_path, bool attach); virtual void setDetachedTableNotInUseForce(const UUID & /*uuid*/) {} - std::atomic_flag directories_created = ATOMIC_FLAG_INIT; - void createDirectories(); - const String metadata_path; const String data_path; }; diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index dd8a3f42ea8..8808261654f 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -55,7 +55,7 @@ static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; static constexpr const char * const CONVERT_TO_REPLICATED_FLAG_NAME = "convert_to_replicated"; DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, ContextPtr context_) - : DatabaseOrdinary(name_, metadata_path_, std::filesystem::path("data") / escapeForFileName(name_) / "", "DatabaseOrdinary (" + name_ + ")", context_) + : DatabaseOrdinary(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseOrdinary (" + name_ + ")", context_) { } @@ -265,7 +265,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables } }; - iterateMetadataFiles(process_metadata); + iterateMetadataFiles(local_context, process_metadata); size_t objects_in_database = metadata.parsed_tables.size() - prev_tables_count; size_t dictionaries_in_database = metadata.total_dictionaries - prev_total_dictionaries; diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index 495733e15fd..801356b3dd7 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -14,8 +14,6 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int CANNOT_GET_CREATE_TABLE_QUERY; - extern const int BAD_ARGUMENTS; - extern const int UNKNOWN_TABLE; } DatabasesOverlay::DatabasesOverlay(const String & name_, ContextPtr context_) @@ -126,39 +124,6 @@ StoragePtr DatabasesOverlay::detachTable(ContextPtr context_, const String & tab getEngineName()); } -void DatabasesOverlay::renameTable( - ContextPtr current_context, - const String & name, - IDatabase & to_database, - const String & to_name, - bool exchange, - bool dictionary) -{ - for (auto & db : databases) - { - if (db->isTableExist(name, current_context)) - { - if (DatabasesOverlay * to_overlay_database = typeid_cast(&to_database)) - { - /// Renaming from Overlay database inside itself or into another Overlay database. - /// Just use the first database in the overlay as a destination. - if (to_overlay_database->databases.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The destination Overlay database {} does not have any members", to_database.getDatabaseName()); - - db->renameTable(current_context, name, *to_overlay_database->databases[0], to_name, exchange, dictionary); - } - else - { - /// Renaming into a different type of database. E.g. from Overlay on top of Atomic database into just Atomic database. - db->renameTable(current_context, name, to_database, to_name, exchange, dictionary); - } - - return; - } - } - throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuote(getDatabaseName()), backQuote(name)); -} - ASTPtr DatabasesOverlay::getCreateTableQueryImpl(const String & name, ContextPtr context_, bool throw_on_error) const { ASTPtr result = nullptr; @@ -213,18 +178,6 @@ String DatabasesOverlay::getTableDataPath(const ASTCreateQuery & query) const return result; } -UUID DatabasesOverlay::getUUID() const -{ - UUID result = UUIDHelpers::Nil; - for (const auto & db : databases) - { - result = db->getUUID(); - if (result != UUIDHelpers::Nil) - break; - } - return result; -} - UUID DatabasesOverlay::tryGetTableUUID(const String & table_name) const { UUID result = UUIDHelpers::Nil; diff --git a/src/Databases/DatabasesOverlay.h b/src/Databases/DatabasesOverlay.h index 40c653e5cb5..b0c7e7e4032 100644 --- a/src/Databases/DatabasesOverlay.h +++ b/src/Databases/DatabasesOverlay.h @@ -35,21 +35,12 @@ public: StoragePtr detachTable(ContextPtr context, const String & table_name) override; - void renameTable( - ContextPtr current_context, - const String & name, - IDatabase & to_database, - const String & to_name, - bool exchange, - bool dictionary) override; - ASTPtr getCreateTableQueryImpl(const String & name, ContextPtr context, bool throw_on_error) const override; ASTPtr getCreateDatabaseQuery() const override; String getTableDataPath(const String & table_name) const override; String getTableDataPath(const ASTCreateQuery & query) const override; - UUID getUUID() const override; UUID tryGetTableUUID(const String & table_name) const override; void drop(ContextPtr context) override; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 02418abb2b0..f94326d220e 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -416,7 +416,6 @@ public: std::lock_guard lock{mutex}; return database_name; } - /// Get UUID of database. virtual UUID getUUID() const { return UUIDHelpers::Nil; } diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 8b3850c4e0c..2f5477a6b9d 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -46,7 +46,6 @@ DatabaseMaterializedMySQL::DatabaseMaterializedMySQL( , settings(std::move(settings_)) , materialize_thread(context_, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), binlog_client_, settings.get()) { - createDirectories(); } void DatabaseMaterializedMySQL::rethrowExceptionIfNeeded() const diff --git a/src/Interpreters/StorageID.h b/src/Interpreters/StorageID.h index ad55d16e284..f9afbc7b98d 100644 --- a/src/Interpreters/StorageID.h +++ b/src/Interpreters/StorageID.h @@ -27,6 +27,7 @@ class ASTQueryWithTableAndOutput; class ASTTableIdentifier; class Context; +// TODO(ilezhankin): refactor and merge |ASTTableIdentifier| struct StorageID { String database_name; diff --git a/tests/queries/0_stateless/01191_rename_dictionary.sql b/tests/queries/0_stateless/01191_rename_dictionary.sql index be95e5a7d4b..c5012dabc81 100644 --- a/tests/queries/0_stateless/01191_rename_dictionary.sql +++ b/tests/queries/0_stateless/01191_rename_dictionary.sql @@ -27,7 +27,6 @@ RENAME DICTIONARY test_01191.t TO test_01191.dict1; -- {serverError INCORRECT_QU DROP DICTIONARY test_01191.t; -- {serverError INCORRECT_QUERY} DROP TABLE test_01191.t; -DROP DATABASE IF EXISTS dummy_db; CREATE DATABASE dummy_db ENGINE=Atomic; RENAME DICTIONARY test_01191.dict TO dummy_db.dict1; RENAME DICTIONARY dummy_db.dict1 TO test_01191.dict; diff --git a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference index 0e74c0a083e..0bb8966cbe4 100644 --- a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference +++ b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference @@ -1,2 +1,2 @@ -CREATE TEMPORARY TABLE `table`\n(\n `key` String\n)\nENGINE = File(TSVWithNamesAndTypes, \'/dev/null\') -CREATE TEMPORARY TABLE `table`\n(\n `key` String\n)\nENGINE = File(TSVWithNamesAndTypes, \'/dev/null\') +CREATE TABLE default.`table`\n(\n `key` String\n)\nENGINE = File(\'TSVWithNamesAndTypes\', \'/dev/null\') +CREATE TABLE foo.`table`\n(\n `key` String\n)\nENGINE = File(\'TSVWithNamesAndTypes\', \'/dev/null\') diff --git a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh index 3a95e59416a..934d87616ac 100755 --- a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh +++ b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh @@ -4,5 +4,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create temporary table table' -$CLICKHOUSE_LOCAL --database foo --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create temporary table table' +$CLICKHOUSE_LOCAL --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create table table' +$CLICKHOUSE_LOCAL --database foo --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create table table' diff --git a/tests/queries/0_stateless/03199_atomic_clickhouse_local.reference b/tests/queries/0_stateless/03199_atomic_clickhouse_local.reference deleted file mode 100644 index 1975397394b..00000000000 --- a/tests/queries/0_stateless/03199_atomic_clickhouse_local.reference +++ /dev/null @@ -1,6 +0,0 @@ -123 -Hello -['Hello','world'] -Hello -Hello -['Hello','world'] diff --git a/tests/queries/0_stateless/03199_atomic_clickhouse_local.sh b/tests/queries/0_stateless/03199_atomic_clickhouse_local.sh deleted file mode 100755 index edaa83b8f95..00000000000 --- a/tests/queries/0_stateless/03199_atomic_clickhouse_local.sh +++ /dev/null @@ -1,24 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -${CLICKHOUSE_LOCAL} -n " -CREATE TABLE test (x UInt8) ORDER BY x; -INSERT INTO test VALUES (123); -SELECT * FROM test; -CREATE OR REPLACE TABLE test (s String) ORDER BY s; -INSERT INTO test VALUES ('Hello'); -SELECT * FROM test; -RENAME TABLE test TO test2; -CREATE OR REPLACE TABLE test (s Array(String)) ORDER BY s; -INSERT INTO test VALUES (['Hello', 'world']); -SELECT * FROM test; -SELECT * FROM test2; -EXCHANGE TABLES test AND test2; -SELECT * FROM test; -SELECT * FROM test2; -DROP TABLE test; -DROP TABLE test2; -" From f4aac7bbd9431e4d95eadfea31239b331ea18d77 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Aug 2024 09:19:45 +0100 Subject: [PATCH 452/644] Another fix --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 4d86afa4bac..830a02a64a3 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -203,7 +203,7 @@ while [ $retry -le $max_retries ]; do output=$(./mc admin service restart clickminio 2>&1) echo "$output" - if echo "$output" | grep -q "Restarted \`clickminio\` successfully in 1 seconds"; then + if echo "$output" | grep -q "Restarted \`clickminio\` successfully"; then echo "Restarted clickminio successfully." break fi From d0f35ce6a60e13b8aff9687a45e293ce89693241 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Aug 2024 10:29:01 +0200 Subject: [PATCH 453/644] Fix setting prefix --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 433a0e96d2e..7205b5b3294 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -305,7 +305,8 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet S3::ListObjectsV2Request request; request.SetBucket(uri.bucket); - request.SetPrefix(path); + if (path != "/") + request.SetPrefix(path); if (max_keys) request.SetMaxKeys(static_cast(max_keys)); else From effaeeeeac72673aa6ff5a84b73bbbd79a2067d4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 08:43:39 +0000 Subject: [PATCH 454/644] Update 3rd party lib guide --- docs/en/development/contrib.md | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/en/development/contrib.md b/docs/en/development/contrib.md index db3eabaecfc..a4353450957 100644 --- a/docs/en/development/contrib.md +++ b/docs/en/development/contrib.md @@ -27,23 +27,23 @@ Avoid dumping copies of external code into the library directory. Instead create a Git submodule to pull third-party code from an external upstream repository. All submodules used by ClickHouse are listed in the `.gitmodule` file. -If the library can be used as-is (the default case), you can reference the upstream repository directly. -If the library needs patching, create a fork of the upstream repository in the [ClickHouse organization on GitHub](https://github.com/ClickHouse). +- If the library can be used as-is (the default case), you can reference the upstream repository directly. +- If the library needs patching, create a fork of the upstream repository in the [ClickHouse organization on GitHub](https://github.com/ClickHouse). In the latter case, we aim to isolate custom patches as much as possible from upstream commits. -To that end, create a branch with prefix `clickhouse/` from the branch or tag you want to integrate, e.g. `clickhouse/master` (for branch `master`) or `clickhouse/release/vX.Y.Z` (for tag `release/vX.Y.Z`). -This ensures that pulls from the upstream repository into the fork will leave custom `clickhouse/` branches unaffected. -Submodules in `contrib/` must only track `clickhouse/` branches of forked third-party repositories. +To that end, create a branch with prefix `ClickHouse/` from the branch or tag you want to integrate, e.g. `ClickHouse/2024_2` (for branch `2024_2`) or `ClickHouse/release/vX.Y.Z` (for tag `release/vX.Y.Z`). +Avoid following upstream development branches `master`/ `main` / `dev` (i.e., prefix branches `ClickHouse/master` / `ClickHouse/main` / `ClickHouse/dev` in the fork repository). +Such branches are moving targets which make proper versioning harder. +"Prefix branches" ensure that pulls from the upstream repository into the fork will leave custom `ClickHouse/` branches unaffected. +Submodules in `contrib/` must only track `ClickHouse/` branches of forked third-party repositories. -Patches are only applied against `clickhouse/` branches of external libraries. -For that, push the patch as a branch with `clickhouse/`, e.g. `clickhouse/fix-some-desaster`. -Then create a PR from the new branch against the custom tracking branch with `clickhouse/` prefix, (e.g. `clickhouse/master` or `clickhouse/release/vX.Y.Z`) and merge the patch. +Patches are only applied against `ClickHouse/` branches of external libraries. + +There are two ways to do that: +- you like to make a new fix against a `ClickHouse/`-prefix branch in the forked repository, e.g. a sanitizer fix. In that case, push the fix as a branch with `ClickHouse/` prefix, e.g. `ClickHouse/fix-sanitizer-disaster`. Then create a PR from the new branch against the custom tracking branch, e.g. `ClickHouse/2024_2 <-- ClickHouse/fix-sanitizer-disaster` and merge the PR. +- you update the submodule and need to forward earlier patches. Re-creating old PRs is overkill in this case. Instead, simply cherry-pick older commits into the new `ClickHouse/` branch (corresponding to the new version). Feel free to squash commits of PRs that had multiple commits. + +Once the submodule has been updated, bump the submodule in ClickHouse to point to the new hash in the fork. Create patches of third-party libraries with the official repository in mind and consider contributing the patch back to the upstream repository. This makes sure that others will also benefit from the patch and it will not be a maintenance burden for the ClickHouse team. - -To pull upstream changes into the submodule, you can use two methods: -- (less work but less clean): merge upstream `master` into the corresponding `clickhouse/` tracking branch in the forked repository. You will need to resolve merge conflicts with previous custom patches. This method can be used when the `clickhouse/` branch tracks an upstream development branch like `master`, `main`, `dev`, etc. -- (more work but cleaner): create a new branch with `clickhouse/` prefix from the upstream commit or tag you like to integrate. Then re-apply all existing patches using new PRs (or squash them into a single PR). This method can be used when the `clickhouse/` branch tracks a specific upstream version branch or tag. It is cleaner in the sense that custom patches and upstream changes are better isolated from each other. - -Once the submodule has been updated, bump the submodule in ClickHouse to point to the new hash in the fork. From 76b8bcd97a40a5e05e43c3940be20d3e42c4f532 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 08:46:46 +0000 Subject: [PATCH 455/644] Update --- docs/en/development/contrib.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/contrib.md b/docs/en/development/contrib.md index a4353450957..c49492c1cb4 100644 --- a/docs/en/development/contrib.md +++ b/docs/en/development/contrib.md @@ -41,7 +41,7 @@ Patches are only applied against `ClickHouse/` branches of external libraries. There are two ways to do that: - you like to make a new fix against a `ClickHouse/`-prefix branch in the forked repository, e.g. a sanitizer fix. In that case, push the fix as a branch with `ClickHouse/` prefix, e.g. `ClickHouse/fix-sanitizer-disaster`. Then create a PR from the new branch against the custom tracking branch, e.g. `ClickHouse/2024_2 <-- ClickHouse/fix-sanitizer-disaster` and merge the PR. -- you update the submodule and need to forward earlier patches. Re-creating old PRs is overkill in this case. Instead, simply cherry-pick older commits into the new `ClickHouse/` branch (corresponding to the new version). Feel free to squash commits of PRs that had multiple commits. +- you update the submodule and need to re-apply earlier patches. In this case, re-creating old PRs is overkill. Instead, simply cherry-pick older commits into the new `ClickHouse/` branch (corresponding to the new version). Feel free to squash commits of PRs that had multiple commits. In the best case, we did contribute custom patches back to upstream and can omit patches in the new version. Once the submodule has been updated, bump the submodule in ClickHouse to point to the new hash in the fork. From 59b737c9ac045ca0ec48eb2b8893c5e54646003a Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 8 Aug 2024 10:50:14 +0200 Subject: [PATCH 456/644] CI: set correct timeout for stateless tests --- tests/ci/ci_definitions.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index b62d2e0aa8e..48847b0d7a6 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -420,7 +420,7 @@ class CommonJobConfigs: ), run_command='functional_test_check.py "$CHECK_NAME"', runner_type=Runners.FUNC_TESTER, - timeout=1001, # test + timeout=9000, ) STATEFUL_TEST = JobConfig( job_name_keyword="stateful", From dbe3035b6d96e40ae78204ba53cc91296b2af765 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 8 Aug 2024 11:32:42 +0200 Subject: [PATCH 457/644] Update src/Disks/DiskFomAST.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Disks/DiskFomAST.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index 35cb124acfd..9f83b09c8a2 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -26,7 +26,7 @@ std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string { Poco::Util::AbstractConfiguration::Keys disk_settings_keys; config->keys(disk_settings_keys); - // Check that no settings are defined when disk from the config is referred. + /// Check that no settings are defined when disk from the config is referred. if (disk_settings_keys.empty()) throw Exception( ErrorCodes::BAD_ARGUMENTS, From 376d643e39b82011d8135a23c926ae22fee4d68a Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 8 Aug 2024 11:32:55 +0200 Subject: [PATCH 458/644] Update src/Disks/DiskFomAST.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Disks/DiskFomAST.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index 9f83b09c8a2..6af7ad21366 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -30,7 +30,7 @@ std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string if (disk_settings_keys.empty()) throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Disk function has no arguments. Invalid disk description."); + "Disk function must have arguments. Invalid disk description."); if (disk_settings_keys.size() == 1 && disk_settings_keys.front() == "name" && !attach) throw Exception( From 8a93b1c7cff3c42f03642b4f7722c80c95937062 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 8 Aug 2024 11:33:10 +0200 Subject: [PATCH 459/644] Update src/Disks/DiskFomAST.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Disks/DiskFomAST.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index 6af7ad21366..5c9ece699c4 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -35,7 +35,7 @@ std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string if (disk_settings_keys.size() == 1 && disk_settings_keys.front() == "name" && !attach) throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Disk function `{}` has to have the other arguments which describe the disk. Invalid disk description.", + "Disk function `{}` must have other arguments apart from `name`, which describe disk configuration. Invalid disk description.", serialization); auto disk_settings_hash = sipHash128(serialization.data(), serialization.size()); From b79e701d8f7fda40bfb647c3c22bd637aef93fce Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 8 Aug 2024 11:33:36 +0200 Subject: [PATCH 460/644] Update src/Disks/DiskFomAST.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Disks/DiskFomAST.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index 5c9ece699c4..6d0a4ec2ea3 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -137,7 +137,7 @@ std::string DiskFomAST::createCustomDisk(const ASTPtr & disk_function_ast, Conte return disk_name; } -std::string DiskFomAST::getConfigDefinedDisk(const std::string &disk_name, ContextPtr context) +std::string DiskFomAST::getConfigDefinedDisk(const std::string & disk_name, ContextPtr context) { if (auto result = context->tryGetDisk(disk_name)) { From 2e5b71cdb17625480d7ed281dc3d7fa2315bfac5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 8 Aug 2024 12:27:03 +0200 Subject: [PATCH 461/644] Update trace_log.md --- docs/en/operations/system-tables/trace_log.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/trace_log.md b/docs/en/operations/system-tables/trace_log.md index 5adc33de37f..a60de2a08d1 100644 --- a/docs/en/operations/system-tables/trace_log.md +++ b/docs/en/operations/system-tables/trace_log.md @@ -3,7 +3,7 @@ slug: /en/operations/system-tables/trace_log --- # trace_log -Contains stack traces collected by the sampling query profiler. +Contains stack traces collected by the [sampling query profiler](../../operations/optimizing-performance/sampling-query-profiler.md). ClickHouse creates this table when the [trace_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) server configuration section is set. Also see settings: [query_profiler_real_time_period_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns), [query_profiler_cpu_time_period_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns), [memory_profiler_step](../../operations/settings/settings.md#memory_profiler_step), [memory_profiler_sample_probability](../../operations/settings/settings.md#memory_profiler_sample_probability), [trace_profile_events](../../operations/settings/settings.md#trace_profile_events). From df23a3456f66f50397091f3d84c6f07a6a1d8d81 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 8 Aug 2024 12:28:26 +0200 Subject: [PATCH 462/644] work with review --- src/Disks/DiskFomAST.cpp | 24 ++++++++------------ src/Disks/DiskFomAST.h | 2 +- src/Disks/StoragePolicy.h | 2 -- src/Interpreters/Context.cpp | 9 -------- src/Interpreters/Context.h | 1 - src/Storages/MergeTree/MergeTreeSettings.cpp | 2 +- 6 files changed, 11 insertions(+), 29 deletions(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index 6d0a4ec2ea3..638161c5c16 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -133,25 +133,19 @@ std::string DiskFomAST::createCustomDisk(const ASTPtr & disk_function_ast, Conte FlattenDiskConfigurationVisitor::Data data{context, attach}; FlattenDiskConfigurationVisitor{data}.visit(ast); - auto disk_name = assert_cast(*ast).value.get(); - return disk_name; + return assert_cast(*ast).value.get(); } -std::string DiskFomAST::getConfigDefinedDisk(const std::string & disk_name, ContextPtr context) +void DiskFomAST::ensureDiskIsNotCustom(const std::string & disk_name, ContextPtr context) { - if (auto result = context->tryGetDisk(disk_name)) - { - if (result->isCustomDisk()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Disk name `{}` is a custom disk that is used in other table. " - "That disk could not be used by a reference by other tables. The custom disk should be fully specified with a disk function.", - disk_name); + auto disk = context->getDisk(disk_name); - return disk_name; - } - - throw Exception(ErrorCodes::UNKNOWN_DISK, "Unknown disk {}", disk_name); + if (disk->isCustomDisk()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Disk name `{}` is a custom disk that is used in other table. " + "That disk could not be used by a reference by other tables. The custom disk should be fully specified with a disk function.", + disk_name); } } diff --git a/src/Disks/DiskFomAST.h b/src/Disks/DiskFomAST.h index 3a70484eda0..0a30834533e 100644 --- a/src/Disks/DiskFomAST.h +++ b/src/Disks/DiskFomAST.h @@ -8,7 +8,7 @@ namespace DB namespace DiskFomAST { - std::string getConfigDefinedDisk(const std::string & name, ContextPtr context); + void ensureDiskIsNotCustom(const std::string & name, ContextPtr context); std::string createCustomDisk(const ASTPtr & disk_function, ContextPtr context, bool attach); } diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index ccf2e2071b2..8e49ed910e3 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -12,8 +12,6 @@ #include #include -#include -#include #include #include #include diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 30f77f799e9..5413b568068 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4395,15 +4395,6 @@ DiskPtr Context::getDisk(const String & name) const return disk_selector->get(name); } -DiskPtr Context::tryGetDisk(const String & name) const -{ - std::lock_guard lock(shared->storage_policies_mutex); - - auto disk_selector = getDiskSelector(lock); - - return disk_selector->tryGet(name); -} - DiskPtr Context::getOrCreateDisk(const String & name, DiskCreator creator) const { std::lock_guard lock(shared->storage_policies_mutex); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 586eff768df..d5e35c3e4b3 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1186,7 +1186,6 @@ public: /// Provides storage disks DiskPtr getDisk(const String & name) const; - DiskPtr tryGetDisk(const String & name) const; using DiskCreator = std::function; DiskPtr getOrCreateDisk(const String & name, DiskCreator creator) const; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index e11af43ed23..dabb6991b0b 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -71,7 +71,7 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr conte } else { - value = DiskFomAST::getConfigDefinedDisk(value.safeGet(), context); + DiskFomAST::ensureDiskIsNotCustom(value.safeGet(), context); } if (has("storage_policy")) From ec145c86f5b4c4c716d4b27334381eeae83f99a8 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 8 Aug 2024 12:28:52 +0200 Subject: [PATCH 463/644] Update src/Disks/DiskFomAST.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Disks/DiskFomAST.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index 638161c5c16..bb2fcda68cb 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -64,7 +64,7 @@ std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string if (!disk->isCustomDisk()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "The disk `{}` is already exist and described by the config." + "Disk `{}` already exists and is described by the config." " It is impossible to redefine it.", disk_name); From 69ac203c9fce9972b89082ca653894fc8709f2fd Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 8 Aug 2024 12:56:33 +0200 Subject: [PATCH 464/644] fix tests --- src/Interpreters/Context.cpp | 2 +- src/Interpreters/SystemLog.h | 2 +- tests/integration/test_system_flush_logs/test.py | 3 +-- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 3051ed3e567..4a08fd5fe5b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4256,7 +4256,7 @@ std::shared_ptr Context::getS3QueueLog() const if (!shared->system_logs) return {}; - return shared->system_logs->s3_queue_log; + return shared->system_logs->s3queue_log; } std::shared_ptr Context::getAzureQueueLog() const diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 6682829c0c6..24ef6a18eb8 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -18,7 +18,7 @@ M(ErrorLog, error_log, "Contains history of error values from table system.errors, periodically flushed to disk.") \ M(FilesystemCacheLog, filesystem_cache_log, "Contains a history of all events occurred with filesystem cache for objects on a remote filesystem.") \ M(FilesystemReadPrefetchesLog, filesystem_read_prefetches_log, "Contains a history of all prefetches done during reading from MergeTables backed by a remote filesystem.") \ - M(ObjectStorageQueueLog, s3_queue_log, "Contains logging entries with the information files processes by S3Queue engine.") \ + M(ObjectStorageQueueLog, s3queue_log, "Contains logging entries with the information files processes by S3Queue engine.") \ M(ObjectStorageQueueLog, azure_queue_log, "Contains logging entries with the information files processes by S3Queue engine.") \ M(AsynchronousMetricLog, asynchronous_metric_log, "Contains the historical values for system.asynchronous_metrics, once per time interval (one second by default).") \ M(OpenTelemetrySpanLog, opentelemetry_span_log, "Contains information about trace spans for executed queries.") \ diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index dd48ef055f5..cfecea5b3d6 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -24,8 +24,7 @@ def start_cluster(): def test_system_logs_exists(): system_logs = [ - # disabled by default - ("system.text_log", 0), + ("system.text_log", 1), ("system.query_log", 1), ("system.query_thread_log", 1), ("system.part_log", 1), From e90487fd54a5c87eb0875191e7547fc2b7f2e229 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 8 Aug 2024 12:57:50 +0200 Subject: [PATCH 465/644] tests/clickhouse-test: remove superior global Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 88ff6753a8f..480dc553247 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -363,8 +363,6 @@ CAPTURE_CLIENT_STACKTRACE = False def kill_process_group(pgid): - global CAPTURE_CLIENT_STACKTRACE - print(f"Killing process group {pgid}") print(f"Processes in process group {pgid}:") print( From 420f97c8506d7c9d4216c868a75547c48a408de2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 8 Aug 2024 12:58:40 +0200 Subject: [PATCH 466/644] tests/clickhouse-test: update return type hint in run_single_test() Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 480dc553247..2da7550158b 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1603,7 +1603,7 @@ class TestCase: def run_single_test( self, server_logs_level, client_options - ) -> Tuple[Optional[Popen], str, str, str, float]: + ) -> Tuple[Optional[Popen], float]: args = self.testcase_args client = args.testcase_client start_time = args.testcase_start_time From 117fedd3bbad038ba449e3ff69c85cd937388e80 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 8 Aug 2024 12:59:32 +0200 Subject: [PATCH 467/644] fix style --- src/Disks/DiskFomAST.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index bb2fcda68cb..b2f1280c507 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -19,7 +19,6 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int UNKNOWN_DISK; } std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string & serialization, ContextPtr context, bool attach) From 979f93df12b8901df5af56f019861f910c7637cc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 8 Aug 2024 13:00:07 +0200 Subject: [PATCH 468/644] tests/clickhouse-test: better english in comment Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 2da7550158b..a70d706c7e7 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -415,8 +415,8 @@ def cleanup_child_processes(pid): ) # Due to start_new_session=True, it is not enough to kill by PGID, we need # to look at children processes as well. - # But we are hoping that nobody create session in the tests (though it is - # possible via timeout(), but we assuming that they will be killed by + # But we are hoping that nobody creates session in the tests (though it is + # possible via timeout(), but we are assuming that they will be killed by # timeout). processes = subprocess.check_output( f"pgrep --parent {pid}", shell=True, stderr=subprocess.STDOUT From bc2740aa7000694a20f3db5f13538d030248b4e0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 8 Aug 2024 13:00:37 +0200 Subject: [PATCH 469/644] tests/clickhouse-test: s/RELEASE_BUILD/RELEASE_NON_SANITIZED/g Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a70d706c7e7..a3d7e0e922d 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -376,7 +376,7 @@ def kill_process_group(pgid): # Let's try to dump stacktrace in client (useful to catch issues there) os.killpg(pgid, signal.SIGTSTP) # Wait some time for clickhouse utilities to gather stacktrace - if RELEASE_BUILD: + if RELEASE_NON_SANITIZED: sleep(0.5) else: sleep(10) @@ -2407,11 +2407,11 @@ class BuildFlags: # Release and non-sanitizer build -RELEASE_BUILD = False +RELEASE_NON_SANITIZED = False def collect_build_flags(args): - global RELEASE_BUILD + global RELEASE_NON_SANITIZED result = [] @@ -2437,7 +2437,7 @@ def collect_build_flags(args): elif b"RelWithDebInfo" in value or b"Release" in value: result.append(BuildFlags.RELEASE) - RELEASE_BUILD = result == [BuildFlags.RELEASE] + RELEASE_NON_SANITIZED = result == [BuildFlags.RELEASE] value = clickhouse_execute( args, From b0ba53788ac758ac1405ceefacd91bb0418b5834 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Aug 2024 16:29:07 +0000 Subject: [PATCH 470/644] Refactor tests for (experimental) statistics --- docs/en/development/tests.md | 4 +- .../statements/alter/statistics.md | 16 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- ...2864_statistics_count_min_sketch.reference | 14 -- .../02864_statistics_count_min_sketch.sql | 70 ------ .../02864_statistics_ddl.reference | 37 +-- .../0_stateless/02864_statistics_ddl.sql | 234 ++++++++++++++---- ...delayed_materialization_in_merge.reference | 12 + ...stics_delayed_materialization_in_merge.sql | 36 +++ .../02864_statistics_exception.reference | 0 .../02864_statistics_exception.sql | 55 ---- ..._statistics_materialize_in_merge.reference | 10 - .../02864_statistics_materialize_in_merge.sql | 52 ---- .../02864_statistics_predicates.reference | 92 +++++++ .../02864_statistics_predicates.sql | 214 ++++++++++++++++ .../02864_statistics_uniq.reference | 35 --- .../0_stateless/02864_statistics_uniq.sql | 73 ------ .../02864_statistics_usage.reference | 20 ++ .../0_stateless/02864_statistics_usage.sql | 42 ++++ 19 files changed, 619 insertions(+), 399 deletions(-) delete mode 100644 tests/queries/0_stateless/02864_statistics_count_min_sketch.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_count_min_sketch.sql create mode 100644 tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference create mode 100644 tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_exception.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_exception.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql create mode 100644 tests/queries/0_stateless/02864_statistics_predicates.reference create mode 100644 tests/queries/0_stateless/02864_statistics_predicates.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_uniq.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_uniq.sql create mode 100644 tests/queries/0_stateless/02864_statistics_usage.reference create mode 100644 tests/queries/0_stateless/02864_statistics_usage.sql diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 269995a1a96..6cb36e2049b 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -14,7 +14,7 @@ Each functional test sends one or multiple queries to the running ClickHouse ser Tests are located in `queries` directory. There are two subdirectories: `stateless` and `stateful`. Stateless tests run queries without any preloaded test data - they often create small synthetic datasets on the fly, within the test itself. Stateful tests require preloaded test data from ClickHouse and it is available to general public. -Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`. +Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`. :::note A common mistake when testing data types `DateTime` and `DateTime64` is assuming that the server uses a specific time zone (e.g. "UTC"). This is not the case, time zones in CI test runs @@ -38,7 +38,7 @@ For more options, see `tests/clickhouse-test --help`. You can simply run all tes ### Adding a New Test -To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client --multiquery < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. +To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables. diff --git a/docs/en/sql-reference/statements/alter/statistics.md b/docs/en/sql-reference/statements/alter/statistics.md index 6880cef0e5c..7a1774a01b5 100644 --- a/docs/en/sql-reference/statements/alter/statistics.md +++ b/docs/en/sql-reference/statements/alter/statistics.md @@ -8,26 +8,28 @@ sidebar_label: STATISTICS The following operations are available: -- `ALTER TABLE [db].table ADD STATISTICS (columns list) TYPE (type list)` - Adds statistic description to tables metadata. +- `ALTER TABLE [db].table ADD STATISTICS [IF NOT EXISTS] (column list) TYPE (type list)` - Adds statistic description to tables metadata. -- `ALTER TABLE [db].table MODIFY STATISTICS (columns list) TYPE (type list)` - Modifies statistic description to tables metadata. +- `ALTER TABLE [db].table MODIFY STATISTICS (column list) TYPE (type list)` - Modifies statistic description to tables metadata. -- `ALTER TABLE [db].table DROP STATISTICS (columns list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns. +- `ALTER TABLE [db].table DROP STATISTICS [IF EXISTS] (column list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns. -- `ALTER TABLE [db].table CLEAR STATISTICS (columns list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`. +- `ALTER TABLE [db].table CLEAR STATISTICS [IF EXISTS] (column list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`. -- `ALTER TABLE [db.]table MATERIALIZE STATISTICS (columns list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +- `ALTER TABLE [db.]table MATERIALIZE STATISTICS [IF EXISTS] (column list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). The first two commands are lightweight in a sense that they only change metadata or remove files. Also, they are replicated, syncing statistics metadata via ZooKeeper. -There is an example adding two statistics types to two columns: +## Example: + +Adding two statistics types to two columns: ``` ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq; ``` :::note -Statistic manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). +Statistic are supported only for [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine tables (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). ::: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 051d52a71cd..fe4857e9449 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3513,7 +3513,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context const auto & new_column = new_metadata.getColumns().get(command.column_name); if (!old_column.type->equals(*new_column.type)) throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, - "ALTER types of column {} with statistics is not not safe " + "ALTER types of column {} with statistics is not safe " "because it can change the representation of statistics", backQuoteIfNeed(command.column_name)); } diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference b/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference deleted file mode 100644 index 02c41656a36..00000000000 --- a/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference +++ /dev/null @@ -1,14 +0,0 @@ -CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -Test statistics count_min: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) -Test statistics multi-types: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) -Test LowCardinality and Nullable data type: -tab2 diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql b/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql deleted file mode 100644 index c730aa7b4a7..00000000000 --- a/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql +++ /dev/null @@ -1,70 +0,0 @@ --- Tags: no-fasttest - -DROP TABLE IF EXISTS tab SYNC; - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; -SET allow_suspicious_low_cardinality_types=1; -SET mutations_sync = 2; - -CREATE TABLE tab -( - a String, - b UInt64, - c Int64, - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; - -SHOW CREATE TABLE tab; - -INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT 'Test statistics count_min:'; - -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -ALTER TABLE tab ADD STATISTICS b TYPE count_min; -ALTER TABLE tab ADD STATISTICS c TYPE count_min; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab DROP STATISTICS a, b, c; - - -SELECT 'Test statistics multi-types:'; - -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; -ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab DROP STATISTICS a, b, c; - -DROP TABLE IF EXISTS tab SYNC; - - -SELECT 'Test LowCardinality and Nullable data type:'; -DROP TABLE IF EXISTS tab2 SYNC; -SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE tab2 -( - a LowCardinality(Int64) STATISTICS(count_min), - b Nullable(Int64) STATISTICS(count_min), - c LowCardinality(Nullable(Int64)) STATISTICS(count_min), - pk String, -) Engine = MergeTree() ORDER BY pk; - -select name from system.tables where name = 'tab2' and database = currentDatabase(); - -DROP TABLE IF EXISTS tab2 SYNC; diff --git a/tests/queries/0_stateless/02864_statistics_ddl.reference b/tests/queries/0_stateless/02864_statistics_ddl.reference index a7ff5caa0b0..0e453b0ee8a 100644 --- a/tests/queries/0_stateless/02864_statistics_ddl.reference +++ b/tests/queries/0_stateless/02864_statistics_ddl.reference @@ -1,31 +1,6 @@ -CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After insert - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) -10 -0 -After drop statistic - Prewhere info - Prewhere filter - Prewhere filter column: and(less(b, 10), less(a, 10)) (removed) -10 -CREATE TABLE default.tab\n(\n `a` Float64,\n `b` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After add statistic -CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After materialize statistic - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) -20 -After merge - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) -20 -CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After rename - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(c, 10)) (removed) -20 +CREATE TABLE default.tab\n(\n `f64` Float64,\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32,\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64,\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32,\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02864_statistics_ddl.sql b/tests/queries/0_stateless/02864_statistics_ddl.sql index fe612efe2ac..32b56a842b7 100644 --- a/tests/queries/0_stateless/02864_statistics_ddl.sql +++ b/tests/queries/0_stateless/02864_statistics_ddl.sql @@ -1,59 +1,195 @@ --- Tests that various DDL statements create/drop/materialize statistics +-- Tags: no-fasttest +-- no-fasttest: 'count_min' sketches need a 3rd party library + +-- Tests that DDL statements which create / drop / materialize statistics + +SET mutations_sync = 1; DROP TABLE IF EXISTS tab; +-- Error case: Can't create statistics when allow_experimental_statistics = 0 +CREATE TABLE tab (col Float64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; + +-- Error case: Unknown statistics types are rejected +CREATE TABLE tab (col Float64 STATISTICS(no_statistics_type)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + +-- Error case: The same statistics type can't exist more than once on a column +CREATE TABLE tab (col Float64 STATISTICS(tdigest, tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + +SET allow_suspicious_low_cardinality_types = 1; + +-- Statistics can only be created on columns of specific data types (depending on the statistics kind), (*) + +-- tdigest requires data_type.isValueRepresentedByInteger +-- These types work: +CREATE TABLE tab (col UInt8 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col UInt256 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Float32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Decimal32(3) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col IPv4 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Nullable(UInt8) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +-- These types don't work: +CREATE TABLE tab (col String STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col FixedString(1) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Array(Float64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col UUID STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col IPv6 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } + +-- uniq requires data_type.isValueRepresentedByInteger +-- These types work: +CREATE TABLE tab (col UInt8 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col UInt256 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Float32 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Decimal32(3) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date32 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime64 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col IPv4 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Nullable(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +-- These types don't work: +CREATE TABLE tab (col String STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col FixedString(1) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Array(Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col UUID STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col IPv6 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } + +-- count_min requires data_type.isValueRepresentedByInteger or data_type = (Fixed)String +-- These types work: +CREATE TABLE tab (col UInt8 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col UInt256 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Float32 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Decimal32(3) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date32 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime64 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col IPv4 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Nullable(UInt8) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col String STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col FixedString(1) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +-- These types don't work: +CREATE TABLE tab (col Array(Float64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col UUID STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col IPv6 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } + +-- CREATE TABLE was easy, ALTER is more fun CREATE TABLE tab ( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; + f64 Float64, + f64_tdigest Float64 STATISTICS(tdigest), + f32 Float32, + s String, + a Array(Float64) +) +Engine = MergeTree() +ORDER BY tuple(); +-- Error case: Unknown statistics types are rejected +-- (relevant for ADD and MODIFY) +ALTER TABLE tab ADD STATISTICS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab MODIFY STATISTICS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } +-- for some reason, ALTER TABLE tab MODIFY STATISTICS IF EXISTS is not supported + +-- Error case: The same statistics type can't exist more than once on a column +-- (relevant for ADD and MODIFY) +-- Create the same statistics object twice +ALTER TABLE tab ADD STATISTICS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab MODIFY STATISTICS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } +-- Create an statistics which exists already +ALTER TABLE tab ADD STATISTICS f64_tdigest TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64_tdigest TYPE tdigest; -- no-op +ALTER TABLE tab MODIFY STATISTICS f64_tdigest TYPE tdigest; -- no-op + +-- Error case: Column does not exist +-- (relevant for ADD, MODIFY, DROP, CLEAR, and MATERIALIZE) +-- Note that the results are unfortunately quite inconsistent ... +ALTER TABLE tab ADD STATISTICS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS IF EXISTS no_such_column; -- no-op +ALTER TABLE tab CLEAR STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab CLEAR STATISTICS IF EXISTS no_such_column; -- no-op +ALTER TABLE tab MATERIALIZE STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MATERIALIZE STATISTICS IF EXISTS no_such_column; -- { serverError ILLEGAL_STATISTICS } + +-- Error case: Column exists but has no statistics +-- (relevant for MODIFY, DROP, CLEAR, and MATERIALIZE) +-- Note that the results are unfortunately quite inconsistent ... +ALTER TABLE tab MODIFY STATISTICS s TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS s; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS IF EXISTS s; -- no-op +ALTER TABLE tab CLEAR STATISTICS s; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab CLEAR STATISTICS IF EXISTS s; -- no-op +ALTER TABLE tab MATERIALIZE STATISTICS s; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MATERIALIZE STATISTICS IF EXISTS s; -- { serverError ILLEGAL_STATISTICS } + +-- We don't check systematically that that statistics can only be created via ALTER ADD STATISTICS on columns of specific data types (the +-- internal type validation code is tested already above, (*)). Only do a rudimentary check for each statistics type with a data type that +-- works and one that doesn't work. +-- tdigest +-- Works: +ALTER TABLE tab ADD STATISTICS f64 TYPE tdigest; ALTER TABLE tab DROP STATISTICS f64; +ALTER TABLE tab MODIFY STATISTICS f64 TYPE tdigest; ALTER TABLE tab DROP STATISTICS f64; +-- Doesn't work: +ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +-- uniq +-- Works: +ALTER TABLE tab ADD STATISTICS f64 TYPE uniq; ALTER TABLE tab DROP STATISTICS f64; +ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; +-- Doesn't work: +ALTER TABLE tab ADD STATISTICS a TYPE uniq; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS a TYPE uniq; -- { serverError ILLEGAL_STATISTICS } +-- count_min +-- Works: +ALTER TABLE tab ADD STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; +ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; +-- Doesn't work: +ALTER TABLE tab ADD STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS } + +-- Any data type changes on columns with statistics are disallowed, for simplicity even if the new data type is compatible with all existing +-- statistics objects (e.g. tdigest can be created on Float64 and UInt64) +ALTER TABLE tab MODIFY COLUMN f64_tdigest UInt64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } + +-- Finally, do a full-circle test of a good case. Print table definition after each step. +-- Intentionally specifying _two_ columns and _two_ statistics types to have that also tested. +SHOW CREATE TABLE tab; +ALTER TABLE tab ADD STATISTICS f64, f32 TYPE tdigest, uniq; +SHOW CREATE TABLE tab; +ALTER TABLE tab MODIFY STATISTICS f64, f32 TYPE tdigest, uniq; +SHOW CREATE TABLE tab; +ALTER TABLE tab CLEAR STATISTICS f64, f32; +SHOW CREATE TABLE tab; +ALTER TABLE tab MATERIALIZE STATISTICS f64, f32; +SHOW CREATE TABLE tab; +ALTER TABLE tab DROP STATISTICS f64, f32; SHOW CREATE TABLE tab; -INSERT INTO tab select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT 'After insert'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; -SELECT count(*) FROM tab WHERE b < NULL and a < '10'; - -ALTER TABLE tab DROP STATISTICS a, b; - -SELECT 'After drop statistic'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; - -SHOW CREATE TABLE tab; - -ALTER TABLE tab ADD STATISTICS a, b TYPE tdigest; - -SELECT 'After add statistic'; - -SHOW CREATE TABLE tab; - -ALTER TABLE tab MATERIALIZE STATISTICS a, b; -INSERT INTO tab select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT 'After materialize statistic'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; - -OPTIMIZE TABLE tab FINAL; - -SELECT 'After merge'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; - -ALTER TABLE tab RENAME COLUMN b TO c; -SHOW CREATE TABLE tab; - -SELECT 'After rename'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE c < 10 and a < 10; - -DROP TABLE IF EXISTS tab; +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference new file mode 100644 index 00000000000..eb5e685597c --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference @@ -0,0 +1,12 @@ +After insert + Prewhere info + Prewhere filter + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) +After merge + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After truncate, insert, and materialize + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql new file mode 100644 index 00000000000..33a5f9052ba --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql @@ -0,0 +1,36 @@ +-- Tests delayed materialization of statistics in merge instead of during insert (setting 'materialize_statistics_on_insert = 0'). +-- (The concrete statistics type, column data type and predicate type don't matter) + +-- Checks by the predicate evaluation order in EXPLAIN. This is quite fragile, a better approach would be helpful (maybe 'send_logs_level'?) + +DROP TABLE IF EXISTS tab; + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET allow_analyzer = 1; + +SET materialize_statistics_on_insert = 0; + +CREATE TABLE tab +( + a Int64 STATISTICS(tdigest), + b Int16 STATISTICS(tdigest), +) ENGINE = MergeTree() ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics. + +INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; +SELECT 'After insert'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks b first, then a (statistics not used) + +OPTIMIZE TABLE tab FINAL; +SELECT 'After merge'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +TRUNCATE TABLE tab; +SET mutations_sync = 2; +INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; +ALTER TABLE tab MATERIALIZE STATISTICS a, b; +SELECT 'After truncate, insert, and materialize'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_exception.reference b/tests/queries/0_stateless/02864_statistics_exception.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02864_statistics_exception.sql b/tests/queries/0_stateless/02864_statistics_exception.sql deleted file mode 100644 index 289ffee6600..00000000000 --- a/tests/queries/0_stateless/02864_statistics_exception.sql +++ /dev/null @@ -1,55 +0,0 @@ --- Tests creating/dropping/materializing statistics produces the right exceptions. - -DROP TABLE IF EXISTS tab; - --- Can't create statistics when allow_experimental_statistics = 0 -CREATE TABLE tab -( - a Float64 STATISTICS(tdigest) -) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - -SET allow_experimental_statistics = 1; - --- The same type of statistics can't exist more than once on a column -CREATE TABLE tab -( - a Float64 STATISTICS(tdigest, tdigest) -) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - --- Unknown statistics types are rejected -CREATE TABLE tab -( - a Float64 STATISTICS(no_statistics_type) -) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - --- tDigest statistics can only be created on numeric columns -CREATE TABLE tab -( - a String STATISTICS(tdigest), -) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } - -CREATE TABLE tab -( - a Float64, - b String -) Engine = MergeTree() ORDER BY tuple(); - -ALTER TABLE tab ADD STATISTICS a TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } -ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -ALTER TABLE tab ADD STATISTICS IF NOT EXISTS a TYPE tdigest; -ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MODIFY STATISTICS a TYPE tdigest; --- Statistics can be created only on integer columns -ALTER TABLE tab ADD STATISTICS b TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS b; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS a; -ALTER TABLE tab DROP STATISTICS IF EXISTS a; -ALTER TABLE tab CLEAR STATISTICS a; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab CLEAR STATISTICS IF EXISTS a; -ALTER TABLE tab MATERIALIZE STATISTICS b; -- { serverError ILLEGAL_STATISTICS } - -ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -ALTER TABLE tab MODIFY COLUMN a Float64 TTL toDateTime(b) + INTERVAL 1 MONTH; -ALTER TABLE tab MODIFY COLUMN a Int64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } - -DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference deleted file mode 100644 index 5e969cf41cb..00000000000 --- a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference +++ /dev/null @@ -1,10 +0,0 @@ -10 -10 -10 -statistics not used Condition less(b, 10_UInt8) moved to PREWHERE -statistics not used Condition less(a, 10_UInt8) moved to PREWHERE -statistics used after merge Condition less(a, 10_UInt8) moved to PREWHERE -statistics used after merge Condition less(b, 10_UInt8) moved to PREWHERE -statistics used after materialize Condition less(a, 10_UInt8) moved to PREWHERE -statistics used after materialize Condition less(b, 10_UInt8) moved to PREWHERE -2 0 diff --git a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql deleted file mode 100644 index 6606cff263f..00000000000 --- a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql +++ /dev/null @@ -1,52 +0,0 @@ --- Tests delayed materialization of statistics in merge instead of during insert (setting 'materialize_statistics_on_insert = 0'). - -DROP TABLE IF EXISTS tab; - -SET enable_analyzer = 1; -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; - -SET materialize_statistics_on_insert = 0; - -CREATE TABLE tab -( - a Int64 STATISTICS(tdigest), - b Int16 STATISTICS(tdigest), -) ENGINE = MergeTree() ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics. - -INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; - -SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics not used'; - -OPTIMIZE TABLE tab FINAL; - -SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after merge'; - -TRUNCATE TABLE tab; -SET mutations_sync = 2; - -INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; -ALTER TABLE tab MATERIALIZE STATISTICS a, b; - -SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after materialize'; - -DROP TABLE tab; - -SYSTEM FLUSH LOGS; - -SELECT log_comment, message FROM system.text_log JOIN -( - SELECT Settings['log_comment'] AS log_comment, query_id FROM system.query_log - WHERE current_database = currentDatabase() - AND query LIKE 'SELECT count(*) FROM tab%' - AND type = 'QueryFinish' -) AS query_log USING (query_id) -WHERE message LIKE '%moved to PREWHERE%' -ORDER BY event_time_microseconds; - -SELECT count(), sum(ProfileEvents['MergeTreeDataWriterStatisticsCalculationMicroseconds']) -FROM system.query_log -WHERE current_database = currentDatabase() - AND query LIKE 'INSERT INTO tab SELECT%' - AND type = 'QueryFinish'; diff --git a/tests/queries/0_stateless/02864_statistics_predicates.reference b/tests/queries/0_stateless/02864_statistics_predicates.reference new file mode 100644 index 00000000000..1c2abd47aaf --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_predicates.reference @@ -0,0 +1,92 @@ +u64 and = +10 +10 +10 +10 +0 +0 +0 +0 +10 +10 +10 +10 +u64 and < +70 +70 +70 +70 +80 +80 +80 +80 +70 +70 +70 +f64 and = +10 +10 +10 +10 +0 +0 +0 +0 +10 +10 +10 +0 +0 +0 +f64 and < +70 +70 +70 +70 +80 +80 +80 +80 +70 +70 +70 +80 +80 +80 +dt and = +0 +0 +0 +0 +10 +10 +10 +10 +dt and < +10000 +10000 +10000 +70 +70 +70 +70 +b and = +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +0 +0 +0 +0 +s and = +10 +10 diff --git a/tests/queries/0_stateless/02864_statistics_predicates.sql b/tests/queries/0_stateless/02864_statistics_predicates.sql new file mode 100644 index 00000000000..3e754dfb1de --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_predicates.sql @@ -0,0 +1,214 @@ +-- Tags: no-fasttest +-- no-fasttest: 'count_min' sketches need a 3rd party library + +-- Tests the cross product of all predicates with all right-hand sides on all data types and all statistics types. + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab +( + u64 UInt64, + u64_tdigest UInt64 STATISTICS(tdigest), + u64_count_min UInt64 STATISTICS(count_min), + u64_uniq UInt64 STATISTICS(uniq), + f64 Float64, + f64_tdigest Float64 STATISTICS(tdigest), + f64_count_min Float64 STATISTICS(count_min), + f64_uniq Float64 STATISTICS(uniq), + dt DateTime, + dt_tdigest DateTime STATISTICS(tdigest), + dt_count_min DateTime STATISTICS(count_min), + dt_uniq DateTime STATISTICS(uniq), + b Bool, + b_tdigest Bool STATISTICS(tdigest), + b_count_min Bool STATISTICS(count_min), + b_uniq Bool STATISTICS(uniq), + s String, + -- s_tdigest String STATISTICS(tdigest), -- not supported by tdigest + s_count_min String STATISTICS(count_min) + -- s_uniq String STATISTICS(uniq), -- not supported by uniq +) Engine = MergeTree() ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO tab +-- SELECT number % 10000, number % 1000, -(number % 100) FROM system.numbers LIMIT 10000; +SELECT number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 2, + number % 2, + number % 2, + number % 2, + toString(number % 1000), + toString(number % 1000) +FROM system.numbers LIMIT 10000; + +-- u64 ---------------------------------------------------- + +SELECT 'u64 and ='; + +SELECT count(*) FROM tab WHERE u64 = 7; +SELECT count(*) FROM tab WHERE u64_tdigest = 7; +SELECT count(*) FROM tab WHERE u64_count_min = 7; +SELECT count(*) FROM tab WHERE u64_uniq = 7; + +SELECT count(*) FROM tab WHERE u64 = 7.7; +SELECT count(*) FROM tab WHERE u64_tdigest = 7.7; +SELECT count(*) FROM tab WHERE u64_count_min = 7.7; +SELECT count(*) FROM tab WHERE u64_uniq = 7.7; + +SELECT count(*) FROM tab WHERE u64 = '7'; +SELECT count(*) FROM tab WHERE u64_tdigest = '7'; +SELECT count(*) FROM tab WHERE u64_count_min = '7'; +SELECT count(*) FROM tab WHERE u64_uniq = '7'; + +SELECT count(*) FROM tab WHERE u64 = '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_tdigest = '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_count_min = '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_uniq = '7.7'; -- { serverError TYPE_MISMATCH } + +SELECT 'u64 and <'; + +SELECT count(*) FROM tab WHERE u64 < 7; +SELECT count(*) FROM tab WHERE u64_tdigest < 7; +SELECT count(*) FROM tab WHERE u64_count_min < 7; +SELECT count(*) FROM tab WHERE u64_uniq < 7; + +SELECT count(*) FROM tab WHERE u64 < 7.7; +SELECT count(*) FROM tab WHERE u64_tdigest < 7.7; +SELECT count(*) FROM tab WHERE u64_count_min < 7.7; +SELECT count(*) FROM tab WHERE u64_uniq < 7.7; + +SELECT count(*) FROM tab WHERE u64 < '7'; +-- SELECT count(*) FROM tab WHERE u64_tdigest < '7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE u64_count_min < '7'; +SELECT count(*) FROM tab WHERE u64_uniq < '7'; + +SELECT count(*) FROM tab WHERE u64 < '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_tdigest < '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_count_min < '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_uniq < '7.7'; -- { serverError TYPE_MISMATCH } + +-- f64 ---------------------------------------------------- + +SELECT 'f64 and ='; + +SELECT count(*) FROM tab WHERE f64 = 7; +SELECT count(*) FROM tab WHERE f64_tdigest = 7; +SELECT count(*) FROM tab WHERE f64_count_min = 7; +SELECT count(*) FROM tab WHERE f64_uniq = 7; + +SELECT count(*) FROM tab WHERE f64 = 7.7; +SELECT count(*) FROM tab WHERE f64_tdigest = 7.7; +SELECT count(*) FROM tab WHERE f64_count_min = 7.7; +SELECT count(*) FROM tab WHERE f64_uniq = 7.7; + +SELECT count(*) FROM tab WHERE f64 = '7'; +-- SELECT count(*) FROM tab WHERE f64_tdigest = '7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_count_min = '7'; +SELECT count(*) FROM tab WHERE f64_uniq = '7'; + +SELECT count(*) FROM tab WHERE f64 = '7.7'; +-- SELECT count(*) FROM tab WHERE f64_tdigest = '7.7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_count_min = '7.7'; +SELECT count(*) FROM tab WHERE f64_uniq = '7.7'; + +SELECT 'f64 and <'; + +SELECT count(*) FROM tab WHERE f64 < 7; +SELECT count(*) FROM tab WHERE f64_tdigest < 7; +SELECT count(*) FROM tab WHERE f64_count_min < 7; +SELECT count(*) FROM tab WHERE f64_uniq < 7; + +SELECT count(*) FROM tab WHERE f64 < 7.7; +SELECT count(*) FROM tab WHERE f64_tdigest < 7.7; +SELECT count(*) FROM tab WHERE f64_count_min < 7.7; +SELECT count(*) FROM tab WHERE f64_uniq < 7.7; + +SELECT count(*) FROM tab WHERE f64 < '7'; +-- SELECT count(*) FROM tab WHERE f64_tdigest < '7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_count_min < '7'; +SELECT count(*) FROM tab WHERE f64_uniq < '7'; + +SELECT count(*) FROM tab WHERE f64 < '7.7'; +-- SELECT count(*) FROM tab WHERE f64_tdigest < '7.7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_count_min < '7.7'; +SELECT count(*) FROM tab WHERE f64_uniq < '7.7'; + +-- dt ---------------------------------------------------- + +SELECT 'dt and ='; + +SELECT count(*) FROM tab WHERE dt = '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_tdigest = '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_count_min = '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_uniq = '2024-08-08 11:12:13'; + +SELECT count(*) FROM tab WHERE dt = 7; +SELECT count(*) FROM tab WHERE dt_tdigest = 7; +SELECT count(*) FROM tab WHERE dt_count_min = 7; +SELECT count(*) FROM tab WHERE dt_uniq = 7; + +SELECT 'dt and <'; + +SELECT count(*) FROM tab WHERE dt < '2024-08-08 11:12:13'; +-- SELECT count(*) FROM tab WHERE dt_tdigest < '2024-08-08 11:12:13'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE dt_count_min < '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_uniq < '2024-08-08 11:12:13'; + +SELECT count(*) FROM tab WHERE dt < 7; +SELECT count(*) FROM tab WHERE dt_tdigest < 7; +SELECT count(*) FROM tab WHERE dt_count_min < 7; +SELECT count(*) FROM tab WHERE dt_uniq < 7; + +-- b ---------------------------------------------------- + +SELECT 'b and ='; + +SELECT count(*) FROM tab WHERE b = true; +SELECT count(*) FROM tab WHERE b_tdigest = true; +SELECT count(*) FROM tab WHERE b_count_min = true; +SELECT count(*) FROM tab WHERE b_uniq = true; + +SELECT count(*) FROM tab WHERE b = 'true'; +SELECT count(*) FROM tab WHERE b_tdigest = 'true'; +SELECT count(*) FROM tab WHERE b_count_min = 'true'; +SELECT count(*) FROM tab WHERE b_uniq = 'true'; + +SELECT count(*) FROM tab WHERE b = 1; +SELECT count(*) FROM tab WHERE b_tdigest = 1; +SELECT count(*) FROM tab WHERE b_count_min = 1; +SELECT count(*) FROM tab WHERE b_uniq = 1; + +SELECT count(*) FROM tab WHERE b = 1.1; +SELECT count(*) FROM tab WHERE b_tdigest = 1.1; +SELECT count(*) FROM tab WHERE b_count_min = 1.1; +SELECT count(*) FROM tab WHERE b_uniq = 1.1; + +-- s ---------------------------------------------------- + +SELECT 's and ='; + +SELECT count(*) FROM tab WHERE s = 7; -- { serverError NO_COMMON_TYPE } +-- SELECT count(*) FROM tab WHERE s_tdigest = 7; -- not supported +SELECT count(*) FROM tab WHERE s_count_min = 7; -- { serverError NO_COMMON_TYPE } +-- SELECT count(*) FROM tab WHERE s_uniq = 7; -- not supported + +SELECT count(*) FROM tab WHERE s = '7'; +-- SELECT count(*) FROM tab WHERE s_tdigest = '7'; -- not supported +SELECT count(*) FROM tab WHERE s_count_min = '7'; +-- SELECT count(*) FROM tab WHERE s_uniq = '7'; -- not supported + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_uniq.reference b/tests/queries/0_stateless/02864_statistics_uniq.reference deleted file mode 100644 index 77786dbdd8c..00000000000 --- a/tests/queries/0_stateless/02864_statistics_uniq.reference +++ /dev/null @@ -1,35 +0,0 @@ -CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest, uniq),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After insert - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) -After merge - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) -After modify TDigest - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(c, -1), less(a, 10), less(b, 10)) (removed) -After drop - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(c, -1), less(b, 10)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_uniq.sql b/tests/queries/0_stateless/02864_statistics_uniq.sql deleted file mode 100644 index 0f5f353c045..00000000000 --- a/tests/queries/0_stateless/02864_statistics_uniq.sql +++ /dev/null @@ -1,73 +0,0 @@ -DROP TABLE IF EXISTS t1; - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; -SET mutations_sync = 1; - -CREATE TABLE t1 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c Int64 STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; - -SHOW CREATE TABLE t1; - -INSERT INTO t1 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; -INSERT INTO t1 select 0, 0, 11, generateUUIDv4(); - -SELECT 'After insert'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -OPTIMIZE TABLE t1 FINAL; - -SELECT 'After merge'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -SELECT 'After modify TDigest'; -ALTER TABLE t1 MODIFY STATISTICS c TYPE TDigest; -ALTER TABLE t1 MATERIALIZE STATISTICS c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - - -ALTER TABLE t1 DROP STATISTICS c; - -SELECT 'After drop'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -DROP TABLE IF EXISTS t1; -DROP TABLE IF EXISTS t2; -SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE t2 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c LowCardinality(Int64) STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO t2 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; - -DROP TABLE IF EXISTS t2; -DROP TABLE IF EXISTS t3; - -CREATE TABLE t3 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c Nullable(Int64) STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO t3 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; - -DROP TABLE IF EXISTS t3; - diff --git a/tests/queries/0_stateless/02864_statistics_usage.reference b/tests/queries/0_stateless/02864_statistics_usage.reference new file mode 100644 index 00000000000..a9f669b88c1 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_usage.reference @@ -0,0 +1,20 @@ +After insert + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After drop statistic + Prewhere info + Prewhere filter + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) +After add and materialize statistic + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After merge + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After rename + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(c, 10_UInt8)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_usage.sql b/tests/queries/0_stateless/02864_statistics_usage.sql new file mode 100644 index 00000000000..f936854df44 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_usage.sql @@ -0,0 +1,42 @@ +-- Test that the optimizer picks up column statistics +-- (The concrete statistics type, column data type and predicate type don't matter) + +-- Checks by the predicate evaluation order in EXPLAIN. This is quite fragile, a better approach would be helpful (maybe 'send_logs_level'?) + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET mutations_sync = 1; +SET allow_analyzer = 1; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest) +) Engine = MergeTree() ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO tab select number, -number FROM system.numbers LIMIT 10000; +SELECT 'After insert'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +ALTER TABLE tab DROP STATISTICS a, b; +SELECT 'After drop statistic'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks b first, then a (statistics not used) + +ALTER TABLE tab ADD STATISTICS a, b TYPE tdigest; +ALTER TABLE tab MATERIALIZE STATISTICS a, b; +INSERT INTO tab select number, -number FROM system.numbers LIMIT 10000; +SELECT 'After add and materialize statistic'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +OPTIMIZE TABLE tab FINAL; +SELECT 'After merge'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +ALTER TABLE tab RENAME COLUMN b TO c; +SELECT 'After rename'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then c (statistics used) + +DROP TABLE IF EXISTS tab; From 4b2234f87d7b7ff2033327bd1c03278735438f1a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 12:06:32 +0000 Subject: [PATCH 471/644] Minor fixups --- .../0_stateless/00945_bloom_filter_index.sql | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/00945_bloom_filter_index.sql b/tests/queries/0_stateless/00945_bloom_filter_index.sql index 71109df79e7..6e3819e74d3 100644 --- a/tests/queries/0_stateless/00945_bloom_filter_index.sql +++ b/tests/queries/0_stateless/00945_bloom_filter_index.sql @@ -375,13 +375,9 @@ SELECT id, ary[indexOf(ary, 'value3')] FROM test_bf_indexOf WHERE ary[indexOf(ar DROP TABLE IF EXISTS test_bf_indexOf; --- expecting cast function to be unknown +-- Test for bug #65597 DROP TABLE IF EXISTS test_bf_cast; - -CREATE TABLE test_bf_cast (c Int32, INDEX x1 (c) type bloom_filter) ENGINE = MergeTree ORDER BY c as select 1; - -SELECT count() FROM test_bf_cast WHERE cast(c=1 or c=9999 as Bool) settings use_skip_indexes=0; - -SELECT count() FROM test_bf_cast WHERE cast(c=1 or c=9999 as Bool) settings use_skip_indexes=1; - -DROP TABLE test_bf_cast; \ No newline at end of file +CREATE TABLE test_bf_cast (c Int32, INDEX x1 (c) type bloom_filter) ENGINE = MergeTree ORDER BY c AS SELECT 1; +SELECT count() FROM test_bf_cast WHERE cast(c = 1 OR c = 9999 AS Bool) SETTINGS use_skip_indexes=0; +SELECT count() FROM test_bf_cast WHERE cast(c = 1 OR c = 9999 AS Bool) SETTINGS use_skip_indexes=1; +DROP TABLE test_bf_cast; From eca6bba0388f21c67d0caf24e56032bc6b7cd339 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 12:50:45 +0000 Subject: [PATCH 472/644] Split 00284_external_aggregation --- .../00284_external_aggregation.reference | 20 ----------------- .../00284_external_aggregation.sql | 19 +++------------- .../00284_external_aggregation_2.reference | 20 +++++++++++++++++ .../00284_external_aggregation_2.sql | 22 +++++++++++++++++++ .../0_stateless/02099_tsv_raw_format_1.sh | 3 +++ .../0_stateless/02099_tsv_raw_format_2.sh | 3 +++ 6 files changed, 51 insertions(+), 36 deletions(-) create mode 100644 tests/queries/0_stateless/00284_external_aggregation_2.reference create mode 100644 tests/queries/0_stateless/00284_external_aggregation_2.sql diff --git a/tests/queries/0_stateless/00284_external_aggregation.reference b/tests/queries/0_stateless/00284_external_aggregation.reference index be0db217a97..48e30e781e0 100644 --- a/tests/queries/0_stateless/00284_external_aggregation.reference +++ b/tests/queries/0_stateless/00284_external_aggregation.reference @@ -1,22 +1,2 @@ 49999995000000 10000000 499999500000 1000000 15 -100033 2 -100034 2 -100035 2 -100036 2 -100037 2 -100038 2 -100039 2 -10004 2 -100040 2 -100041 2 -100033 2 -100034 2 -100035 2 -100036 2 -100037 2 -100038 2 -100039 2 -10004 2 -100040 2 -100041 2 diff --git a/tests/queries/0_stateless/00284_external_aggregation.sql b/tests/queries/0_stateless/00284_external_aggregation.sql index c1140faaa28..cdc31ff68c8 100644 --- a/tests/queries/0_stateless/00284_external_aggregation.sql +++ b/tests/queries/0_stateless/00284_external_aggregation.sql @@ -1,5 +1,8 @@ -- Tags: long +-- This test was split in two due to long runtimes in sanitizers. +-- The other part is 00284_external_aggregation_2. + SET max_bytes_before_external_group_by = 100000000; SET max_memory_usage = 410000000; SET group_by_two_level_threshold = 100000; @@ -7,19 +10,3 @@ SET group_by_two_level_threshold_bytes = 50000000; SELECT sum(k), sum(c) FROM (SELECT number AS k, count() AS c FROM (SELECT * FROM system.numbers LIMIT 10000000) GROUP BY k); SELECT sum(k), sum(c), max(u) FROM (SELECT number AS k, count() AS c, uniqArray(range(number % 16)) AS u FROM (SELECT * FROM system.numbers LIMIT 1000000) GROUP BY k); - -SET max_memory_usage = 0; -SET group_by_two_level_threshold = 100000; -SET max_bytes_before_external_group_by = '1Mi'; - --- method: key_string & key_string_two_level -CREATE TABLE t_00284_str(s String) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; -INSERT INTO t_00284_str SELECT toString(number) FROM numbers_mt(1e6); -INSERT INTO t_00284_str SELECT toString(number) FROM numbers_mt(1e6); -SELECT s, count() FROM t_00284_str GROUP BY s ORDER BY s LIMIT 10 OFFSET 42; - --- method: low_cardinality_key_string & low_cardinality_key_string_two_level -CREATE TABLE t_00284_lc_str(s LowCardinality(String)) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; -INSERT INTO t_00284_lc_str SELECT toString(number) FROM numbers_mt(1e6); -INSERT INTO t_00284_lc_str SELECT toString(number) FROM numbers_mt(1e6); -SELECT s, count() FROM t_00284_lc_str GROUP BY s ORDER BY s LIMIT 10 OFFSET 42; diff --git a/tests/queries/0_stateless/00284_external_aggregation_2.reference b/tests/queries/0_stateless/00284_external_aggregation_2.reference new file mode 100644 index 00000000000..71d2e96d4b0 --- /dev/null +++ b/tests/queries/0_stateless/00284_external_aggregation_2.reference @@ -0,0 +1,20 @@ +100033 2 +100034 2 +100035 2 +100036 2 +100037 2 +100038 2 +100039 2 +10004 2 +100040 2 +100041 2 +100033 2 +100034 2 +100035 2 +100036 2 +100037 2 +100038 2 +100039 2 +10004 2 +100040 2 +100041 2 diff --git a/tests/queries/0_stateless/00284_external_aggregation_2.sql b/tests/queries/0_stateless/00284_external_aggregation_2.sql new file mode 100644 index 00000000000..7960e3894d0 --- /dev/null +++ b/tests/queries/0_stateless/00284_external_aggregation_2.sql @@ -0,0 +1,22 @@ +-- Tags: long + +-- This test was split in two due to long runtimes in sanitizers. +-- The other part is 00284_external_aggregation. + +SET group_by_two_level_threshold_bytes = 50000000; +SET max_memory_usage = 0; +SET group_by_two_level_threshold = 100000; +SET max_bytes_before_external_group_by = '1Mi'; + +-- method: key_string & key_string_two_level +CREATE TABLE t_00284_str(s String) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; +INSERT INTO t_00284_str SELECT toString(number) FROM numbers_mt(1e6); +INSERT INTO t_00284_str SELECT toString(number) FROM numbers_mt(1e6); +SELECT s, count() FROM t_00284_str GROUP BY s ORDER BY s LIMIT 10 OFFSET 42; + +-- method: low_cardinality_key_string & low_cardinality_key_string_two_level +CREATE TABLE t_00284_lc_str(s LowCardinality(String)) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; +INSERT INTO t_00284_lc_str SELECT toString(number) FROM numbers_mt(1e6); +INSERT INTO t_00284_lc_str SELECT toString(number) FROM numbers_mt(1e6); +SELECT s, count() FROM t_00284_lc_str GROUP BY s ORDER BY s LIMIT 10 OFFSET 42; + diff --git a/tests/queries/0_stateless/02099_tsv_raw_format_1.sh b/tests/queries/0_stateless/02099_tsv_raw_format_1.sh index a3468f46ca0..bd1f8731717 100755 --- a/tests/queries/0_stateless/02099_tsv_raw_format_1.sh +++ b/tests/queries/0_stateless/02099_tsv_raw_format_1.sh @@ -1,6 +1,9 @@ #!/usr/bin/env bash # Tags: long +# This test was split in two due to long runtimes in sanitizers. +# The other part is 02099_tsv_raw_format_2.sh. + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh diff --git a/tests/queries/0_stateless/02099_tsv_raw_format_2.sh b/tests/queries/0_stateless/02099_tsv_raw_format_2.sh index d6034a0616f..9f57eea42f2 100755 --- a/tests/queries/0_stateless/02099_tsv_raw_format_2.sh +++ b/tests/queries/0_stateless/02099_tsv_raw_format_2.sh @@ -1,6 +1,9 @@ #!/usr/bin/env bash # Tags: long +# This test was split in two due to long runtimes in sanitizers. +# The other part is 02099_tsv_raw_format_1.sh. +# CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh From 9cf60214e13da94ca2dcee82491d6795c784aaa5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 8 Aug 2024 13:43:12 +0000 Subject: [PATCH 473/644] Update version_date.tsv and changelogs after v24.7.3.42-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.7.3.42-stable.md | 37 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 5 files changed, 41 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v24.7.3.42-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 94603763572..a44664259fb 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.7.2.13" +ARG VERSION="24.7.3.42" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index f40118c7b06..2565828c846 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.7.2.13" +ARG VERSION="24.7.3.42" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 032aa862e4a..5ac8a58afea 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -28,7 +28,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.7.2.13" +ARG VERSION="24.7.3.42" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.7.3.42-stable.md b/docs/changelogs/v24.7.3.42-stable.md new file mode 100644 index 00000000000..48f6e301f3c --- /dev/null +++ b/docs/changelogs/v24.7.3.42-stable.md @@ -0,0 +1,37 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.7.3.42-stable (63730bc4293) FIXME as compared to v24.7.2.13-stable (6e41f601b2f) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#67969](https://github.com/ClickHouse/ClickHouse/issues/67969): Fixed reading of subcolumns after `ALTER ADD COLUMN` query. [#66243](https://github.com/ClickHouse/ClickHouse/pull/66243) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#67637](https://github.com/ClickHouse/ClickHouse/issues/67637): Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#67820](https://github.com/ClickHouse/ClickHouse/issues/67820): Fix possible deadlock on query cancel with parallel replicas. [#66905](https://github.com/ClickHouse/ClickHouse/pull/66905) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67881](https://github.com/ClickHouse/ClickHouse/issues/67881): Correctly parse file name/URI containing `::` if it's not an archive. [#67433](https://github.com/ClickHouse/ClickHouse/pull/67433) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67713](https://github.com/ClickHouse/ClickHouse/issues/67713): Fix reloading SQL UDFs with UNION. Previously, restarting the server could make UDF invalid. [#67665](https://github.com/ClickHouse/ClickHouse/pull/67665) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67995](https://github.com/ClickHouse/ClickHouse/issues/67995): Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) + +* Backported in [#67818](https://github.com/ClickHouse/ClickHouse/issues/67818): Only relevant to the experimental Variant data type. Fix crash with Variant + AggregateFunction type. [#67122](https://github.com/ClickHouse/ClickHouse/pull/67122) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67766](https://github.com/ClickHouse/ClickHouse/issues/67766): Fix crash of `uniq` and `uniqTheta ` with `tuple()` argument. Closes [#67303](https://github.com/ClickHouse/ClickHouse/issues/67303). [#67306](https://github.com/ClickHouse/ClickHouse/pull/67306) ([flynn](https://github.com/ucasfl)). +* Backported in [#67854](https://github.com/ClickHouse/ClickHouse/issues/67854): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#67840](https://github.com/ClickHouse/ClickHouse/issues/67840): Fix potential stack overflow in `JSONMergePatch` function. Renamed this function from `jsonMergePatch` to `JSONMergePatch` because the previous name was wrong. The previous name is still kept for compatibility. Improved diagnostic of errors in the function. This closes [#67304](https://github.com/ClickHouse/ClickHouse/issues/67304). [#67756](https://github.com/ClickHouse/ClickHouse/pull/67756) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#67518](https://github.com/ClickHouse/ClickHouse/issues/67518): Split slow test 03036_dynamic_read_subcolumns. [#66954](https://github.com/ClickHouse/ClickHouse/pull/66954) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67516](https://github.com/ClickHouse/ClickHouse/issues/67516): Split 01508_partition_pruning_long. [#66983](https://github.com/ClickHouse/ClickHouse/pull/66983) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67529](https://github.com/ClickHouse/ClickHouse/issues/67529): Reduce max time of 00763_long_lock_buffer_alter_destination_table. [#67185](https://github.com/ClickHouse/ClickHouse/pull/67185) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#67643](https://github.com/ClickHouse/ClickHouse/issues/67643): [Green CI] Fix potentially flaky test_mask_sensitive_info integration test. [#67506](https://github.com/ClickHouse/ClickHouse/pull/67506) ([Alexey Katsman](https://github.com/alexkats)). +* Backported in [#67609](https://github.com/ClickHouse/ClickHouse/issues/67609): Fix test_zookeeper_config_load_balancing after adding the xdist worker name to the instance. [#67590](https://github.com/ClickHouse/ClickHouse/pull/67590) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#67871](https://github.com/ClickHouse/ClickHouse/issues/67871): Fix 02434_cancel_insert_when_client_dies. [#67600](https://github.com/ClickHouse/ClickHouse/pull/67600) ([vdimir](https://github.com/vdimir)). +* Backported in [#67704](https://github.com/ClickHouse/ClickHouse/issues/67704): Fix 02910_bad_logs_level_in_local in fast tests. [#67603](https://github.com/ClickHouse/ClickHouse/pull/67603) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#67689](https://github.com/ClickHouse/ClickHouse/issues/67689): Fix 01605_adaptive_granularity_block_borders. [#67605](https://github.com/ClickHouse/ClickHouse/pull/67605) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67827](https://github.com/ClickHouse/ClickHouse/issues/67827): Try fix 03143_asof_join_ddb_long. [#67620](https://github.com/ClickHouse/ClickHouse/pull/67620) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67892](https://github.com/ClickHouse/ClickHouse/issues/67892): Revert "Merge pull request [#66510](https://github.com/ClickHouse/ClickHouse/issues/66510) from canhld94/fix_trivial_count_non_deterministic_func". [#67800](https://github.com/ClickHouse/ClickHouse/pull/67800) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index e410f31ca5a..f46353277e2 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.3.95-stable 2024-08-06 From cab274e1b696e8e355066cce3b05d4337c486157 Mon Sep 17 00:00:00 2001 From: kruglov Date: Fri, 2 Aug 2024 10:46:56 +0300 Subject: [PATCH 474/644] Fixed error on generated columns in MaterializedPostgreSQL --- .../fetchPostgreSQLTableStructure.cpp | 34 +++++++++----- .../fetchPostgreSQLTableStructure.h | 1 + .../test.py | 44 ++++++++++++++++++- 3 files changed, 67 insertions(+), 12 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 943f3ae502e..e2f2358c892 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -196,7 +196,7 @@ PostgreSQLTableStructure::ColumnsInfoPtr readNamesAndTypesList( } else { - std::tuple row; + std::tuple row; while (stream >> row) { const auto column_name = std::get<0>(row); @@ -206,13 +206,14 @@ PostgreSQLTableStructure::ColumnsInfoPtr readNamesAndTypesList( std::get<3>(row)); columns.push_back(NameAndTypePair(column_name, data_type)); - auto attgenerated = std::get<6>(row); + auto attgenerated = std::get<7>(row); attributes.emplace( column_name, PostgreSQLTableStructure::PGAttribute{ .atttypid = parse(std::get<4>(row)), .atttypmod = parse(std::get<5>(row)), + .attnum = parse(std::get<6>(row)), .atthasdef = false, .attgenerated = attgenerated.empty() ? char{} : char(attgenerated[0]), .attr_def = {} @@ -308,6 +309,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( "attndims AS dims, " /// array dimensions "atttypid as type_id, " "atttypmod as type_modifier, " + "attnum as att_num, " "attgenerated as generated " /// if column has GENERATED "FROM pg_attribute " "WHERE attrelid = (SELECT oid FROM pg_class WHERE {}) " @@ -338,17 +340,29 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( "WHERE adrelid = (SELECT oid FROM pg_class WHERE {});", where); pqxx::result result{tx.exec(attrdef_query)}; - for (const auto row : result) + if (static_cast(result.size()) > table.physical_columns->names.size()) { - size_t adnum = row[0].as(); - if (!adnum || adnum > table.physical_columns->names.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Received {} attrdef, but currently fetched columns list has {} columns", + result.size(), table.physical_columns->attributes.size()); + } + + for (const auto & column_attrs : table.physical_columns->attributes) + { + if (column_attrs.second.attgenerated != 's') { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Received adnum {}, but currently fetched columns list has {} columns", - adnum, table.physical_columns->attributes.size()); + continue; + } + + for (const auto row : result) + { + int adnum = row[0].as(); + if (column_attrs.second.attnum == adnum) + { + table.physical_columns->attributes.at(column_attrs.first).attr_def = row[1].as(); + break; + } } - const auto column_name = table.physical_columns->names[adnum - 1]; - table.physical_columns->attributes.at(column_name).attr_def = row[1].as(); } } diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index 81bf7b278fc..25ece6909fd 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -16,6 +16,7 @@ struct PostgreSQLTableStructure { Int32 atttypid; Int32 atttypmod; + Int32 attnum; bool atthasdef; char attgenerated; std::string attr_def; diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 406b50bc486..75edb22aab1 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -953,12 +953,14 @@ def test_generated_columns(started_cluster): "", f"""CREATE TABLE {table} ( key integer PRIMARY KEY, - x integer, + x integer DEFAULT 0, + temp integer DEFAULT 0, y integer GENERATED ALWAYS AS (x*2) STORED, - z text); + z text DEFAULT 'z'); """, ) + pg_manager.execute(f"alter table {table} drop column temp;") pg_manager.execute(f"insert into {table} (key, x, z) values (1,1,'1');") pg_manager.execute(f"insert into {table} (key, x, z) values (2,2,'2');") @@ -991,6 +993,44 @@ def test_generated_columns(started_cluster): ) +def test_generated_columns_with_sequence(started_cluster): + table = "test_generated_columns_with_sequence" + + pg_manager.create_postgres_table( + table, + "", + f"""CREATE TABLE {table} ( + key integer PRIMARY KEY, + x integer, + y integer GENERATED ALWAYS AS (x*2) STORED, + z text); + """, + ) + + pg_manager.execute( + f"create sequence {table}_id_seq increment by 1 minvalue 1 start 1;" + ) + pg_manager.execute( + f"alter table {table} alter key set default nextval('{table}_id_seq');" + ) + pg_manager.execute(f"insert into {table} (key, x, z) values (1,1,'1');") + pg_manager.execute(f"insert into {table} (key, x, z) values (2,2,'2');") + + pg_manager.create_materialized_db( + ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=[ + f"materialized_postgresql_tables_list = '{table}'", + "materialized_postgresql_backoff_min_ms = 100", + "materialized_postgresql_backoff_max_ms = 100", + ], + ) + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + + def test_default_columns(started_cluster): table = "test_default_columns" From 72bc5cd2e99cee09d0e003fb75192c0bb3114bad Mon Sep 17 00:00:00 2001 From: Kruglov Kirill Date: Mon, 5 Aug 2024 16:10:27 +0300 Subject: [PATCH 475/644] Update src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index e2f2358c892..b9fd9c325f8 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -349,7 +349,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( for (const auto & column_attrs : table.physical_columns->attributes) { - if (column_attrs.second.attgenerated != 's') + if (column_attrs.second.attgenerated != 's') /// e.g. not a generated column { continue; } From 55b2000d38e0bc6282714fdb1204d450437433ec Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 13:58:43 +0000 Subject: [PATCH 476/644] Fix fasttest --- .../02864_statistics_delayed_materialization_in_merge.sql | 2 +- tests/queries/0_stateless/02864_statistics_usage.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql index 33a5f9052ba..d469a4c2036 100644 --- a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql +++ b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS tab; SET allow_experimental_statistics = 1; SET allow_statistics_optimize = 1; -SET allow_analyzer = 1; +SET enable_analyzer = 1; SET materialize_statistics_on_insert = 0; diff --git a/tests/queries/0_stateless/02864_statistics_usage.sql b/tests/queries/0_stateless/02864_statistics_usage.sql index f936854df44..4956bd27e87 100644 --- a/tests/queries/0_stateless/02864_statistics_usage.sql +++ b/tests/queries/0_stateless/02864_statistics_usage.sql @@ -6,7 +6,7 @@ SET allow_experimental_statistics = 1; SET allow_statistics_optimize = 1; SET mutations_sync = 1; -SET allow_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS tab; From 34bbf3d9c49f0f563eee8973ac3c3bae0e64c6de Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Aug 2024 17:12:27 +0200 Subject: [PATCH 477/644] Use async inserts --- docker/test/stateless/run.sh | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 830a02a64a3..c81f33ace01 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -192,8 +192,9 @@ ENGINE = MergeTree ORDER BY tuple()" # create minio log webhooks for both audit and server logs -./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" -./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" +# use async inserts to avoid creating too many parts +./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" +./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" max_retries=100 retry=1 @@ -375,6 +376,12 @@ done # collect minio audit and server logs +has_async_inserts=$(clickhouse-client -q "SELECT count() FROM system.asynchronous_inserts WHERE table = 'minio_audit_logs' OR table = 'minio_server_logs'") +if [[ has_async_inserts -eq 1 ]]; then + echo "Waiting for async inserts to flush" + sleep 5 +fi + clickhouse-client -q "SELECT log FROM minio_audit_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_audit_logs.jsonl.zst' FORMAT JSONEachRow" clickhouse-client -q "SELECT log FROM minio_server_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_server_logs.jsonl.zst' FORMAT JSONEachRow" From 33ba78ee42bc85690dce69c82fd51d723a6d2eab Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 8 Aug 2024 17:47:12 +0200 Subject: [PATCH 478/644] Update test.py --- tests/integration/test_drop_is_lock_free/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_drop_is_lock_free/test.py b/tests/integration/test_drop_is_lock_free/test.py index 1bb8767a9a0..3855bc21f90 100644 --- a/tests/integration/test_drop_is_lock_free/test.py +++ b/tests/integration/test_drop_is_lock_free/test.py @@ -176,7 +176,7 @@ def test_query_is_permanent(transaction, permanent, exclusive_table): select_handler = node.get_query_request( f""" - SELECT sleepEachRow(3) FROM {exclusive_table} SETTINGS function_sleep_max_microseconds_per_block = 0; + SELECT sleepEachRow(3) FROM {exclusive_table} SETTINGS function_sleep_max_microseconds_per_block = 0, max_threads=1; """, query_id=query_id, ) From e264ecd2011c7860c4898208a53555d676222bbb Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Aug 2024 16:10:46 +0000 Subject: [PATCH 479/644] fix skip of parts in mutation with analyzer --- src/Interpreters/MutationsInterpreter.cpp | 11 +++++----- src/Interpreters/MutationsInterpreter.h | 1 - src/Storages/MergeTree/MutateTask.cpp | 2 +- ...3221_mutation_analyzer_skip_part.reference | 4 ++++ .../03221_mutation_analyzer_skip_part.sql | 21 +++++++++++++++++++ 5 files changed, 31 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/03221_mutation_analyzer_skip_part.reference create mode 100644 tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 57ad5caa4c7..5de33b3ed86 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -146,7 +146,6 @@ ColumnDependencies getAllColumnDependencies( bool isStorageTouchedByMutations( - MergeTreeData & storage, MergeTreeData::DataPartPtr source_part, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, @@ -155,7 +154,9 @@ bool isStorageTouchedByMutations( if (commands.empty()) return false; + auto storage_from_part = std::make_shared(source_part); bool all_commands_can_be_skipped = true; + for (const auto & command : commands) { if (command.type == MutationCommand::APPLY_DELETED_MASK) @@ -170,7 +171,7 @@ bool isStorageTouchedByMutations( if (command.partition) { - const String partition_id = storage.getPartitionIDFromQuery(command.partition, context); + const String partition_id = storage_from_part->getPartitionIDFromQuery(command.partition, context); if (partition_id == source_part->info.partition_id) all_commands_can_be_skipped = false; } @@ -184,20 +185,18 @@ bool isStorageTouchedByMutations( if (all_commands_can_be_skipped) return false; - auto storage_from_part = std::make_shared(source_part); - std::optional interpreter_select_query; BlockIO io; if (context->getSettingsRef().allow_experimental_analyzer) { - auto select_query_tree = prepareQueryAffectedQueryTree(commands, storage.shared_from_this(), context); + auto select_query_tree = prepareQueryAffectedQueryTree(commands, storage_from_part, context); InterpreterSelectQueryAnalyzer interpreter(select_query_tree, context, SelectQueryOptions().ignoreLimits()); io = interpreter.execute(); } else { - ASTPtr select_query = prepareQueryAffectedAST(commands, storage.shared_from_this(), context); + ASTPtr select_query = prepareQueryAffectedAST(commands, storage_from_part, context); /// Interpreter must be alive, when we use result of execute() method. /// For some reason it may copy context and give it into ExpressionTransform /// after that we will use context from destroyed stack frame in our stream. diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 6aaa233cda3..57863e9ae73 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -19,7 +19,6 @@ using QueryPipelineBuilderPtr = std::unique_ptr; /// Return false if the data isn't going to be changed by mutations. bool isStorageTouchedByMutations( - MergeTreeData & storage, MergeTreeData::DataPartPtr source_part, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 9a775db73e2..5fcf699de59 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2134,7 +2134,7 @@ bool MutateTask::prepare() ctx->commands_for_part.emplace_back(command); if (ctx->source_part->isStoredOnDisk() && !isStorageTouchedByMutations( - *ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->commands_for_part, context_for_reading)) + ctx->source_part, ctx->metadata_snapshot, ctx->commands_for_part, context_for_reading)) { NameSet files_to_copy_instead_of_hardlinks; auto settings_ptr = ctx->data->getSettings(); diff --git a/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.reference b/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.reference new file mode 100644 index 00000000000..68f8708dbaf --- /dev/null +++ b/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.reference @@ -0,0 +1,4 @@ +1_1_1_0_3 10000 +1_1_1_0_4 0 +2_2_2_0_3 0 +2_2_2_0_4 10000 diff --git a/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql b/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql new file mode 100644 index 00000000000..bf9a10e2af4 --- /dev/null +++ b/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS t_mutate_skip_part; + +CREATE TABLE t_mutate_skip_part (key UInt64, id UInt64, v1 UInt64, v2 UInt64) ENGINE = MergeTree ORDER BY id PARTITION BY key; + +INSERT INTO t_mutate_skip_part SELECT 1, number, number, number FROM numbers(10000); +INSERT INTO t_mutate_skip_part SELECT 2, number, number, number FROM numbers(10000); + +SET mutations_sync = 2; + +ALTER TABLE t_mutate_skip_part UPDATE v1 = 1000 WHERE key = 1; +ALTER TABLE t_mutate_skip_part DELETE WHERE key = 2 AND v2 % 10 = 0; + +SYSTEM FLUSH LOGS; + +-- If part is skipped in mutation and hardlinked then read_rows must be 0. +SELECT part_name, read_rows +FROM system.part_log +WHERE database = currentDatabase() AND table = 't_mutate_skip_part' AND event_type = 'MutatePart' +ORDER BY part_name; + +DROP TABLE IF EXISTS t_mutate_skip_part; From f9f13a8e415e9f2130281d069c28dd6e9a68be75 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Aug 2024 16:27:25 +0000 Subject: [PATCH 480/644] enable setting optimize_functions_to_subcolumns by default --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f04c696645a..f0a8d0c2647 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -605,7 +605,7 @@ class IColumn; M(Bool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \ M(Bool, optimize_multiif_to_if, true, "Replace 'multiIf' with only one condition to 'if'.", 0) \ M(Bool, optimize_if_transform_strings_to_enum, false, "Replaces string-type arguments in If and Transform to enum. Disabled by default cause it could make inconsistent change in distributed query that would lead to its fail.", 0) \ - M(Bool, optimize_functions_to_subcolumns, false, "Transform functions to subcolumns, if possible, to reduce amount of read data. E.g. 'length(arr)' -> 'arr.size0', 'col IS NULL' -> 'col.null' ", 0) \ + M(Bool, optimize_functions_to_subcolumns, true, "Transform functions to subcolumns, if possible, to reduce amount of read data. E.g. 'length(arr)' -> 'arr.size0', 'col IS NULL' -> 'col.null' ", 0) \ M(Bool, optimize_using_constraints, false, "Use constraints for query optimization", 0) \ M(Bool, optimize_substitute_columns, false, "Use constraints for column substitution", 0) \ M(Bool, optimize_append_index, false, "Use constraints in order to append index condition (indexHint)", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 41e4ac2e154..b00d0964e01 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -83,6 +83,7 @@ static std::initializer_list Date: Thu, 8 Aug 2024 18:31:24 +0200 Subject: [PATCH 481/644] Fix --- ...ckhouse_local_interactive_exception.expect | 29 ----- ...ckhouse_local_interactive_exception.python | 110 ++++++++++++++++++ ...ouse_local_interactive_exception.reference | 1 + 3 files changed, 111 insertions(+), 29 deletions(-) delete mode 100755 tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect create mode 100644 tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python diff --git a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect deleted file mode 100755 index add977c4fce..00000000000 --- a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect +++ /dev/null @@ -1,29 +0,0 @@ -#!/usr/bin/expect -f - -set basedir [file dirname $argv0] -set basename [file tail $argv0] -if {[info exists env(CLICKHOUSE_TMP)]} { - set CLICKHOUSE_TMP $env(CLICKHOUSE_TMP) -} else { - set CLICKHOUSE_TMP "." -} -exp_internal -f $CLICKHOUSE_TMP/$basename.debuglog 0 - -log_user 0 -set timeout 20 -match_max 100000 - -expect_after { - -i $any_spawn_id eof { exp_continue } - -i $any_spawn_id timeout { exit 1 } -} - -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_LOCAL --disable_suggestion" - -expect ":) " -send -- "insert into table function null() format TSV some trash here 123 \n 456\r" -expect "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -expect ":) " - -send -- "" -expect eof diff --git a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python new file mode 100644 index 00000000000..03f8d493ec2 --- /dev/null +++ b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python @@ -0,0 +1,110 @@ +import pty +import os +import shlex +import time +import multiprocessing + +COMPLETION_TIMEOUT_SECONDS = 30 +DEBUG_LOG = os.path.join( + os.environ["CLICKHOUSE_TMP"], + os.path.basename(os.path.abspath(__file__)).strip(".python") + ".debuglog", +) + +STATE_MAP = { + -1: "process did not start", + 0: "all good", + 1: "process started and said ':)'", + 2: "prompt search was started", + 3: "prompt is missing", +} + + +def run_with_timeout(func, args, timeout): + for _ in range(5): + state = multiprocessing.Value("i", -1) + process = multiprocessing.Process( + target=func, args=args, kwargs={"state": state} + ) + process.start() + process.join(timeout) + + if state.value in (0, 3): + return + + if process.is_alive(): + process.terminate() + + if state.value == -1: + continue + + print(f"Timeout, state: {STATE_MAP[state.value]}") + return + + +def test_completion(program, argv, prompt, state=None): + shell_pid, master = pty.fork() + if shell_pid == 0: + os.execv(program, argv) + else: + try: + debug_log_fd = open(DEBUG_LOG, "a") + + output_b = os.read(master, 4096) + output = output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + while not ":)" in output: + output_b = os.read(master, 4096) + output += output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + + state.value = 1 + + os.write(master, bytes(prompt.encode())) + output_b = os.read(master, 4096) + output = output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + while not prompt[:-10] in output: + output_b = os.read(master, 4096) + output += output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + + time.sleep(0.01) + os.write(master, b"\r") + + state.value = 2 + + output_b = os.read(master, 4096) + output = output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + while not "CANNOT_PARSE_INPUT_ASSERTION_FAILED" in output: + output_b = os.read(master, 4096) + output += output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + + while not ":)" in output: + output_b = os.read(master, 4096) + output += output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + + print("OK") + state.value = 0 + finally: + os.close(master) + debug_log_fd.close() + + +if __name__ == "__main__": + clickhouse_local = os.environ["CLICKHOUSE_LOCAL"] + args = shlex.split(clickhouse_local) + args.append("--disable_suggestion") + args.append("--highlight=0") + run_with_timeout( + test_completion, [args[0], args, "insert into table function null() format TSV some trash here 123 \n 456"], COMPLETION_TIMEOUT_SECONDS + ) diff --git a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.reference b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.reference index e69de29bb2d..d86bac9de59 100644 --- a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.reference +++ b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.reference @@ -0,0 +1 @@ +OK From 34d2c71eadfefd79e3cfb62d46d156cdbdaab681 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 8 Aug 2024 18:50:16 +0200 Subject: [PATCH 482/644] Cleanup --- ...ckhouse_local_interactive_exception.python | 44 +++++++------------ 1 file changed, 15 insertions(+), 29 deletions(-) diff --git a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python index 03f8d493ec2..9527991c36e 100644 --- a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python +++ b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python @@ -40,6 +40,18 @@ def run_with_timeout(func, args, timeout): print(f"Timeout, state: {STATE_MAP[state.value]}") return +def expect(text, master, debug_log_fd): + output_b = os.read(master, 4096) + output = output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + while not text in output: + output_b = os.read(master, 4096) + output += output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + + return output def test_completion(program, argv, prompt, state=None): shell_pid, master = pty.fork() @@ -49,43 +61,17 @@ def test_completion(program, argv, prompt, state=None): try: debug_log_fd = open(DEBUG_LOG, "a") - output_b = os.read(master, 4096) - output = output_b.decode() - debug_log_fd.write(repr(output_b) + "\n") - debug_log_fd.flush() - while not ":)" in output: - output_b = os.read(master, 4096) - output += output_b.decode() - debug_log_fd.write(repr(output_b) + "\n") - debug_log_fd.flush() + expect(":)", master, debug_log_fd) state.value = 1 - os.write(master, bytes(prompt.encode())) - output_b = os.read(master, 4096) - output = output_b.decode() - debug_log_fd.write(repr(output_b) + "\n") - debug_log_fd.flush() - while not prompt[:-10] in output: - output_b = os.read(master, 4096) - output += output_b.decode() - debug_log_fd.write(repr(output_b) + "\n") - debug_log_fd.flush() + expect(prompt[:-10], master, debug_log_fd) time.sleep(0.01) os.write(master, b"\r") - state.value = 2 - output_b = os.read(master, 4096) - output = output_b.decode() - debug_log_fd.write(repr(output_b) + "\n") - debug_log_fd.flush() - while not "CANNOT_PARSE_INPUT_ASSERTION_FAILED" in output: - output_b = os.read(master, 4096) - output += output_b.decode() - debug_log_fd.write(repr(output_b) + "\n") - debug_log_fd.flush() + output = expect("CANNOT_PARSE_INPUT_ASSERTION_FAILED", master, debug_log_fd) while not ":)" in output: output_b = os.read(master, 4096) From cd69fa5a4c6f7128c473c780c797d60dc314b2ee Mon Sep 17 00:00:00 2001 From: Jacob Reckhard Date: Thu, 8 Aug 2024 11:20:55 -0600 Subject: [PATCH 483/644] fixed typos --- docs/en/sql-reference/functions/geo/polygon.md | 2 +- src/Functions/geometryConverters.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/polygon.md b/docs/en/sql-reference/functions/geo/polygon.md index c054e05d39c..be9e9810626 100644 --- a/docs/en/sql-reference/functions/geo/polygon.md +++ b/docs/en/sql-reference/functions/geo/polygon.md @@ -197,7 +197,7 @@ The function returns a ClickHouse internal representation of the multilinestring ### Example ```sql -SELECT wkt(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3), (4 4, 5 5, 6 6))')); +SELECT readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3), (4 4, 5 5, 6 6))'); ``` ```response diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index bf975017a6d..f1156d81f01 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -471,7 +471,7 @@ static void callOnGeometryDataType(DataTypePtr type, F && f) return f(ConverterType>()); /// We should take the name into consideration to avoid ambiguity. - /// Because for example both MultiLineString and Polygon are resolved to Array(Tuple(Point)). + /// Because for example both MultiLineString and Polygon are resolved to Array(Array(Point)). else if (factory.get("MultiLineString")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "MultiLineString") return f(ConverterType>()); From e55d81fd427af0a62e3534d838d64eb847419248 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 8 Aug 2024 19:38:48 +0200 Subject: [PATCH 484/644] fix test --- .../0_stateless/02808_custom_disk_with_user_defined_name.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh index b62adea5683..63fa60bd548 100755 --- a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh +++ b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh @@ -13,7 +13,7 @@ DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) ENGINE = MergeTree() ORDER BY tuple() SETTINGS disk = disk(name = 's3_disk', type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); -""" 2>&1 | grep -q "The disk \`s3_disk\` is already exist and described by the config" && echo 'OK' || echo 'FAIL' +""" 2>&1 | grep -q "Disk \`s3_disk\` already exists and is described by the config" && echo 'OK' || echo 'FAIL' disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}" From 2bd7606280b0714febc1d034233ee0fb9a3532a6 Mon Sep 17 00:00:00 2001 From: Jacob Reckhard Date: Thu, 8 Aug 2024 11:39:06 -0600 Subject: [PATCH 485/644] added additional tests --- .../03215_multilinestring_geometry.reference | 10 ++++++++++ .../0_stateless/03215_multilinestring_geometry.sql | 12 ++++++++++++ 2 files changed, 22 insertions(+) diff --git a/tests/queries/0_stateless/03215_multilinestring_geometry.reference b/tests/queries/0_stateless/03215_multilinestring_geometry.reference index f4c5774018e..f20e21d86f6 100644 --- a/tests/queries/0_stateless/03215_multilinestring_geometry.reference +++ b/tests/queries/0_stateless/03215_multilinestring_geometry.reference @@ -15,3 +15,13 @@ MULTILINESTRING((1 1,2 2,3 3,1 1),(1 0,2 0,3 0)) WITH wkt(CAST([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'Array(Array(Tuple(Float64, Float64)))')) as x SELECT x, toTypeName(x), readWKTPolygon(x) as y, toTypeName(y); POLYGON((1 1,2 2,3 3,1 1)) String [[(1,1),(2,2),(3,3),(1,1)]] Polygon +-- Non constant tests + +CREATE TABLE IF NOT EXISTS t (shape Array(Array(Tuple(Float64, Float64))), wkt_string String) Engine = Memory; +INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); +INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)], [(1, 0), (2, 0), (3, 0)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); +-- Native Array(Array(Tuple(Float64, Float64))) is treated as Polygon, not as MultiLineString. +-- but reading MultiLineString should still return an Array(Array(Tuple(Float64, Float64))) +select wkt(shape), readWKTMultiLineString(wkt_string), readWKTMultiLineString(wkt_string) = shape from t; +POLYGON((1 1,2 2,3 3,1 1)) [[(1,1),(2,2),(3,3),(1,1)]] 1 +POLYGON((1 1,2 2,3 3,1 1),(1 0,2 0,3 0,1 0)) [[(1,1),(2,2),(3,3),(1,1)],[(1,0),(2,0),(3,0)]] 1 diff --git a/tests/queries/0_stateless/03215_multilinestring_geometry.sql b/tests/queries/0_stateless/03215_multilinestring_geometry.sql index 71344920c52..6081e081fb3 100644 --- a/tests/queries/0_stateless/03215_multilinestring_geometry.sql +++ b/tests/queries/0_stateless/03215_multilinestring_geometry.sql @@ -10,3 +10,15 @@ SELECT wkt(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, -- Native Array(Array(Tuple(Float64, Float64))) is treated as Polygon, not as MultiLineString. WITH wkt(CAST([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'Array(Array(Tuple(Float64, Float64)))')) as x SELECT x, toTypeName(x), readWKTPolygon(x) as y, toTypeName(y); + +-- Non constant tests + +CREATE TABLE IF NOT EXISTS t (shape Array(Array(Tuple(Float64, Float64))), wkt_string String) Engine = Memory; +INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); +INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)], [(1, 0), (2, 0), (3, 0)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); + +-- Native Array(Array(Tuple(Float64, Float64))) is treated as Polygon, not as MultiLineString. +-- but reading MultiLineString should still return an Array(Array(Tuple(Float64, Float64))) +select wkt(shape), readWKTMultiLineString(wkt_string), readWKTMultiLineString(wkt_string) = shape from t; + + From b0b150c599052d596b8c746a31ba50019601b6a0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 18:29:20 +0000 Subject: [PATCH 486/644] Remove workarounds for solved bugs --- .../02864_statistics_predicates.reference | 6 ++++++ .../0_stateless/02864_statistics_predicates.sql | 12 ++++++------ 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02864_statistics_predicates.reference b/tests/queries/0_stateless/02864_statistics_predicates.reference index 1c2abd47aaf..ffbd7269e05 100644 --- a/tests/queries/0_stateless/02864_statistics_predicates.reference +++ b/tests/queries/0_stateless/02864_statistics_predicates.reference @@ -23,6 +23,7 @@ u64 and < 70 70 70 +70 f64 and = 10 10 @@ -35,6 +36,8 @@ f64 and = 10 10 10 +10 +0 0 0 0 @@ -50,6 +53,8 @@ f64 and < 70 70 70 +70 +80 80 80 80 @@ -66,6 +71,7 @@ dt and < 10000 10000 10000 +10000 70 70 70 diff --git a/tests/queries/0_stateless/02864_statistics_predicates.sql b/tests/queries/0_stateless/02864_statistics_predicates.sql index 3e754dfb1de..779116cf19a 100644 --- a/tests/queries/0_stateless/02864_statistics_predicates.sql +++ b/tests/queries/0_stateless/02864_statistics_predicates.sql @@ -92,7 +92,7 @@ SELECT count(*) FROM tab WHERE u64_count_min < 7.7; SELECT count(*) FROM tab WHERE u64_uniq < 7.7; SELECT count(*) FROM tab WHERE u64 < '7'; --- SELECT count(*) FROM tab WHERE u64_tdigest < '7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE u64_tdigest < '7'; SELECT count(*) FROM tab WHERE u64_count_min < '7'; SELECT count(*) FROM tab WHERE u64_uniq < '7'; @@ -116,12 +116,12 @@ SELECT count(*) FROM tab WHERE f64_count_min = 7.7; SELECT count(*) FROM tab WHERE f64_uniq = 7.7; SELECT count(*) FROM tab WHERE f64 = '7'; --- SELECT count(*) FROM tab WHERE f64_tdigest = '7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_tdigest = '7'; SELECT count(*) FROM tab WHERE f64_count_min = '7'; SELECT count(*) FROM tab WHERE f64_uniq = '7'; SELECT count(*) FROM tab WHERE f64 = '7.7'; --- SELECT count(*) FROM tab WHERE f64_tdigest = '7.7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_tdigest = '7.7'; SELECT count(*) FROM tab WHERE f64_count_min = '7.7'; SELECT count(*) FROM tab WHERE f64_uniq = '7.7'; @@ -138,12 +138,12 @@ SELECT count(*) FROM tab WHERE f64_count_min < 7.7; SELECT count(*) FROM tab WHERE f64_uniq < 7.7; SELECT count(*) FROM tab WHERE f64 < '7'; --- SELECT count(*) FROM tab WHERE f64_tdigest < '7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_tdigest < '7'; SELECT count(*) FROM tab WHERE f64_count_min < '7'; SELECT count(*) FROM tab WHERE f64_uniq < '7'; SELECT count(*) FROM tab WHERE f64 < '7.7'; --- SELECT count(*) FROM tab WHERE f64_tdigest < '7.7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_tdigest < '7.7'; SELECT count(*) FROM tab WHERE f64_count_min < '7.7'; SELECT count(*) FROM tab WHERE f64_uniq < '7.7'; @@ -164,7 +164,7 @@ SELECT count(*) FROM tab WHERE dt_uniq = 7; SELECT 'dt and <'; SELECT count(*) FROM tab WHERE dt < '2024-08-08 11:12:13'; --- SELECT count(*) FROM tab WHERE dt_tdigest < '2024-08-08 11:12:13'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE dt_tdigest < '2024-08-08 11:12:13'; SELECT count(*) FROM tab WHERE dt_count_min < '2024-08-08 11:12:13'; SELECT count(*) FROM tab WHERE dt_uniq < '2024-08-08 11:12:13'; From 3062f221eb537a49f61bdc4ee058e011ed4d2246 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 8 Aug 2024 18:37:31 +0000 Subject: [PATCH 487/644] Automatic style fix --- ...02164_clickhouse_local_interactive_exception.python | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python index 9527991c36e..5ca7ac4e286 100644 --- a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python +++ b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python @@ -40,6 +40,7 @@ def run_with_timeout(func, args, timeout): print(f"Timeout, state: {STATE_MAP[state.value]}") return + def expect(text, master, debug_log_fd): output_b = os.read(master, 4096) output = output_b.decode() @@ -53,6 +54,7 @@ def expect(text, master, debug_log_fd): return output + def test_completion(program, argv, prompt, state=None): shell_pid, master = pty.fork() if shell_pid == 0: @@ -92,5 +94,11 @@ if __name__ == "__main__": args.append("--disable_suggestion") args.append("--highlight=0") run_with_timeout( - test_completion, [args[0], args, "insert into table function null() format TSV some trash here 123 \n 456"], COMPLETION_TIMEOUT_SECONDS + test_completion, + [ + args[0], + args, + "insert into table function null() format TSV some trash here 123 \n 456", + ], + COMPLETION_TIMEOUT_SECONDS, ) From 343accc92d86748b975f845e1154837d585dcd54 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Aug 2024 20:41:11 +0200 Subject: [PATCH 488/644] Disable randomization of `trace_profile_events` in clickhouse-test --- tests/clickhouse-test | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index ffdd6169777..907d773337a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -836,7 +836,6 @@ class SettingsRandomizer: "cross_join_min_bytes_to_compress": lambda: random.choice([0, 1, 100000000]), "min_external_table_block_size_bytes": lambda: random.choice([0, 1, 100000000]), "max_parsing_threads": lambda: random.choice([0, 1, 10]), - "trace_profile_events": lambda: random.randint(0, 1), "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), } From fe234bd88f3eb1dca8cdb8b217606abfbcea1d54 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Aug 2024 20:56:51 +0200 Subject: [PATCH 489/644] Fix test --- tests/queries/0_stateless/03218_materialize_msan.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03218_materialize_msan.sql b/tests/queries/0_stateless/03218_materialize_msan.sql index b41300ea1e3..7e7043e687b 100644 --- a/tests/queries/0_stateless/03218_materialize_msan.sql +++ b/tests/queries/0_stateless/03218_materialize_msan.sql @@ -1,3 +1,5 @@ +SET enable_analyzer = 1; + SELECT materialize([(NULL, '11\01111111\011111', '1111')]) AS t, (t[1048576]).2, From bc20b637eae8bba72ecfab5256c7eace40586976 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 8 Aug 2024 21:47:23 +0200 Subject: [PATCH 490/644] Add missing file --- .../test_broken_projections/config.d/dont_start_broken.xml | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 tests/integration/test_broken_projections/config.d/dont_start_broken.xml diff --git a/tests/integration/test_broken_projections/config.d/dont_start_broken.xml b/tests/integration/test_broken_projections/config.d/dont_start_broken.xml new file mode 100644 index 00000000000..9603cdc7e3e --- /dev/null +++ b/tests/integration/test_broken_projections/config.d/dont_start_broken.xml @@ -0,0 +1,6 @@ + + + + 0 + + From 6e3df43ae3c13c037fa8e88ae7d102f94bb7398a Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 8 Aug 2024 22:09:58 +0200 Subject: [PATCH 491/644] Cleanup --- ...64_clickhouse_local_interactive_exception.python | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python index 5ca7ac4e286..4c2df9556a1 100644 --- a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python +++ b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python @@ -41,11 +41,7 @@ def run_with_timeout(func, args, timeout): return -def expect(text, master, debug_log_fd): - output_b = os.read(master, 4096) - output = output_b.decode() - debug_log_fd.write(repr(output_b) + "\n") - debug_log_fd.flush() +def expect(text, master, debug_log_fd, output=""): while not text in output: output_b = os.read(master, 4096) output += output_b.decode() @@ -74,12 +70,7 @@ def test_completion(program, argv, prompt, state=None): state.value = 2 output = expect("CANNOT_PARSE_INPUT_ASSERTION_FAILED", master, debug_log_fd) - - while not ":)" in output: - output_b = os.read(master, 4096) - output += output_b.decode() - debug_log_fd.write(repr(output_b) + "\n") - debug_log_fd.flush() + expect(":)", master, debug_log_fd, output) print("OK") state.value = 0 From 6f346c161463330c31d0bce2566df6875cf1d614 Mon Sep 17 00:00:00 2001 From: Jacob Reckhard Date: Thu, 8 Aug 2024 15:20:07 -0600 Subject: [PATCH 492/644] added order for repeatablility of test --- .../03215_multilinestring_geometry.reference | 11 +++++++---- .../0_stateless/03215_multilinestring_geometry.sql | 10 ++++++---- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/03215_multilinestring_geometry.reference b/tests/queries/0_stateless/03215_multilinestring_geometry.reference index f20e21d86f6..9702dd6d6f8 100644 --- a/tests/queries/0_stateless/03215_multilinestring_geometry.reference +++ b/tests/queries/0_stateless/03215_multilinestring_geometry.reference @@ -17,11 +17,14 @@ SELECT x, toTypeName(x), readWKTPolygon(x) as y, toTypeName(y); POLYGON((1 1,2 2,3 3,1 1)) String [[(1,1),(2,2),(3,3),(1,1)]] Polygon -- Non constant tests -CREATE TABLE IF NOT EXISTS t (shape Array(Array(Tuple(Float64, Float64))), wkt_string String) Engine = Memory; -INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); -INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)], [(1, 0), (2, 0), (3, 0)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); +DROP TABLE IF EXISTS t; +CREATE TABLE IF NOT EXISTS t (shape Array(Array(Tuple(Float64, Float64))), wkt_string String, ord Float64) Engine = Memory; +INSERT INTO t (ord, shape, wkt_string) VALUES (1, [[(1, 1), (2, 2), (3, 3), (1, 1)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); +INSERT INTO t (ord, shape, wkt_string) VALUES (2, [[(1, 1), (2, 2), (3, 3), (1, 1)], [(1, 0), (2, 0), (3, 0)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); +INSERT INTO t (ord, shape, wkt_string) VALUES (3, [[(1, 0), (2, 1), (3, 0), (4, 1), (5, 0), (6, 1), (7, 0), (8, 1), (9, 0), (10, 1)]], 'MULTILINESTRING ((1 0, 2 1, 3 0, 4 1, 5 0, 6 1, 7 0, 8 1, 9 0, 10 1))'); -- Native Array(Array(Tuple(Float64, Float64))) is treated as Polygon, not as MultiLineString. -- but reading MultiLineString should still return an Array(Array(Tuple(Float64, Float64))) -select wkt(shape), readWKTMultiLineString(wkt_string), readWKTMultiLineString(wkt_string) = shape from t; +select wkt(shape), readWKTMultiLineString(wkt_string), readWKTMultiLineString(wkt_string) = shape from t order by ord; POLYGON((1 1,2 2,3 3,1 1)) [[(1,1),(2,2),(3,3),(1,1)]] 1 POLYGON((1 1,2 2,3 3,1 1),(1 0,2 0,3 0,1 0)) [[(1,1),(2,2),(3,3),(1,1)],[(1,0),(2,0),(3,0)]] 1 +POLYGON((1 0,2 1,3 0,4 1,5 0,6 1,7 0,8 1,9 0,10 1,1 0)) [[(1,0),(2,1),(3,0),(4,1),(5,0),(6,1),(7,0),(8,1),(9,0),(10,1)]] 1 diff --git a/tests/queries/0_stateless/03215_multilinestring_geometry.sql b/tests/queries/0_stateless/03215_multilinestring_geometry.sql index 6081e081fb3..cf4ef15f63d 100644 --- a/tests/queries/0_stateless/03215_multilinestring_geometry.sql +++ b/tests/queries/0_stateless/03215_multilinestring_geometry.sql @@ -13,12 +13,14 @@ SELECT x, toTypeName(x), readWKTPolygon(x) as y, toTypeName(y); -- Non constant tests -CREATE TABLE IF NOT EXISTS t (shape Array(Array(Tuple(Float64, Float64))), wkt_string String) Engine = Memory; -INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); -INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)], [(1, 0), (2, 0), (3, 0)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); +DROP TABLE IF EXISTS t; +CREATE TABLE IF NOT EXISTS t (shape Array(Array(Tuple(Float64, Float64))), wkt_string String, ord Float64) Engine = Memory; +INSERT INTO t (ord, shape, wkt_string) VALUES (1, [[(1, 1), (2, 2), (3, 3), (1, 1)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); +INSERT INTO t (ord, shape, wkt_string) VALUES (2, [[(1, 1), (2, 2), (3, 3), (1, 1)], [(1, 0), (2, 0), (3, 0)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); +INSERT INTO t (ord, shape, wkt_string) VALUES (3, [[(1, 0), (2, 1), (3, 0), (4, 1), (5, 0), (6, 1), (7, 0), (8, 1), (9, 0), (10, 1)]], 'MULTILINESTRING ((1 0, 2 1, 3 0, 4 1, 5 0, 6 1, 7 0, 8 1, 9 0, 10 1))'); -- Native Array(Array(Tuple(Float64, Float64))) is treated as Polygon, not as MultiLineString. -- but reading MultiLineString should still return an Array(Array(Tuple(Float64, Float64))) -select wkt(shape), readWKTMultiLineString(wkt_string), readWKTMultiLineString(wkt_string) = shape from t; +select wkt(shape), readWKTMultiLineString(wkt_string), readWKTMultiLineString(wkt_string) = shape from t order by ord; From 8d0c8318ea269ea30707669889932c2b4b66d612 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 9 Aug 2024 00:05:26 +0200 Subject: [PATCH 493/644] Apply suggestions from code review --- src/QueryPipeline/QueryPipeline.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 844b9e3b039..c9c0bad7553 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -283,7 +283,7 @@ static void initRowsBeforeAggregation(std::shared_ptr processors, IO if (!processors->empty()) { RowsBeforeStepCounterPtr rows_before_aggregation = std::make_shared(); - for (auto processor : *processors) + for (const auto & processor : *processors) { if (typeid_cast(processor.get()) || typeid_cast(processor.get())) { @@ -545,7 +545,7 @@ void QueryPipeline::complete(std::shared_ptr format) extremes = nullptr; initRowsBeforeLimit(format.get()); - for (const auto context : resources.interpreter_context) + for (const auto & context : resources.interpreter_context) { if (context->getSettingsRef().rows_before_aggregation) { From eb4ea0757730f8b8e59b9230d72aea3ca4bb1ff3 Mon Sep 17 00:00:00 2001 From: justindeguzman Date: Thu, 8 Aug 2024 16:55:39 -0700 Subject: [PATCH 494/644] [Docs] Fix broken links --- docs/en/interfaces/prometheus.md | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/docs/en/interfaces/prometheus.md b/docs/en/interfaces/prometheus.md index 75a68c59219..bf541901b34 100644 --- a/docs/en/interfaces/prometheus.md +++ b/docs/en/interfaces/prometheus.md @@ -25,7 +25,7 @@ ClickHouse can expose its own metrics for scraping from Prometheus: Section `` can be used to make more extended handlers. -This section is similar to [](http.md) but works for prometheus protocols: +This section is similar to [](/en/interfaces/http) but works for prometheus protocols: ```xml @@ -51,11 +51,11 @@ Settings: |---|---|---|---| | `port` | none | Port for serving the exposing metrics protocol. | | `endpoint` | `/metrics` | HTTP endpoint for scraping metrics by prometheus server. Starts with `/`. Should not be used with the `` section. | -| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](http.md) section. | -| `metrics` | true | Expose metrics from the [system.metrics](../operations/system-tables/metrics.md) table. | -| `asynchronous_metrics` | true | Expose current metrics values from the [system.asynchronous_metrics](../operations/system-tables/asynchronous_metrics.md) table. | -| `events` | true | Expose metrics from the [system.events](../operations/system-tables/events.md) table. | -| `errors` | true | Expose the number of errors by error codes occurred since the last server restart. This information could be obtained from the [system.errors](../operations/system-tables/errors.md) as well. | +| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](/en/interfaces/http) section. | +| `metrics` | true | Expose metrics from the [system.metrics](/en/operations/system-tables/metrics) table. | +| `asynchronous_metrics` | true | Expose current metrics values from the [system.asynchronous_metrics](/en/operations/system-tables/asynchronous_metrics) table. | +| `events` | true | Expose metrics from the [system.events](/en/operations/system-tables/events) table. | +| `errors` | true | Expose the number of errors by error codes occurred since the last server restart. This information could be obtained from the [system.errors](/en/operations/system-tables/errors) as well. | Check (replace `127.0.0.1` with the IP addr or hostname of your ClickHouse server): ```bash @@ -65,7 +65,7 @@ curl 127.0.0.1:9363/metrics ## Remote-write protocol {#remote-write} ClickHouse supports the [remote-write](https://prometheus.io/docs/specs/remote_write_spec/) protocol. -Data are received by this protocol and written to a [TimeSeries](../engines/table-engines/integrations/time-series.md) table +Data are received by this protocol and written to a [TimeSeries](/en/engines/table-engines/special/time_series) table (which should be created beforehand). ```xml @@ -89,14 +89,14 @@ Settings: | Name | Default | Description | |---|---|---|---| | `port` | none | Port for serving the `remote-write` protocol. | -| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](http.md) section. | -| `table` | none | The name of a [TimeSeries](../engines/table-engines/integrations/time-series.md) table to write data received by the `remote-write` protocol. This name can optionally contain the name of a database too. | +| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](/en/interfaces/http) section. | +| `table` | none | The name of a [TimeSeries](/en/engines/table-engines/special/time_series) table to write data received by the `remote-write` protocol. This name can optionally contain the name of a database too. | | `database` | none | The name of a database where the table specified in the `table` setting is located if it's not specified in the `table` setting. | ## Remote-read protocol {#remote-read} ClickHouse supports the [remote-read](https://prometheus.io/docs/prometheus/latest/querying/remote_read_api/) protocol. -Data are read from a [TimeSeries](../engines/table-engines/integrations/time-series.md) table and sent via this protocol. +Data are read from a [TimeSeries](/en/engines/table-engines/special/time_series) table and sent via this protocol. ```xml @@ -119,8 +119,8 @@ Settings: | Name | Default | Description | |---|---|---|---| | `port` | none | Port for serving the `remote-read` protocol. | -| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](http.md) section. | -| `table` | none | The name of a [TimeSeries](../engines/table-engines/integrations/time-series.md) table to read data to send by the `remote-read` protocol. This name can optionally contain the name of a database too. | +| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](/en/interfaces/http) section. | +| `table` | none | The name of a [TimeSeries](/en/engines/table-engines/special/time_series) table to read data to send by the `remote-read` protocol. This name can optionally contain the name of a database too. | | `database` | none | The name of a database where the table specified in the `table` setting is located if it's not specified in the `table` setting. | ## Configuration for multiple protocols {#multiple-protocols} From d7442e0670b1900e73299341d44287d21eafd0ad Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 8 Aug 2024 20:45:43 -0400 Subject: [PATCH 495/644] Fix flacky 02572_query_views_log_background_thread --- .../02572_query_views_log_background_thread.sh | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.sh b/tests/queries/0_stateless/02572_query_views_log_background_thread.sh index a3e428e75c8..509cd03f6c2 100755 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.sh +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.sh @@ -13,13 +13,16 @@ ${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "drop table if exists b ${CLICKHOUSE_CLIENT} --query="create table copy_02572 (key Int) engine=Memory();" ${CLICKHOUSE_CLIENT} --query="create table data_02572 (key Int) engine=Memory();" -${CLICKHOUSE_CLIENT} --query="create table buffer_02572 (key Int) engine=Buffer(currentDatabase(), data_02572, 1, 3, 3, 1, 1e9, 1, 1e9);" +${CLICKHOUSE_CLIENT} --query="create table buffer_02572 (key Int) engine=Buffer(currentDatabase(), data_02572, 1, 8, 8, 1, 1e9, 1, 1e9);" ${CLICKHOUSE_CLIENT} --query="create materialized view mv_02572 to copy_02572 as select * from data_02572;" +start=$(date +%s) ${CLICKHOUSE_CLIENT} --query="insert into buffer_02572 values (1);" -# ensure that the flush was not direct -${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "select * from data_02572; select * from copy_02572;" +if [ $(( $(date +%s) - start )) -gt 6 ]; then # clickhouse test cluster is overloaded, will skip + # ensure that the flush was not direct + ${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "select * from data_02572; select * from copy_02572;" +fi # we cannot use OPTIMIZE, this will attach query context, so let's wait for _ in {1..100}; do From 248da0341aca537104486243b84ca230e9c4f9fb Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 8 Aug 2024 15:54:56 +0800 Subject: [PATCH 496/644] fixed --- .../HashJoin/HashJoinMethodsImpl.h | 53 ++++++++++++++----- src/Interpreters/joinDispatch.h | 8 +-- ..._join_on_inequal_expression_fast.reference | 1 - 3 files changed, 44 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 5fefe53d145..aedd24630d1 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -1,11 +1,12 @@ #pragma once #include + namespace DB { namespace ErrorCodes { - extern const int UNSUPPORTED_JOIN_KEYS; - extern const int LOGICAL_ERROR; +extern const int UNSUPPORTED_JOIN_KEYS; +extern const int LOGICAL_ERROR; } template size_t HashJoinMethods::insertFromBlockImpl( @@ -156,7 +157,6 @@ Block HashJoinMethods::joinBlockImpl( block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate); } } - return remaining_block; } @@ -596,7 +596,7 @@ ColumnPtr HashJoinMethods::buildAdditionalFilter template template -size_t HashJoinMethods::joinRightColumnsWithAddtitionalFilter( +size_t HashJoinMethods::joinRightColumnsWithAddtitionalFilter( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, @@ -662,6 +662,8 @@ size_t HashJoinMethods::joinRightColumnsWithAddti { auto & mapped = find_result.getMapped(); find_results.push_back(find_result); + /// We don't add missing in addFoundRowAll here. we will add it after filter is applied. + /// it's different from `joinRightColumns`. if (flag_per_row) addFoundRowAll(mapped, selected_rows, current_added_rows, all_flag_known_rows, nullptr); else @@ -682,32 +684,54 @@ size_t HashJoinMethods::joinRightColumnsWithAddti for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i) { bool any_matched = false; - /// For right join, flag_per_row is true, we need mark used flags for each row. + /// right/full join or multiple disjuncts, we need to mark used flags for each row. if (flag_per_row) { for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) { if (filter_flags[replicated_row]) { - any_matched = true; if constexpr (join_features.is_semi_join || join_features.is_any_join) { - auto used_once = used_flags.template setUsedOnce( - selected_right_row_it->block, selected_right_row_it->row_num, 0); - if (used_once) + /// For LEFT/INNER SEMI/ANY JOIN, we need to add only first appeared row from left, + if constexpr (join_features.left || join_features.inner) { - total_added_rows += 1; - added_columns.appendFromBlock( - *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + if (!any_matched) + { + // For inner join, we need mark each right row'flag, because we only use each right row once. + auto used_once = used_flags.template setUsedOnce( + selected_right_row_it->block, selected_right_row_it->row_num, 0); + if (used_once) + { + any_matched = true; + total_added_rows += 1; + added_columns.appendFromBlock( + *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + } + } + } + else + { + auto used_once = used_flags.template setUsedOnce( + selected_right_row_it->block, selected_right_row_it->row_num, 0); + if (used_once) + { + any_matched = true; + total_added_rows += 1; + added_columns.appendFromBlock( + *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + } } } else if constexpr (join_features.is_anti_join) { + any_matched = true; if constexpr (join_features.right && join_features.need_flags) used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); } else { + any_matched = true; total_added_rows += 1; added_columns.appendFromBlock( *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); @@ -715,6 +739,7 @@ size_t HashJoinMethods::joinRightColumnsWithAddti selected_right_row_it->block, selected_right_row_it->row_num, 0); } } + ++selected_right_row_it; } } @@ -892,7 +917,8 @@ void HashJoinMethods::correctNullabilityInplace( } template -void HashJoinMethods::correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map) +void HashJoinMethods::correctNullabilityInplace( + ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map) { if (nullable) { @@ -908,4 +934,3 @@ void HashJoinMethods::correctNullabilityInplace( JoinCommon::removeColumnNullability(column); } } - diff --git a/src/Interpreters/joinDispatch.h b/src/Interpreters/joinDispatch.h index 5d4bd8f92e5..4aabc49c29b 100644 --- a/src/Interpreters/joinDispatch.h +++ b/src/Interpreters/joinDispatch.h @@ -16,11 +16,13 @@ namespace DB /// When only need to match only one row from right table, use HashJoin::MapsOne. For example, LEFT ANY/SEMI/ANTI. /// /// HashJoin::MapsAll will store all rows for each key in the map. It is used when need to match multiple rows from right table. -/// For example, RIGHT ANY/ALL, FULL JOIN, INNER JOIN. +/// For example, LEFT ALL, INNER ALL, RIGHT ALL/ANY. /// /// prefer_use_maps_all is true when there is mixed inequal condition in the join condition. For example, `t1.a = t2.a AND t1.b > t2.b`. /// In this case, we need to use HashJoin::MapsAll to store all rows for each key in the map. We will select all matched rows from the map /// and filter them by `t1.b > t2.b`. +/// +/// flagged indicates whether we need to store flags for each row whether it has been used in the join. See JoinUsedFlags.h. template struct MapGetter; @@ -30,7 +32,7 @@ template struct MapGetter struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; @@ -43,7 +45,7 @@ template struct MapGetter struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference index aa8d4103db2..a70e70ef7e9 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference @@ -742,4 +742,3 @@ key1 b 2 3 2 key1 c 3 2 1 key1 d 4 7 2 key2 a2 1 1 1 -key4 f 2 3 4 From 54cd980e13d70e84e2c0aad51420a2dd31217131 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 09:44:30 +0200 Subject: [PATCH 497/644] Increase queue size and fix tests --- docker/test/stateless/run.sh | 11 +++-------- .../0_stateless/02726_async_insert_flush_queue.sql | 4 +++- .../0_stateless/02726_async_insert_flush_stress.sh | 2 +- 3 files changed, 7 insertions(+), 10 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index c81f33ace01..cd2a61cce4e 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -193,8 +193,8 @@ ORDER BY tuple()" # create minio log webhooks for both audit and server logs # use async inserts to avoid creating too many parts -./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" -./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" +./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" queue_size=300000 +./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" queue_size=300000 max_retries=100 retry=1 @@ -376,12 +376,7 @@ done # collect minio audit and server logs -has_async_inserts=$(clickhouse-client -q "SELECT count() FROM system.asynchronous_inserts WHERE table = 'minio_audit_logs' OR table = 'minio_server_logs'") -if [[ has_async_inserts -eq 1 ]]; then - echo "Waiting for async inserts to flush" - sleep 5 -fi - +clickhouse-client -q "SYSTEM FLUSH ASYNC INSERT QUEUE" clickhouse-client -q "SELECT log FROM minio_audit_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_audit_logs.jsonl.zst' FORMAT JSONEachRow" clickhouse-client -q "SELECT log FROM minio_server_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_server_logs.jsonl.zst' FORMAT JSONEachRow" diff --git a/tests/queries/0_stateless/02726_async_insert_flush_queue.sql b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql index 97d644fa4d6..5d941adcb81 100644 --- a/tests/queries/0_stateless/02726_async_insert_flush_queue.sql +++ b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql @@ -30,7 +30,9 @@ SELECT count() FROM t_async_inserts_flush; SYSTEM FLUSH ASYNC INSERT QUEUE; -SELECT count() FROM system.asynchronous_inserts; +SELECT count() FROM system.asynchronous_inserts +WHERE database = currentDatabase() AND table = 't_async_inserts_flush'; + SELECT count() FROM t_async_inserts_flush; DROP TABLE t_async_inserts_flush; diff --git a/tests/queries/0_stateless/02726_async_insert_flush_stress.sh b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh index 876766d0780..61bbbd620f0 100755 --- a/tests/queries/0_stateless/02726_async_insert_flush_stress.sh +++ b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh @@ -91,5 +91,5 @@ flush1 $TIMEOUT & wait ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH ASYNC INSERT QUEUE" -${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.asynchronous_inserts" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.asynchronous_inserts WHERE database = currentDatabase() AND table = 'async_inserts'" ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts"; From 47f429a52435d84b3e1cee06202e743c11bcba0d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 20:54:29 +0000 Subject: [PATCH 498/644] Proper CMake for libfiu --- CMakeLists.txt | 2 +- contrib/libfiu-cmake/CMakeLists.txt | 27 ++++++++++++++------------- src/CMakeLists.txt | 4 ++-- src/Common/FailPoint.cpp | 6 ++++-- src/Common/FailPoint.h | 9 ++++----- src/Common/config.h.in | 2 +- src/configure_config.cmake | 4 ++-- 7 files changed, 28 insertions(+), 26 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 7b4e0484ab1..134f3afd727 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -402,7 +402,7 @@ if ((NOT OS_LINUX AND NOT OS_ANDROID) OR (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") set(ENABLE_GWP_ASAN OFF) endif () -option (ENABLE_FIU "Enable Fiu" ON) +option (ENABLE_LIBFIU "Enable libfiu" ON) option(WERROR "Enable -Werror compiler option" ON) diff --git a/contrib/libfiu-cmake/CMakeLists.txt b/contrib/libfiu-cmake/CMakeLists.txt index e805491edbb..eab55087c98 100644 --- a/contrib/libfiu-cmake/CMakeLists.txt +++ b/contrib/libfiu-cmake/CMakeLists.txt @@ -1,20 +1,21 @@ -if (NOT ENABLE_FIU) - message (STATUS "Not using fiu") +if (NOT ENABLE_LIBFIU) + message (STATUS "Not using libfiu") return () endif () -set(FIU_DIR "${ClickHouse_SOURCE_DIR}/contrib/libfiu/") +set(LIBFIU_DIR "${ClickHouse_SOURCE_DIR}/contrib/libfiu/") -set(FIU_SOURCES - ${FIU_DIR}/libfiu/fiu.c - ${FIU_DIR}/libfiu/fiu-rc.c - ${FIU_DIR}/libfiu/backtrace.c - ${FIU_DIR}/libfiu/wtable.c +set(LIBFIU_SOURCES + ${LIBFIU_DIR}/libfiu/fiu.c + ${LIBFIU_DIR}/libfiu/fiu-rc.c + ${LIBFIU_DIR}/libfiu/backtrace.c + ${LIBFIU_DIR}/libfiu/wtable.c ) -set(FIU_HEADERS "${FIU_DIR}/libfiu") +set(LIBFIU_HEADERS "${LIBFIU_DIR}/libfiu") -add_library(_fiu ${FIU_SOURCES}) -target_compile_definitions(_fiu PUBLIC DUMMY_BACKTRACE) -target_include_directories(_fiu PUBLIC ${FIU_HEADERS}) -add_library(ch_contrib::fiu ALIAS _fiu) +add_library(_libfiu ${LIBFIU_SOURCES}) +target_compile_definitions(_libfiu PUBLIC DUMMY_BACKTRACE) +target_compile_definitions(_libfiu PUBLIC FIU_ENABLE) +target_include_directories(_libfiu PUBLIC ${LIBFIU_HEADERS}) +add_library(ch_contrib::libfiu ALIAS _libfiu) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 98dd0601a1b..db3ef0f489f 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -353,8 +353,8 @@ target_link_libraries(clickhouse_common_io Poco::Foundation ) -if (TARGET ch_contrib::fiu) - target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::fiu) +if (TARGET ch_contrib::libfiu) + target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::libfiu) endif() if (TARGET ch_contrib::cpuid) diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index 0b1ec552d43..b2fcbc77c56 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -7,6 +7,8 @@ #include #include +#include "config.h" + namespace DB { @@ -15,7 +17,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; }; -#if FIU_ENABLE +#if USE_LIBFIU static struct InitFiu { InitFiu() @@ -135,7 +137,7 @@ void FailPointInjection::pauseFailPoint(const String & fail_point_name) void FailPointInjection::enableFailPoint(const String & fail_point_name) { -#if FIU_ENABLE +#if USE_LIBFIU #define SUB_M(NAME, flags, pause) \ if (fail_point_name == FailPoints::NAME) \ { \ diff --git a/src/Common/FailPoint.h b/src/Common/FailPoint.h index b3e1214d597..1af13d08553 100644 --- a/src/Common/FailPoint.h +++ b/src/Common/FailPoint.h @@ -1,17 +1,16 @@ #pragma once -#include "config.h" #include #include #include +#include "config.h" + #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wdocumentation" #pragma clang diagnostic ignored "-Wreserved-macro-identifier" - -#include -#include - +# include +# include #pragma clang diagnostic pop #include diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 6a0090130a3..56a067b06e8 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -59,7 +59,7 @@ #cmakedefine01 USE_SKIM #cmakedefine01 USE_PRQL #cmakedefine01 USE_ULID -#cmakedefine01 FIU_ENABLE +#cmakedefine01 USE_LIBFIU #cmakedefine01 USE_BCRYPT #cmakedefine01 USE_LIBARCHIVE #cmakedefine01 USE_POCKETFFT diff --git a/src/configure_config.cmake b/src/configure_config.cmake index d22bf674df4..721041bc11b 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -161,8 +161,8 @@ endif() if (TARGET ch_contrib::ssh) set(USE_SSH 1) endif() -if (TARGET ch_contrib::fiu) - set(FIU_ENABLE 1) +if (TARGET ch_contrib::libfiu) + set(USE_LIBFIU 1) endif() if (TARGET ch_contrib::libarchive) set(USE_LIBARCHIVE 1) From 30d8e407723f45ed79dc960604d77e8ee02f3edb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 21:02:04 +0000 Subject: [PATCH 499/644] Fix referenced variable for vectorscan in system.build_options --- src/Storages/System/StorageSystemBuildOptions.cpp.in | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp.in b/src/Storages/System/StorageSystemBuildOptions.cpp.in index a81bcb08bfc..3f84a7468fd 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.cpp.in @@ -36,7 +36,7 @@ const char * auto_config_build[] "USE_SSL", "@USE_SSL@", "OPENSSL_VERSION", "@OPENSSL_VERSION@", "OPENSSL_IS_BORING_SSL", "@OPENSSL_IS_BORING_SSL@", - "USE_VECTORSCAN", "@ENABLE_VECTORSCAN@", + "USE_VECTORSCAN", "@USE_VECTORSCAN@", "USE_SIMDJSON", "@USE_SIMDJSON@", "USE_ODBC", "@USE_ODBC@", "USE_GRPC", "@USE_GRPC@", From b242a129f811c9838167d98df56d060ceac24b85 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 21:03:42 +0000 Subject: [PATCH 500/644] Fix referenced variable for jemalloc in system.build_options --- src/Storages/System/StorageSystemBuildOptions.cpp.in | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp.in b/src/Storages/System/StorageSystemBuildOptions.cpp.in index 3f84a7468fd..6cecef5a7ad 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.cpp.in @@ -21,7 +21,7 @@ const char * auto_config_build[] "BUILD_COMPILE_DEFINITIONS", "@BUILD_COMPILE_DEFINITIONS@", "USE_EMBEDDED_COMPILER", "@USE_EMBEDDED_COMPILER@", "USE_GLIBC_COMPATIBILITY", "@GLIBC_COMPATIBILITY@", - "USE_JEMALLOC", "@ENABLE_JEMALLOC@", + "USE_JEMALLOC", "@USE_JEMALLOC@", "USE_ICU", "@USE_ICU@", "USE_H3", "@USE_H3@", "USE_MYSQL", "@USE_MYSQL@", From 1e2eea9f6333b165b1b15acef5f489ad067a57f3 Mon Sep 17 00:00:00 2001 From: kruglov Date: Fri, 9 Aug 2024 10:16:15 +0300 Subject: [PATCH 501/644] Fixed errors when publication name contents symbols except [a-z_] --- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index f632e553a0d..01f78673ed8 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -659,7 +659,7 @@ void PostgreSQLReplicationHandler::dropReplicationSlot(pqxx::nontransaction & tx void PostgreSQLReplicationHandler::dropPublication(pqxx::nontransaction & tx) { - std::string query_str = fmt::format("DROP PUBLICATION IF EXISTS {}", publication_name); + std::string query_str = fmt::format("DROP PUBLICATION IF EXISTS {}", doubleQuoteString(publication_name)); tx.exec(query_str); LOG_DEBUG(log, "Dropped publication: {}", publication_name); } @@ -667,7 +667,7 @@ void PostgreSQLReplicationHandler::dropPublication(pqxx::nontransaction & tx) void PostgreSQLReplicationHandler::addTableToPublication(pqxx::nontransaction & ntx, const String & table_name) { - std::string query_str = fmt::format("ALTER PUBLICATION {} ADD TABLE ONLY {}", publication_name, doubleQuoteWithSchema(table_name)); + std::string query_str = fmt::format("ALTER PUBLICATION {} ADD TABLE ONLY {}", doubleQuoteString(publication_name), doubleQuoteWithSchema(table_name)); ntx.exec(query_str); LOG_TRACE(log, "Added table {} to publication `{}`", doubleQuoteWithSchema(table_name), publication_name); } From a497a2391455de21dec19b365cc939defdc56b1e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Aug 2024 08:33:11 +0000 Subject: [PATCH 502/644] Fix CMake for QPL --- contrib/qpl-cmake/CMakeLists.txt | 4 ---- src/CMakeLists.txt | 2 ++ src/Common/config.h.in | 1 + .../CompressionCodecDeflateQpl.cpp | 8 +++---- src/Compression/CompressionCodecDeflateQpl.h | 6 +++++ src/Compression/CompressionFactory.cpp | 22 +++++++++---------- .../System/StorageSystemBuildOptions.cpp.in | 2 +- src/configure_config.cmake | 3 +++ 8 files changed, 28 insertions(+), 20 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index e62612cff5a..89332ae0f7a 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -728,10 +728,6 @@ add_library(_qpl STATIC ${LIB_DEPS}) target_include_directories(_qpl PUBLIC $ $) - -target_compile_definitions(_qpl - PUBLIC -DENABLE_QPL_COMPRESSION) - target_link_libraries(_qpl PRIVATE ch_contrib::accel-config) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index db3ef0f489f..eba04d93df5 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -556,6 +556,8 @@ target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::lz4) if (TARGET ch_contrib::qpl) dbms_target_link_libraries(PUBLIC ch_contrib::qpl) + target_link_libraries (clickhouse_compression PUBLIC ch_contrib::qpl) + target_link_libraries (clickhouse_compression PUBLIC ch_contrib::accel-config) endif () if (TARGET ch_contrib::accel-config) diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 56a067b06e8..1680cde22a2 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -32,6 +32,7 @@ #cmakedefine01 USE_IDNA #cmakedefine01 USE_NLP #cmakedefine01 USE_VECTORSCAN +#cmakedefine01 USE_QPL #cmakedefine01 USE_LIBURING #cmakedefine01 USE_AVRO #cmakedefine01 USE_CAPNP diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index f1b5b24e866..c82ee861a6f 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -1,7 +1,3 @@ -#ifdef ENABLE_QPL_COMPRESSION - -#include -#include #include #include #include @@ -11,6 +7,10 @@ #include #include #include +#include +#include + +#if USE_QPL #include "libaccel_config.h" diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index 86fd9051bd8..d9abc0fb7e0 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -4,6 +4,11 @@ #include #include #include + +#include "config.h" + +#if USE_QPL + #include namespace Poco @@ -117,3 +122,4 @@ private: }; } +#endif diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 2e7aa0d086f..fbb5664d441 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -1,20 +1,20 @@ -#include "config.h" - #include +#include +#include +#include +#include #include #include #include -#include -#include -#include -#include #include -#include -#include -#include +#include +#include +#include #include +#include "config.h" + namespace DB { @@ -179,7 +179,7 @@ void registerCodecZSTD(CompressionCodecFactory & factory); void registerCodecZSTDQAT(CompressionCodecFactory & factory); #endif void registerCodecMultiple(CompressionCodecFactory & factory); -#ifdef ENABLE_QPL_COMPRESSION +#if USE_QPL void registerCodecDeflateQpl(CompressionCodecFactory & factory); #endif @@ -209,7 +209,7 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecGorilla(*this); registerCodecEncrypted(*this); registerCodecFPC(*this); -#ifdef ENABLE_QPL_COMPRESSION +#if USE_QPL registerCodecDeflateQpl(*this); #endif registerCodecGCD(*this); diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp.in b/src/Storages/System/StorageSystemBuildOptions.cpp.in index 6cecef5a7ad..f7edfb17542 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.cpp.in @@ -62,7 +62,7 @@ const char * auto_config_build[] "USE_ARROW", "@USE_ARROW@", "USE_ORC", "@USE_ORC@", "USE_MSGPACK", "@USE_MSGPACK@", - "USE_QPL", "@ENABLE_QPL@", + "USE_QPL", "@USE_QPL@", "USE_QAT", "@ENABLE_QATLIB@", "GIT_HASH", "@GIT_HASH@", "GIT_BRANCH", R"IRjaNsZIL9Yh7FQ4(@GIT_BRANCH@)IRjaNsZIL9Yh7FQ4", diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 721041bc11b..6782cc6d824 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -135,6 +135,9 @@ endif() if (TARGET ch_contrib::vectorscan) set(USE_VECTORSCAN 1) endif() +if (TARGET ch_contrib::qpl) + set(USE_QPL 1) +endif() if (TARGET ch_contrib::avrocpp) set(USE_AVRO 1) endif() From eec5fe087c273aba443d97824da0e7aadfd52cdd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Aug 2024 08:50:52 +0000 Subject: [PATCH 503/644] Fix CMake for QATlib --- contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt | 4 ++-- src/CMakeLists.txt | 7 ++----- src/Common/config.h.in | 1 + src/Compression/CompressionCodecZSTDQAT.cpp | 5 ++++- src/Compression/CompressionFactory.cpp | 4 ++-- src/Storages/System/StorageSystemBuildOptions.cpp.in | 2 +- src/configure_config.cmake | 3 +++ 7 files changed, 15 insertions(+), 11 deletions(-) diff --git a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt index 72d21a8572b..fc18092f574 100644 --- a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt +++ b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt @@ -27,7 +27,7 @@ if (ENABLE_QAT_OUT_OF_TREE_BUILD) ${QAT_AL_INCLUDE_DIR} ${QAT_USDM_INCLUDE_DIR} ${ZSTD_LIBRARY_DIR}) - target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_ZSTD_QAT_CODEC) + target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0) add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) else () # In-tree build message(STATUS "Intel QATZSTD in-tree build") @@ -78,7 +78,7 @@ else () # In-tree build ${QAT_USDM_INCLUDE_DIR} ${ZSTD_LIBRARY_DIR} ${LIBQAT_HEADER_DIR}) - target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_ZSTD_QAT_CODEC -DINTREE) + target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DINTREE) target_include_directories(_qatzstd_plugin SYSTEM PUBLIC $ $) add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) endif () diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index eba04d93df5..43092d10dd2 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -560,12 +560,9 @@ if (TARGET ch_contrib::qpl) target_link_libraries (clickhouse_compression PUBLIC ch_contrib::accel-config) endif () -if (TARGET ch_contrib::accel-config) - dbms_target_link_libraries(PUBLIC ch_contrib::accel-config) -endif () - -if (TARGET ch_contrib::qatzstd_plugin) +if (TARGET ch_contrib::accel-config AND TARGET ch_contrib::qatzstd_plugin) dbms_target_link_libraries(PUBLIC ch_contrib::qatzstd_plugin) + dbms_target_link_libraries(PUBLIC ch_contrib::accel-config) target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::qatzstd_plugin) endif () diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 1680cde22a2..e3f8882850f 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -33,6 +33,7 @@ #cmakedefine01 USE_NLP #cmakedefine01 USE_VECTORSCAN #cmakedefine01 USE_QPL +#cmakedefine01 USE_QATLIB #cmakedefine01 USE_LIBURING #cmakedefine01 USE_AVRO #cmakedefine01 USE_CAPNP diff --git a/src/Compression/CompressionCodecZSTDQAT.cpp b/src/Compression/CompressionCodecZSTDQAT.cpp index 5a4ef70a30a..e19b7e4a001 100644 --- a/src/Compression/CompressionCodecZSTDQAT.cpp +++ b/src/Compression/CompressionCodecZSTDQAT.cpp @@ -1,4 +1,6 @@ -#ifdef ENABLE_ZSTD_QAT_CODEC +#include "config.h" + +#if USE_QATLIB #include #include @@ -6,6 +8,7 @@ #include #include #include + #include #include diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index fbb5664d441..ac00f571568 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -175,7 +175,7 @@ void registerCodecNone(CompressionCodecFactory & factory); void registerCodecLZ4(CompressionCodecFactory & factory); void registerCodecLZ4HC(CompressionCodecFactory & factory); void registerCodecZSTD(CompressionCodecFactory & factory); -#ifdef ENABLE_ZSTD_QAT_CODEC +#if USE_QATLIB void registerCodecZSTDQAT(CompressionCodecFactory & factory); #endif void registerCodecMultiple(CompressionCodecFactory & factory); @@ -198,7 +198,7 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecNone(*this); registerCodecLZ4(*this); registerCodecZSTD(*this); -#ifdef ENABLE_ZSTD_QAT_CODEC +#if USE_QATLIB registerCodecZSTDQAT(*this); #endif registerCodecLZ4HC(*this); diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp.in b/src/Storages/System/StorageSystemBuildOptions.cpp.in index f7edfb17542..9e5adbfe825 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.cpp.in @@ -63,7 +63,7 @@ const char * auto_config_build[] "USE_ORC", "@USE_ORC@", "USE_MSGPACK", "@USE_MSGPACK@", "USE_QPL", "@USE_QPL@", - "USE_QAT", "@ENABLE_QATLIB@", + "USE_QATLIB", "@USE_QATLIB@", "GIT_HASH", "@GIT_HASH@", "GIT_BRANCH", R"IRjaNsZIL9Yh7FQ4(@GIT_BRANCH@)IRjaNsZIL9Yh7FQ4", "GIT_DATE", "@GIT_DATE@", diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 6782cc6d824..5b24f79ef6e 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -138,6 +138,9 @@ endif() if (TARGET ch_contrib::qpl) set(USE_QPL 1) endif() +if (TARGET ch_contrib::qatlib) + set(USE_QATLIB 1) +endif() if (TARGET ch_contrib::avrocpp) set(USE_AVRO 1) endif() From 759299910c2892b809735caaf663fe374c315c5c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 9 Aug 2024 09:14:00 +0000 Subject: [PATCH 504/644] Force new analyzer for test --- tests/queries/0_stateless/03217_filtering_in_storage_merge.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03217_filtering_in_storage_merge.sql b/tests/queries/0_stateless/03217_filtering_in_storage_merge.sql index 5ecc1e7c672..42d31e95f9c 100644 --- a/tests/queries/0_stateless/03217_filtering_in_storage_merge.sql +++ b/tests/queries/0_stateless/03217_filtering_in_storage_merge.sql @@ -13,4 +13,4 @@ INSERT INTO test_03217_merge_replica_1 SELECT number AS x FROM numbers(10); SYSTEM SYNC REPLICA test_03217_merge_replica_2; -- If the filter on _table is not applied, then the plan will show both replicas -EXPLAIN SELECT _table, count() FROM test_03217_all_replicas WHERE _table = 'test_03217_merge_replica_1' AND x >= 0 GROUP BY _table; +EXPLAIN SELECT _table, count() FROM test_03217_all_replicas WHERE _table = 'test_03217_merge_replica_1' AND x >= 0 GROUP BY _table SETTINGS allow_experimental_analyzer=1; From dc64550536ff249b1c12070ed646bc4321bc68bf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 9 Aug 2024 09:14:13 +0000 Subject: [PATCH 505/644] Remove wrong check from test --- .../0_stateless/03217_filtering_in_system_tables.reference | 4 ++-- .../0_stateless/03217_filtering_in_system_tables.sql | 7 +++---- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03217_filtering_in_system_tables.reference b/tests/queries/0_stateless/03217_filtering_in_system_tables.reference index 218fddf92e0..d7ccd989f53 100644 --- a/tests/queries/0_stateless/03217_filtering_in_system_tables.reference +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.reference @@ -1,4 +1,4 @@ information_schema tables default test_03217_system_tables_replica_1 r1 -1 1 -1 1 +1 +1 diff --git a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql index 72ca7c8684d..0db846bc500 100644 --- a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql @@ -10,18 +10,17 @@ CREATE TABLE test_03217_system_tables_replica_2(x UInt32) -- If filtering is not done correctly on database-table column, then this query report to read 2 rows, which are the above tables SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = 'test_03217_system_tables_replica_1' AND replica_name = 'r1'; - SYSTEM FLUSH LOGS; --- argMin-argMax is necessary to make the test repeatable +-- argMax is necessary to make the test repeatable -- StorageSystemTables -SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 +SELECT argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 AND current_database = currentDatabase() AND query LIKE '%SELECT database, table FROM system.tables WHERE database = \'information_schema\' AND table = \'tables\';' AND type = 'QueryFinish'; -- StorageSystemReplicas -SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 +SELECT argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 AND current_database = currentDatabase() AND query LIKE '%SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = \'test_03217_system_tables_replica_1\' AND replica_name = \'r1\';' AND type = 'QueryFinish'; From 6360687b307ea2ea7c5cf5746d83655b72a73a75 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 11:28:18 +0200 Subject: [PATCH 506/644] Try fix flaky 02675_profile_events_from_query_log_and_client --- ...events_from_query_log_and_client.reference | 14 ++++++------ ...rofile_events_from_query_log_and_client.sh | 22 +++++++++++++++---- 2 files changed, 25 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference index babcecf7004..9dbac8d34f2 100644 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference @@ -1,11 +1,11 @@ INSERT TO S3 - [ 0 ] S3Clients: 1 - [ 0 ] S3CompleteMultipartUpload: 1 - [ 0 ] S3CreateMultipartUpload: 1 - [ 0 ] S3HeadObject: 2 - [ 0 ] S3ReadRequestsCount: 2 - [ 0 ] S3UploadPart: 1 - [ 0 ] S3WriteRequestsCount: 3 +Successful write requests 3 +S3Clients 1 +S3CompleteMultipartUpload 1 +S3CreateMultipartUpload 1 +S3HeadObject 2 +S3ReadRequestsCount 2 +S3UploadPart 1 CHECK WITH query_log QueryFinish S3CreateMultipartUpload 1 S3UploadPart 1 S3CompleteMultipartUpload 1 S3PutObject 0 CREATE diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh index e346d9893a7..cae20be79dc 100755 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh @@ -9,7 +9,21 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo "INSERT TO S3" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " INSERT INTO TABLE FUNCTION s3('http://localhost:11111/test/profile_events.csv', 'test', 'testtest', 'CSV', 'number UInt64') SELECT number FROM numbers(1000000) SETTINGS s3_max_single_part_upload_size = 10, s3_truncate_on_insert = 1; -" 2>&1 | grep -o -e '\ \[\ .*\ \]\ S3.*:\ .*\ ' | grep -v 'Microseconds' | grep -v 'S3DiskConnections' | grep -v 'S3DiskAddresses' | sort +" 2>&1 | $CLICKHOUSE_LOCAL -q " +WITH '(\\w+): (\\d+)' AS pattern, + (SELECT (groupArray(regexpExtract(line, pattern, 1)), + groupArray(regexpExtract(line, pattern, 2)::UInt64))::Map(String, UInt64) + FROM file(stdin, 'LineAsString', 'line String') + WHERE line LIKE '% S3%' + AND line NOT LIKE '%Microseconds%' + AND line NOT LIKE '%S3DiskConnections%' + AND line NOT LIKE '%S3DiskAddresses') AS pe_map +SELECT untuple(arrayJoin(pe_map) AS pe) +WHERE tupleElement(pe, 1) not like '%WriteRequests%' +UNION ALL +SELECT 'Successful write requests', + (pe_map['S3WriteRequestsCount'] - pe_map['S3WriteRequestsErrors'])::UInt64 +" echo "CHECK WITH query_log" $CLICKHOUSE_CLIENT -nq " @@ -40,19 +54,19 @@ CREATE TABLE times (t DateTime) ENGINE MergeTree ORDER BY t echo "INSERT" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " INSERT INTO times SELECT now() + INTERVAL 1 day SETTINGS optimize_on_insert = 0; -" 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' +" 2>&1 | grep -o -e ' \[ .* \] FileOpen: .* ' echo "READ" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " SELECT '1', min(t) FROM times SETTINGS optimize_use_implicit_projections = 1; -" 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' +" 2>&1 | grep -o -e ' \[ .* \] FileOpen: .* ' echo "INSERT and READ INSERT" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " INSERT INTO times SELECT now() + INTERVAL 2 day SETTINGS optimize_on_insert = 0; SELECT '2', min(t) FROM times SETTINGS optimize_use_implicit_projections = 1; INSERT INTO times SELECT now() + INTERVAL 3 day SETTINGS optimize_on_insert = 0; -" 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' +" 2>&1 | grep -o -e ' \[ .* \] FileOpen: .* ' echo "DROP" $CLICKHOUSE_CLIENT -nq " From e4903858c8ae108b87b726a8056acab10dd6b851 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 9 Aug 2024 09:31:33 +0000 Subject: [PATCH 507/644] Add extra check to make sure both replicas are present in system.replicas --- .../0_stateless/03217_filtering_in_system_tables.reference | 2 ++ tests/queries/0_stateless/03217_filtering_in_system_tables.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/03217_filtering_in_system_tables.reference b/tests/queries/0_stateless/03217_filtering_in_system_tables.reference index d7ccd989f53..c0761c3f689 100644 --- a/tests/queries/0_stateless/03217_filtering_in_system_tables.reference +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.reference @@ -1,4 +1,6 @@ information_schema tables +both default test_03217_system_tables_replica_1 r1 +both default test_03217_system_tables_replica_2 r2 default test_03217_system_tables_replica_1 r1 1 1 diff --git a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql index 0db846bc500..2ce63559b99 100644 --- a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql @@ -8,6 +8,8 @@ CREATE TABLE test_03217_system_tables_replica_2(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217_system_tables_replica', 'r2') ORDER BY x; +-- Make sure we can read both replicas +SELECT 'both', database, table, replica_name FROM system.replicas WHERE database = currentDatabase(); -- If filtering is not done correctly on database-table column, then this query report to read 2 rows, which are the above tables SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = 'test_03217_system_tables_replica_1' AND replica_name = 'r1'; SYSTEM FLUSH LOGS; From da907d535623422b7beb1d8cf3e7389698567e28 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 10:37:48 +0100 Subject: [PATCH 508/644] Better parsing --- docker/test/stateless/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index cd2a61cce4e..c59d36114ae 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -178,7 +178,7 @@ attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 202 clickhouse-client --query "CREATE TABLE minio_audit_logs ( log String, - event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(substring(JSONExtractRaw(log, 'time'), 2, 29), 9, 'UTC') + event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(trim(BOTH '"' FROM JSONExtractRaw(log, 'time')), 9, 'UTC') ) ENGINE = MergeTree ORDER BY tuple()" @@ -186,7 +186,7 @@ ORDER BY tuple()" clickhouse-client --query "CREATE TABLE minio_server_logs ( log String, - event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(substring(JSONExtractRaw(log, 'time'), 2, 29), 9, 'UTC') + event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(trim(BOTH '"' FROM JSONExtractRaw(log, 'time')), 9, 'UTC') ) ENGINE = MergeTree ORDER BY tuple()" From a5e06c7c311f142aa8d790cf398af19d86cbce4f Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 9 Aug 2024 11:06:11 +0200 Subject: [PATCH 509/644] Fix UB in hopEnd and hopStart This was causing segfaults because of a NULL pointer dereference --- src/Functions/FunctionsTimeWindow.cpp | 7 ++++++- .../01049_window_view_window_functions.reference | 2 ++ .../0_stateless/01049_window_view_window_functions.sql | 3 +++ 3 files changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsTimeWindow.cpp b/src/Functions/FunctionsTimeWindow.cpp index 77d740803be..faea9c6ba58 100644 --- a/src/Functions/FunctionsTimeWindow.cpp +++ b/src/Functions/FunctionsTimeWindow.cpp @@ -622,7 +622,12 @@ struct TimeWindowImpl { auto type = WhichDataType(arguments[0].type); if (type.isTuple()) - return std::static_pointer_cast(arguments[0].type)->getElement(0); + { + const auto & tuple_elems = std::static_pointer_cast(arguments[0].type)->getElements(); + if (tuple_elems.empty()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Tuple passed to {} should not be empty", function_name); + return tuple_elems[0]; + } else if (type.isUInt32()) return std::make_shared(); else diff --git a/tests/queries/0_stateless/01049_window_view_window_functions.reference b/tests/queries/0_stateless/01049_window_view_window_functions.reference index 2d49664b280..47d1ccc57dd 100644 --- a/tests/queries/0_stateless/01049_window_view_window_functions.reference +++ b/tests/queries/0_stateless/01049_window_view_window_functions.reference @@ -67,3 +67,5 @@ SELECT toDateTime(hopEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 2020-01-10 00:00:00 SELECT hopEnd(hop(toDateTime('2019-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa')); 2019-01-10 00:00:00 +SELECT hopStart(tuple()); -- { serverError ILLEGAL_COLUMN } +SELECT hopEnd(tuple()); -- { serverError ILLEGAL_COLUMN } diff --git a/tests/queries/0_stateless/01049_window_view_window_functions.sql b/tests/queries/0_stateless/01049_window_view_window_functions.sql index 617019bd2c6..3638dd1a3b2 100644 --- a/tests/queries/0_stateless/01049_window_view_window_functions.sql +++ b/tests/queries/0_stateless/01049_window_view_window_functions.sql @@ -36,3 +36,6 @@ SELECT hopEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, I SELECT toDateTime(hopEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); SELECT toDateTime(hopEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); SELECT hopEnd(hop(toDateTime('2019-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa')); + +SELECT hopStart(tuple()); -- { serverError ILLEGAL_COLUMN } +SELECT hopEnd(tuple()); -- { serverError ILLEGAL_COLUMN } From 2657e2b3ef2f98802fd2b8ebcd359fe756b709c6 Mon Sep 17 00:00:00 2001 From: Graham Campbell Date: Fri, 9 Aug 2024 11:08:41 +0100 Subject: [PATCH 510/644] Do not apply redundant sorting removal when there's an offset --- .../Optimizations/removeRedundantSorting.cpp | 8 ++-- .../02496_remove_redundant_sorting.reference | 37 +++++++++++++++++ .../02496_remove_redundant_sorting.sh | 22 ++++++++++ ...emove_redundant_sorting_analyzer.reference | 41 +++++++++++++++++++ 4 files changed, 105 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp index 7cac7bee6ec..f0094f0f8d2 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -59,9 +60,10 @@ public: if (typeid_cast(current_step) || typeid_cast(current_step) /// (1) if there are LIMITs on top of ORDER BY, the ORDER BY is non-removable - || typeid_cast(current_step) /// (2) if ORDER BY is with FILL WITH, it is non-removable - || typeid_cast(current_step) /// (3) ORDER BY will change order of previous sorting - || typeid_cast(current_step)) /// (4) aggregation change order + || typeid_cast(current_step) /// (2) OFFSET on top of ORDER BY, the ORDER BY is non-removable + || typeid_cast(current_step) /// (3) if ORDER BY is with FILL WITH, it is non-removable + || typeid_cast(current_step) /// (4) ORDER BY will change order of previous sorting + || typeid_cast(current_step)) /// (5) aggregation change order { logStep("nodes_affect_order/push", current_node); nodes_affect_order.push_back(current_node); diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference index 77ef213b36d..a0a1fd60812 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference @@ -527,3 +527,40 @@ Expression (Projection) 2 4 1 3 0 2 +-- presence of an inner OFFSET retains the ORDER BY +-- query +WITH + t1 AS ( + SELECT SUM(a) AS a, b + FROM + VALUES ( + 'b UInt32, a Int32', + (1, 1), + (2, 0) + ) + GROUP BY 2 + ) +SELECT + SUM(a) +FROM ( + SELECT a, b + FROM t1 + ORDER BY 1 DESC, 2 + OFFSET 1 +) t2 +-- explain +Expression (Projection) + Expression (Before ORDER BY) + Aggregating + Expression (Before GROUP BY) + Offset + Expression (Projection) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + Expression (Projection) + Expression (Before ORDER BY) + Aggregating + Expression (Before GROUP BY) + ReadFromStorage (Values) +-- execute +0 diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh index 646e2501a99..d59b4387101 100755 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh @@ -329,3 +329,25 @@ FROM ORDER BY number DESC )" run_query "$query" + +echo "-- presence of an inner OFFSET retains the ORDER BY" +query="WITH + t1 AS ( + SELECT SUM(a) AS a, b + FROM + VALUES ( + 'b UInt32, a Int32', + (1, 1), + (2, 0) + ) + GROUP BY 2 + ) +SELECT + SUM(a) +FROM ( + SELECT a, b + FROM t1 + ORDER BY 1 DESC, 2 + OFFSET 1 +) t2" +run_query "$query" diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference index b6a2e3182df..58441de5f22 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference @@ -533,3 +533,44 @@ Expression (Project names) 2 4 1 3 0 2 +-- presence of an inner OFFSET retains the ORDER BY +-- query +WITH + t1 AS ( + SELECT SUM(a) AS a, b + FROM + VALUES ( + 'b UInt32, a Int32', + (1, 1), + (2, 0) + ) + GROUP BY 2 + ) +SELECT + SUM(a) +FROM ( + SELECT a, b + FROM t1 + ORDER BY 1 DESC, 2 + OFFSET 1 +) t2 +-- explain +Expression (Project names) + Expression (Projection) + Aggregating + Expression (Before GROUP BY) + Expression (Change column names to column identifiers) + Expression (Project names) + Offset + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + Expression (Projection) + Expression (Change column names to column identifiers) + Expression (Project names) + Expression (Projection) + Aggregating + Expression (Before GROUP BY) + Expression (Change column names to column identifiers) + ReadFromStorage (Values) +-- execute +0 From dccb6bdd88ef26244ddb1c9de8d1232140036294 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 9 Aug 2024 18:33:05 +0800 Subject: [PATCH 511/644] fix failed uts --- .../Formats/Impl/ORCBlockOutputFormat.cpp | 47 +++++++------------ .../Formats/Impl/ORCBlockOutputFormat.h | 5 -- 2 files changed, 17 insertions(+), 35 deletions(-) diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index bd89ae0fa86..4a7a23158ff 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -280,20 +280,28 @@ void ORCBlockOutputFormat::writeColumn( DataTypePtr & type, const PaddedPODArray * null_bytemap) { - orc_column.numElements = column.size(); + size_t rows = column.size(); + orc_column.resize(rows); + orc_column.numElements = rows; + + /// Calculate orc_column.hasNulls if (null_bytemap) - { orc_column.hasNulls = !memoryIsZero(null_bytemap->data(), 0, null_bytemap->size()); - if (orc_column.hasNulls) - { - orc_column.notNull.resize(null_bytemap->size()); - for (size_t i = 0; i < null_bytemap->size(); ++i) - orc_column.notNull[i] = !(*null_bytemap)[i]; - } - } else orc_column.hasNulls = false; + /// Fill orc_column.notNull + if (orc_column.hasNulls) + { + for (size_t i = 0; i < rows; ++i) + orc_column.notNull[i] = !(*null_bytemap)[i]; + } + else + { + for (size_t i = 0; i < rows; ++i) + orc_column.notNull[i] = 1; + } + /// ORC doesn't have unsigned types, so cast everything to signed and sign-extend to Int64 to /// make the ORC library calculate min and max correctly. switch (type->getTypeId()) @@ -516,27 +524,6 @@ void ORCBlockOutputFormat::writeColumn( } } -size_t ORCBlockOutputFormat::getColumnSize(const IColumn & column, DataTypePtr & type) -{ - if (type->getTypeId() == TypeIndex::Array) - { - auto nested_type = assert_cast(*type).getNestedType(); - const IColumn & nested_column = assert_cast(column).getData(); - return std::max(column.size(), getColumnSize(nested_column, nested_type)); - } - - return column.size(); -} - -size_t ORCBlockOutputFormat::getMaxColumnSize(Chunk & chunk) -{ - size_t columns_num = chunk.getNumColumns(); - size_t max_column_size = 0; - for (size_t i = 0; i != columns_num; ++i) - max_column_size = std::max(max_column_size, getColumnSize(*chunk.getColumns()[i], data_types[i])); - return max_column_size; -} - void ORCBlockOutputFormat::consume(Chunk chunk) { if (!writer) diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h index 28837193d1a..06ecac9b820 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h @@ -69,11 +69,6 @@ private: void writeColumn(orc::ColumnVectorBatch & orc_column, const IColumn & column, DataTypePtr & type, const PaddedPODArray * null_bytemap); - /// These two functions are needed to know maximum nested size of arrays to - /// create an ORC Batch with the appropriate size - size_t getColumnSize(const IColumn & column, DataTypePtr & type); - size_t getMaxColumnSize(Chunk & chunk); - void prepareWriter(); const FormatSettings format_settings; From 4ced1f37e8e478ef806bd623a411916c18169dfe Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 11:56:53 +0100 Subject: [PATCH 512/644] Escape quote --- docker/test/stateless/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index c59d36114ae..acdcda753d7 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -178,7 +178,7 @@ attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 202 clickhouse-client --query "CREATE TABLE minio_audit_logs ( log String, - event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(trim(BOTH '"' FROM JSONExtractRaw(log, 'time')), 9, 'UTC') + event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(trim(BOTH '\"' FROM JSONExtractRaw(log, 'time')), 9, 'UTC') ) ENGINE = MergeTree ORDER BY tuple()" @@ -186,7 +186,7 @@ ORDER BY tuple()" clickhouse-client --query "CREATE TABLE minio_server_logs ( log String, - event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(trim(BOTH '"' FROM JSONExtractRaw(log, 'time')), 9, 'UTC') + event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(trim(BOTH '\"' FROM JSONExtractRaw(log, 'time')), 9, 'UTC') ) ENGINE = MergeTree ORDER BY tuple()" From ade1228b9578d5c0d7124a9d5c40ac3207e48074 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 13:01:00 +0200 Subject: [PATCH 513/644] Fix order --- ...rofile_events_from_query_log_and_client.reference | 2 +- ...02675_profile_events_from_query_log_and_client.sh | 12 +++++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference index 9dbac8d34f2..448eca3e5b1 100644 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference @@ -1,11 +1,11 @@ INSERT TO S3 -Successful write requests 3 S3Clients 1 S3CompleteMultipartUpload 1 S3CreateMultipartUpload 1 S3HeadObject 2 S3ReadRequestsCount 2 S3UploadPart 1 +Successful write requests 3 CHECK WITH query_log QueryFinish S3CreateMultipartUpload 1 S3UploadPart 1 S3CompleteMultipartUpload 1 S3PutObject 0 CREATE diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh index cae20be79dc..6d770b308b5 100755 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh @@ -18,11 +18,13 @@ WITH '(\\w+): (\\d+)' AS pattern, AND line NOT LIKE '%Microseconds%' AND line NOT LIKE '%S3DiskConnections%' AND line NOT LIKE '%S3DiskAddresses') AS pe_map -SELECT untuple(arrayJoin(pe_map) AS pe) -WHERE tupleElement(pe, 1) not like '%WriteRequests%' -UNION ALL -SELECT 'Successful write requests', - (pe_map['S3WriteRequestsCount'] - pe_map['S3WriteRequestsErrors'])::UInt64 +SELECT * FROM ( + SELECT untuple(arrayJoin(pe_map) AS pe) + WHERE tupleElement(pe, 1) not like '%WriteRequests%' + UNION ALL + SELECT 'Successful write requests', + (pe_map['S3WriteRequestsCount'] - pe_map['S3WriteRequestsErrors'])::UInt64 +) ORDER BY 1 " echo "CHECK WITH query_log" From 36c0c4562b8622b84012a12e29175f272bda2b0b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 13:14:30 +0200 Subject: [PATCH 514/644] Fix race in WithRetries --- src/Backups/WithRetries.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Backups/WithRetries.cpp b/src/Backups/WithRetries.cpp index 181e6331ac9..9f22085f5a9 100644 --- a/src/Backups/WithRetries.cpp +++ b/src/Backups/WithRetries.cpp @@ -68,13 +68,19 @@ const WithRetries::KeeperSettings & WithRetries::getKeeperSettings() const WithRetries::FaultyKeeper WithRetries::getFaultyZooKeeper() const { - /// We need to create new instance of ZooKeeperWithFaultInjection each time a copy a pointer to ZooKeeper client there + zkutil::ZooKeeperPtr current_zookeeper; + { + std::lock_guard lock(zookeeper_mutex); + current_zookeeper = zookeeper; + } + + /// We need to create new instance of ZooKeeperWithFaultInjection each time and copy a pointer to ZooKeeper client there /// The reason is that ZooKeeperWithFaultInjection may reset the underlying pointer and there could be a race condition /// when the same object is used from multiple threads. auto faulty_zookeeper = ZooKeeperWithFaultInjection::createInstance( settings.keeper_fault_injection_probability, settings.keeper_fault_injection_seed, - zookeeper, + current_zookeeper, log->name(), log); From 1e3ccbc3ec81c5b9d79a034159181f1f6bdb195c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 9 Aug 2024 11:22:44 +0000 Subject: [PATCH 515/644] add perf test for subcolumns --- .../optimize_functions_to_subcolumns.xml | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) create mode 100644 tests/performance/optimize_functions_to_subcolumns.xml diff --git a/tests/performance/optimize_functions_to_subcolumns.xml b/tests/performance/optimize_functions_to_subcolumns.xml new file mode 100644 index 00000000000..a246aae7950 --- /dev/null +++ b/tests/performance/optimize_functions_to_subcolumns.xml @@ -0,0 +1,27 @@ + + + 1 + 4 + + + + CREATE TABLE t_subcolumns (a Array(UInt64), s Nullable(String), m Map(String, UInt64)) ENGINE = MergeTree ORDER BY tuple() + + + + INSERT INTO t_subcolumns SELECT range(number % 20), toString(number), mapFromArrays(range(number % 20), range(number % 20)) FROM numbers_mt(50000000) + + + + OPTIMIZE TABLE t_subcolumns FINAL + + + SELECT count() FROM t_subcolumns WHERE NOT ignore(length(a)) + SELECT count() FROM t_subcolumns WHERE notEmpty(a) + SELECT count() FROM t_subcolumns WHERE NOT ignore(length(m)) + SELECT count() FROM t_subcolumns WHERE notEmpty(m) + SELECT count() FROM t_subcolumns WHERE isNotNull(s) + SELECT count(s) FROM t_subcolumns + + DROP TABLE t_subcolumns + From e8f2f65e62c65878463879396b7ebdceed48c5e3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Aug 2024 11:51:23 +0000 Subject: [PATCH 516/644] Avoid converting type to string and back in _CAST --- src/Functions/CastOverloadResolver.cpp | 23 +++++++++++++++---- src/Functions/CastOverloadResolver.h | 6 ++++- src/Functions/toBool.cpp | 3 +-- src/Interpreters/ActionsDAG.cpp | 12 +++++----- src/Interpreters/castColumn.cpp | 6 ++--- .../optimizeUseAggregateProjection.cpp | 21 ++++++----------- src/Processors/Transforms/WindowTransform.cpp | 17 ++------------ src/Storages/MergeTree/KeyCondition.cpp | 7 ++---- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 20 +++++----------- ...rojection_with_normalized_states.reference | 1 + ...gate_projection_with_normalized_states.sql | 2 ++ 11 files changed, 52 insertions(+), 66 deletions(-) diff --git a/src/Functions/CastOverloadResolver.cpp b/src/Functions/CastOverloadResolver.cpp index 49f63073aaf..6cb4d492fd8 100644 --- a/src/Functions/CastOverloadResolver.cpp +++ b/src/Functions/CastOverloadResolver.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -35,7 +36,7 @@ FunctionBasePtr createFunctionBaseCast( class CastOverloadResolverImpl : public IFunctionOverloadResolver { public: - const char * getNameImpl() const + static const char * getNameImpl(CastType cast_type, bool internal) { if (cast_type == CastType::accurate) return "accurateCast"; @@ -49,7 +50,7 @@ public: String getName() const override { - return getNameImpl(); + return getNameImpl(cast_type, internal); } size_t getNumberOfArguments() const override { return 2; } @@ -79,10 +80,22 @@ public: } } + static FunctionBasePtr createInternalCast(ColumnWithTypeAndName from, DataTypePtr to, CastType cast_type, std::optional diagnostic) + { + if (cast_type == CastType::accurateOrNull && !isVariant(to)) + to = makeNullable(to); + + ColumnsWithTypeAndName arguments; + arguments.emplace_back(std::move(from)); + arguments.emplace_back().type = std::make_unique(); + + return createFunctionBaseCast(nullptr, getNameImpl(cast_type, true), arguments, to, diagnostic, cast_type); + } + protected: FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override { - return createFunctionBaseCast(context, getNameImpl(), arguments, return_type, diagnostic, cast_type); + return createFunctionBaseCast(context, getNameImpl(cast_type, internal), arguments, return_type, diagnostic, cast_type); } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override @@ -130,9 +143,9 @@ private: }; -FunctionOverloadResolverPtr createInternalCastOverloadResolver(CastType type, std::optional diagnostic) +FunctionBasePtr createInternalCast(ColumnWithTypeAndName from, DataTypePtr to, CastType cast_type, std::optional diagnostic) { - return CastOverloadResolverImpl::create(ContextPtr{}, type, true, diagnostic); + return CastOverloadResolverImpl::createInternalCast(std::move(from), std::move(to), cast_type, std::move(diagnostic)); } REGISTER_FUNCTION(CastOverloadResolvers) diff --git a/src/Functions/CastOverloadResolver.h b/src/Functions/CastOverloadResolver.h index 7d98f774812..66f9d6cfcaf 100644 --- a/src/Functions/CastOverloadResolver.h +++ b/src/Functions/CastOverloadResolver.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -11,6 +12,9 @@ namespace DB class IFunctionOverloadResolver; using FunctionOverloadResolverPtr = std::shared_ptr; +class IFunctionBase; +using FunctionBasePtr = std::shared_ptr; + enum class CastType : uint8_t { nonAccurate, @@ -24,6 +28,6 @@ struct CastDiagnostic std::string column_to; }; -FunctionOverloadResolverPtr createInternalCastOverloadResolver(CastType type, std::optional diagnostic); +FunctionBasePtr createInternalCast(ColumnWithTypeAndName from, DataTypePtr to, CastType cast_type, std::optional diagnostic); } diff --git a/src/Functions/toBool.cpp b/src/Functions/toBool.cpp index 6f2c436c1ea..ac595d313e3 100644 --- a/src/Functions/toBool.cpp +++ b/src/Functions/toBool.cpp @@ -54,8 +54,7 @@ namespace } }; - FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {}); - auto func_cast = func_builder_cast->build(cast_args); + auto func_cast = createInternalCast(arguments[0], result_type, CastType::nonAccurate, {}); return func_cast->execute(cast_args, result_type, arguments[0].column->size()); } }; diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index df1c0aa1f2a..2a594839c6a 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -301,11 +301,11 @@ const ActionsDAG::Node & ActionsDAG::addCast(const Node & node_to_cast, const Da column.column = DataTypeString().createColumnConst(0, cast_type_constant_value); column.type = std::make_shared(); - const auto * cast_type_constant_node = &addColumn(std::move(column)); + const auto * cast_type_constant_node = &addColumn(column); ActionsDAG::NodeRawConstPtrs children = {&node_to_cast, cast_type_constant_node}; - FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {}); + auto func_base_cast = createInternalCast(ColumnWithTypeAndName{node_to_cast.result_type, node_to_cast.result_name}, cast_type, CastType::nonAccurate, {}); - return addFunction(func_builder_cast, std::move(children), result_name); + return addFunction(func_base_cast, std::move(children), result_name); } const ActionsDAG::Node & ActionsDAG::addFunctionImpl( @@ -1547,11 +1547,11 @@ ActionsDAG ActionsDAG::makeConvertingActions( const auto * left_arg = dst_node; CastDiagnostic diagnostic = {dst_node->result_name, res_elem.name}; - FunctionOverloadResolverPtr func_builder_cast - = createInternalCastOverloadResolver(CastType::nonAccurate, std::move(diagnostic)); + ColumnWithTypeAndName left_column{nullptr, dst_node->result_type, {}}; + auto func_base_cast = createInternalCast(std::move(left_column), res_elem.type, CastType::nonAccurate, std::move(diagnostic)); NodeRawConstPtrs children = { left_arg, right_arg }; - dst_node = &actions_dag.addFunction(func_builder_cast, std::move(children), {}); + dst_node = &actions_dag.addFunction(func_base_cast, std::move(children), {}); } if (dst_node->column && isColumnConst(*dst_node->column) && !(res_elem.column && isColumnConst(*res_elem.column))) diff --git a/src/Interpreters/castColumn.cpp b/src/Interpreters/castColumn.cpp index 906dfb84b14..a779c9bc34d 100644 --- a/src/Interpreters/castColumn.cpp +++ b/src/Interpreters/castColumn.cpp @@ -26,11 +26,9 @@ static ColumnPtr castColumn(CastType cast_type, const ColumnWithTypeAndName & ar "" } }; - auto get_cast_func = [cast_type, &arguments] + auto get_cast_func = [from = arg, to = type, cast_type] { - - FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(cast_type, {}); - return func_builder_cast->build(arguments); + return createInternalCast(from, to, cast_type, {}); }; FunctionBasePtr func_cast = cache ? cache->getOrSet(cast_type, from_name, to_name, std::move(get_cast_func)) : get_cast_func(); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 52d1931c51e..b31ee7ea53c 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -255,20 +255,13 @@ static void appendAggregateFunctions( const auto * node = input; - if (node->result_name != aggregate.column_name) - { - if (DataTypeAggregateFunction::strictEquals(type, node->result_type)) - { - node = &proj_dag.addAlias(*node, aggregate.column_name); - } - else - { - /// Cast to aggregate types specified in query if it's not - /// strictly the same as the one specified in projection. This - /// is required to generate correct results during finalization. - node = &proj_dag.addCast(*node, type, aggregate.column_name); - } - } + if (!DataTypeAggregateFunction::strictEquals(type, node->result_type)) + /// Cast to aggregate types specified in query if it's not + /// strictly the same as the one specified in projection. This + /// is required to generate correct results during finalization. + node = &proj_dag.addCast(*node, type, aggregate.column_name); + else if (node->result_name != aggregate.column_name) + node = &proj_dag.addAlias(*node, aggregate.column_name); proj_dag_outputs.push_back(node); } diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index c26cd7cc8c3..c27c230c741 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2337,22 +2337,9 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction argument_types[2]->getName()); } - const auto from_name = argument_types[2]->getName(); - const auto to_name = argument_types[0]->getName(); - ColumnsWithTypeAndName arguments + auto get_cast_func = [from = argument_types[2], to = argument_types[0]] { - { argument_types[2], "" }, - { - DataTypeString().createColumnConst(0, to_name), - std::make_shared(), - "" - } - }; - - auto get_cast_func = [&arguments] - { - FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::accurate, {}); - return func_builder_cast->build(arguments); + return createInternalCast({from, {}}, to, CastType::accurate, {}); }; func_cast = get_cast_func(); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index dfb43c4e75d..aa7a498d5a3 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1956,11 +1956,8 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme auto common_type_maybe_nullable = (key_expr_type_is_nullable && !common_type->isNullable()) ? DataTypePtr(std::make_shared(common_type)) : common_type; - ColumnsWithTypeAndName arguments{ - {nullptr, key_expr_type, ""}, - {DataTypeString().createColumnConst(1, common_type_maybe_nullable->getName()), common_type_maybe_nullable, ""}}; - FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {}); - auto func_cast = func_builder_cast->build(arguments); + + auto func_cast = createInternalCast({key_expr_type, {}}, common_type_maybe_nullable, CastType::nonAccurate, {}); /// If we know the given range only contains one value, then we treat all functions as positive monotonic. if (!single_point && !func_cast->hasInformationAboutMonotonicity()) diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 36ff6c0a4bd..9c82817e8cb 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -152,23 +152,15 @@ const ActionsDAG::Node & addFunction( const ActionsDAG::Node & addCast( const ActionsDAGPtr & dag, const ActionsDAG::Node & node_to_cast, - const String & type_name, + const DataTypePtr & to_type, OriginalToNewNodeMap & node_remap) { - if (node_to_cast.result_type->getName() == type_name) + if (!node_to_cast.result_type->equals(*to_type)) return node_to_cast; - Field cast_type_constant_value(type_name); - - ColumnWithTypeAndName column; - column.column = DataTypeString().createColumnConst(0, cast_type_constant_value); - column.type = std::make_shared(); - - const auto * cast_type_constant_node = &dag->addColumn(std::move(column)); - ActionsDAG::NodeRawConstPtrs children = {&node_to_cast, cast_type_constant_node}; - FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {}); - - return addFunction(dag, func_builder_cast, std::move(children), node_remap); + const auto & new_node = dag->addCast(node_to_cast, to_type, {}); + node_remap[new_node.result_name] = {dag.get(), &new_node}; + return new_node; } /// Normalizes the filter node by adding AND with a constant true. @@ -332,7 +324,7 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction /// Build AND(last_step_result_node, true) const auto & and_node = addAndTrue(last_step_dag, *last_step_result_node_info.node, node_remap); /// Build CAST(and_node, type of PREWHERE column) - const auto & cast_node = addCast(last_step_dag, and_node, output->result_type->getName(), node_remap); + const auto & cast_node = addCast(last_step_dag, and_node, output->result_type, node_remap); /// Add alias for the result with the name of the PREWHERE column const auto & prewhere_result_node = last_step_dag->addAlias(cast_node, output->result_name); last_step_dag->addOrReplaceInOutputs(prewhere_result_node); diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference index 25aa9dc5dec..37993873983 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference @@ -1,2 +1,3 @@ 3 950 990 500 2000 +[950] [999] diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql index 5375823aa8e..956bf3711a2 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql @@ -29,4 +29,6 @@ FROM cluster('test_cluster_two_shards', currentDatabase(), r) WHERE a = 'x' settings prefer_localhost_replica=0; +SELECT quantilesTimingMerge(0.95)(q), quantilesTimingMerge(toInt64(1))(q) FROM remote('127.0.0.{1,2}', currentDatabase(), r); + DROP TABLE r; From 3d850f8ceb0ca5cfae26e8faa7c4d900cc4e8fda Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 9 Aug 2024 13:58:02 +0200 Subject: [PATCH 517/644] fix --- src/Processors/Sources/ShellCommandSource.cpp | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 23359367a9b..f55a3713215 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -70,17 +70,16 @@ static void makeFdBlocking(int fd) static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_milliseconds) { + auto logger = getLogger("TimeoutReadBufferFromFileDescriptor"); + auto describe_fd = [](const auto & pollfd) { return fmt::format("(fd={}, flags={})", pollfd.fd, fcntl(pollfd.fd, F_GETFL)); }; + int res; while (true) { Stopwatch watch; - auto describe_fd = [](const auto & pollfd) { return fmt::format("(fd={}, flags={})", pollfd.fd, fcntl(pollfd.fd, F_GETFL)); }; - LOG_TEST( - getLogger("TimeoutReadBufferFromFileDescriptor"), - "Polling descriptors: {}", - fmt::join(std::span(pfds, pfds + num) | std::views::transform(describe_fd), ", ")); + LOG_TEST(logger, "Polling descriptors: {}", fmt::join(std::span(pfds, pfds + num) | std::views::transform(describe_fd), ", ")); res = poll(pfds, static_cast(num), static_cast(timeout_milliseconds)); @@ -92,11 +91,7 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond const auto elapsed = watch.elapsedMilliseconds(); if (timeout_milliseconds <= elapsed) { - LOG_TEST( - getLogger("TimeoutReadBufferFromFileDescriptor"), - "Timeout exceeded: elapsed={}, timeout={}", - elapsed, - timeout_milliseconds); + LOG_TEST(logger, "Timeout exceeded: elapsed={}, timeout={}", elapsed, timeout_milliseconds); break; } timeout_milliseconds -= elapsed; @@ -107,9 +102,8 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond } } - auto describe_fd = [](const auto & pollfd) { return fmt::format("(fd={}, flags={})", pollfd.fd, fcntl(pollfd.fd, F_GETFL)); }; LOG_TEST( - getLogger("TimeoutReadBufferFromFileDescriptor"), + logger, "Poll for descriptors: {} returned {}", fmt::join(std::span(pfds, pfds + num) | std::views::transform(describe_fd), ", "), res); From 1875e8d9cd581871c247e3a4cc58f1f57ffa0659 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 9 Aug 2024 14:06:52 +0200 Subject: [PATCH 518/644] Fix UB in tumbleEnd and tumbleStart This was causing segfaults because of a NULL pointer dereference --- src/Functions/FunctionsTimeWindow.cpp | 7 ++++++- .../01049_window_view_window_functions.reference | 2 ++ .../0_stateless/01049_window_view_window_functions.sql | 2 ++ 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsTimeWindow.cpp b/src/Functions/FunctionsTimeWindow.cpp index faea9c6ba58..88b85c48326 100644 --- a/src/Functions/FunctionsTimeWindow.cpp +++ b/src/Functions/FunctionsTimeWindow.cpp @@ -267,7 +267,12 @@ struct TimeWindowImpl { auto type = WhichDataType(arguments[0].type); if (type.isTuple()) - return std::static_pointer_cast(arguments[0].type)->getElement(0); + { + const auto & tuple_elems = std::static_pointer_cast(arguments[0].type)->getElements(); + if (tuple_elems.empty()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Tuple passed to {} should not be empty", function_name); + return tuple_elems[0]; + } else if (type.isUInt32()) return std::make_shared(); else diff --git a/tests/queries/0_stateless/01049_window_view_window_functions.reference b/tests/queries/0_stateless/01049_window_view_window_functions.reference index 47d1ccc57dd..073301104d2 100644 --- a/tests/queries/0_stateless/01049_window_view_window_functions.reference +++ b/tests/queries/0_stateless/01049_window_view_window_functions.reference @@ -69,3 +69,5 @@ SELECT hopEnd(hop(toDateTime('2019-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DA 2019-01-10 00:00:00 SELECT hopStart(tuple()); -- { serverError ILLEGAL_COLUMN } SELECT hopEnd(tuple()); -- { serverError ILLEGAL_COLUMN } +SELECT tumbleStart(tuple()); -- { serverError ILLEGAL_COLUMN } +SELECT tumbleEnd(tuple()); -- { serverError ILLEGAL_COLUMN } diff --git a/tests/queries/0_stateless/01049_window_view_window_functions.sql b/tests/queries/0_stateless/01049_window_view_window_functions.sql index 3638dd1a3b2..fb2b4b4949a 100644 --- a/tests/queries/0_stateless/01049_window_view_window_functions.sql +++ b/tests/queries/0_stateless/01049_window_view_window_functions.sql @@ -39,3 +39,5 @@ SELECT hopEnd(hop(toDateTime('2019-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DA SELECT hopStart(tuple()); -- { serverError ILLEGAL_COLUMN } SELECT hopEnd(tuple()); -- { serverError ILLEGAL_COLUMN } +SELECT tumbleStart(tuple()); -- { serverError ILLEGAL_COLUMN } +SELECT tumbleEnd(tuple()); -- { serverError ILLEGAL_COLUMN } From 20563bc6cbc6e70eb6926c5f21fded356270f40f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 9 Aug 2024 12:19:44 +0000 Subject: [PATCH 519/644] Make test work with ReplicatedDatabase in test --- .../0_stateless/03217_filtering_in_system_tables.sql | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql index 2ce63559b99..eb506dfe39a 100644 --- a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql @@ -9,9 +9,11 @@ CREATE TABLE test_03217_system_tables_replica_2(x UInt32) ORDER BY x; -- Make sure we can read both replicas -SELECT 'both', database, table, replica_name FROM system.replicas WHERE database = currentDatabase(); +-- The replica name might be altered because of `_functional_tests_helper_database_replicated_replace_args_macros`, +-- thus we need to use `left` +SELECT 'both', database, table, left(replica_name, 2) FROM system.replicas WHERE database = currentDatabase(); -- If filtering is not done correctly on database-table column, then this query report to read 2 rows, which are the above tables -SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = 'test_03217_system_tables_replica_1' AND replica_name = 'r1'; +SELECT database, table, left(replica_name, 2) FROM system.replicas WHERE database = currentDatabase() AND table = 'test_03217_system_tables_replica_1' AND replica_name LIKE 'r1%'; SYSTEM FLUSH LOGS; -- argMax is necessary to make the test repeatable @@ -24,5 +26,5 @@ SELECT argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 -- StorageSystemReplicas SELECT argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 AND current_database = currentDatabase() - AND query LIKE '%SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = \'test_03217_system_tables_replica_1\' AND replica_name = \'r1\';' + AND query LIKE '%SELECT database, table, left(replica_name, 2) FROM system.replicas WHERE database = currentDatabase() AND table = \'test_03217_system_tables_replica_1\' AND replica_name LIKE \'r1\%\';' AND type = 'QueryFinish'; From 96b54df1638043e27887099df5f7cf310d1e7fa4 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 9 Aug 2024 15:04:03 +0200 Subject: [PATCH 520/644] fix bugprone-macro-parentheses --- src/Interpreters/SystemLog.cpp | 4 +++- src/Interpreters/SystemLog.h | 2 ++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index d4403b72583..832c39bfaf8 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -284,11 +284,13 @@ ASTPtr getCreateTableQueryClean(const StorageID & table_id, ContextPtr context) SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConfiguration & config) { +/// NOLINTBEGIN(bugprone-macro-parentheses) #define CREATE_PUBLIC_MEMBERS(log_type, member, descr) \ member = createSystemLog(global_context, "system", #member, config, #member, descr); \ LIST_OF_ALL_SYSTEM_LOGS(CREATE_PUBLIC_MEMBERS) #undef CREATE_PUBLIC_MEMBERS +/// NOLINTEND(bugprone-macro-parentheses) if (session_log) global_context->addWarningMessage("Table system.session_log is enabled. It's unreliable and may contain garbage. Do not use it for any kind of security monitoring."); @@ -333,7 +335,7 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf std::vector SystemLogs::getAllLogs() const { #define GET_RAW_POINTERS(log_type, member, descr) \ - member.get(), \ + (member).get(), \ std::vector result = { LIST_OF_ALL_SYSTEM_LOGS(GET_RAW_POINTERS) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 24ef6a18eb8..9e1af3578bd 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -60,11 +60,13 @@ namespace DB }; */ +/// NOLINTBEGIN(bugprone-macro-parentheses) #define FORWARD_DECLARATION(log_type, member, descr) \ class log_type; \ LIST_OF_ALL_SYSTEM_LOGS(FORWARD_DECLARATION) #undef FORWARD_DECLARATION +/// NOLINTEND(bugprone-macro-parentheses) /// System logs should be destroyed in destructor of the last Context and before tables, From 21be195dd8f841c3cfaf453bb7faba02627b9c0a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 9 Aug 2024 13:14:09 +0000 Subject: [PATCH 521/644] Revert unnecessary change --- .../clickhouse_path/format_schemas/test.capnp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp index 247e7b9ceca..44f1961205b 100644 --- a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp +++ b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp @@ -7,4 +7,4 @@ struct TestRecordStruct val1 @2 : Text; val2 @3 : Float32; val3 @4 : UInt8; -} +} \ No newline at end of file From 8cf5f6d6168342a69b188b17588566a4ac85fa69 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 9 Aug 2024 13:20:05 +0000 Subject: [PATCH 522/644] Add empty cell to reports when time is missing --- tests/ci/report.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 0b6c818aed0..15b1512896a 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -770,10 +770,12 @@ def create_test_html_report( row.append(f'{test_result.status}') colspan += 1 + row.append("") if test_result.time is not None: has_test_time = True - row.append(f"{test_result.time}") - colspan += 1 + row.append(str(test_result.time)) + row.append("") + colspan += 1 if test_result.log_urls is not None: has_log_urls = True From b757522fc4ac545451acc398ab230323fb7c0fd3 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 9 Aug 2024 14:20:57 +0100 Subject: [PATCH 523/644] fix build --- programs/keeper/Keeper.cpp | 2 +- src/Server/HTTPHandlerFactory.cpp | 41 ++----------------------- src/Server/PrometheusRequestHandler.cpp | 4 +-- src/Server/PrometheusRequestHandler.h | 7 +++-- 4 files changed, 10 insertions(+), 44 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index ae51a62ff9c..a447a9e50f6 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -515,7 +515,7 @@ try "Prometheus: http://" + address.toString(), std::make_unique( std::move(my_http_context), - createKeeperPrometheusHandlerFactory(config_getter(), async_metrics, "PrometheusHandler-factory"), + createKeeperPrometheusHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 0ee45783d52..fc31ad2874e 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -122,7 +122,8 @@ static inline auto createHandlersFactoryFromConfig( } else if (handler_type == "prometheus") { - main_handler_factory->addHandler(createPrometheusHandlerFactoryForHTTPRule(config, prefix + "." + key, async_metrics)); + main_handler_factory->addHandler( + createPrometheusHandlerFactoryForHTTPRule(server, config, prefix + "." + key, async_metrics)); } else if (handler_type == "replicas_status") { @@ -199,19 +200,7 @@ HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco:: else if (name == "InterserverIOHTTPHandler-factory" || name == "InterserverIOHTTPSHandler-factory") return createInterserverHTTPHandlerFactory(server, name); else if (name == "PrometheusHandler-factory") -<<<<<<< HEAD - { - auto metrics_writer = std::make_shared(config, "prometheus", async_metrics); - return createPrometheusMainHandlerFactory(config, metrics_writer, name); - } -||||||| 02b8d563e3a - { - auto metrics_writer = std::make_shared(config, "prometheus", async_metrics); - return createPrometheusMainHandlerFactory(server, config, metrics_writer, name); - } -======= return createPrometheusHandlerFactory(server, config, async_metrics, name); ->>>>>>> master throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown HTTP handler factory name."); } @@ -298,34 +287,8 @@ void addDefaultHandlersFactory( ); factory.addHandler(query_handler); -<<<<<<< HEAD - /// We check that prometheus handler will be served on current (default) port. - /// Otherwise it will be created separately, see createHandlerFactory(...). - if (config.has("prometheus") && config.getInt("prometheus.port", 0) == 0) - { - auto writer = std::make_shared(config, "prometheus", async_metrics); - auto creator - = [writer]() -> std::unique_ptr { return std::make_unique(writer); }; - auto prometheus_handler = std::make_shared>(std::move(creator)); - prometheus_handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); - prometheus_handler->allowGetAndHeadRequest(); -||||||| 02b8d563e3a - /// We check that prometheus handler will be served on current (default) port. - /// Otherwise it will be created separately, see createHandlerFactory(...). - if (config.has("prometheus") && config.getInt("prometheus.port", 0) == 0) - { - auto writer = std::make_shared(config, "prometheus", async_metrics); - auto creator = [&server, writer] () -> std::unique_ptr - { - return std::make_unique(server, writer); - }; - auto prometheus_handler = std::make_shared>(std::move(creator)); - prometheus_handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); - prometheus_handler->allowGetAndHeadRequest(); -======= /// createPrometheusHandlerFactoryForHTTPRuleDefaults() can return nullptr if prometheus protocols must not be served on http port. if (auto prometheus_handler = createPrometheusHandlerFactoryForHTTPRuleDefaults(server, config, async_metrics)) ->>>>>>> master factory.addHandler(prometheus_handler); } diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index 52cda92d9b4..ae1fb6d629e 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -95,7 +95,7 @@ public: class PrometheusRequestHandler::ImplWithContext : public Impl { public: - explicit ImplWithContext(PrometheusRequestHandler & parent) : Impl(parent), default_settings(parent.server.context()->getSettingsRef()) { } + explicit ImplWithContext(PrometheusRequestHandler & parent) : Impl(parent), default_settings(server().context()->getSettingsRef()) { } virtual void handlingRequestWithContext(HTTPServerRequest & request, HTTPServerResponse & response) = 0; @@ -353,7 +353,7 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe if (request.getVersion() == HTTPServerRequest::HTTP_1_1) response.setChunkedTransferEncoding(true); - setResponseDefaultHeaders(response); + setResponseDefaultHeaders(response); impl->beforeHandlingRequest(request); impl->handleRequest(request, response); diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index 7aeed11d6b8..281ecf5260e 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -15,8 +15,11 @@ class WriteBufferFromHTTPServerResponse; class PrometheusRequestHandler : public HTTPRequestHandler { public: - PrometheusRequestHandler(const PrometheusRequestHandlerConfig & config_, - const AsynchronousMetrics & async_metrics_, std::shared_ptr metrics_writer_); + PrometheusRequestHandler( + IServer & server_, + const PrometheusRequestHandlerConfig & config_, + const AsynchronousMetrics & async_metrics_, + std::shared_ptr metrics_writer_); ~PrometheusRequestHandler() override; void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event_) override; From 65ebcd6f21b26144cb47e6b71c939517b1fb38a2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Aug 2024 13:55:47 +0000 Subject: [PATCH 524/644] Fixing test. --- .../0_stateless/01656_test_query_log_factories_info.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01656_test_query_log_factories_info.reference b/tests/queries/0_stateless/01656_test_query_log_factories_info.reference index 47b3133ceca..44531c19ab7 100644 --- a/tests/queries/0_stateless/01656_test_query_log_factories_info.reference +++ b/tests/queries/0_stateless/01656_test_query_log_factories_info.reference @@ -17,7 +17,7 @@ used_functions ['repeat'] arraySort(used_data_type_families) -['Array','Int32','Nullable','String'] +['Int32','Nullable','String'] used_database_engines ['Atomic'] From 43a38fb5f0563f50d38cf5d988db3b181b64f606 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 9 Aug 2024 15:11:08 +0100 Subject: [PATCH 525/644] rm redundant file --- programs/server/config.d/listen.xml | 3 --- 1 file changed, 3 deletions(-) delete mode 100644 programs/server/config.d/listen.xml diff --git a/programs/server/config.d/listen.xml b/programs/server/config.d/listen.xml deleted file mode 100644 index f94e5c88568..00000000000 --- a/programs/server/config.d/listen.xml +++ /dev/null @@ -1,3 +0,0 @@ - - :: - From 79cfffbaf88c3c34213d5144fd20f7888b6cabe0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 15:57:16 +0100 Subject: [PATCH 526/644] Even bigger queue --- docker/test/stateless/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index acdcda753d7..a030be92506 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -193,8 +193,8 @@ ORDER BY tuple()" # create minio log webhooks for both audit and server logs # use async inserts to avoid creating too many parts -./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" queue_size=300000 -./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" queue_size=300000 +./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" queue_size=1000000 +./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" queue_size=1000000 max_retries=100 retry=1 From 97eded0ac7aa41a9320729b418c8ab2ff1821202 Mon Sep 17 00:00:00 2001 From: kruglov Date: Fri, 9 Aug 2024 17:38:24 +0300 Subject: [PATCH 527/644] Fixed test_dependent_loading. event_time_microseconds has two dates connected with "\n" --- .../test_postgresql_replica_database_engine_2/test.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 75edb22aab1..7fdd17625a9 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -1127,9 +1127,13 @@ def test_dependent_loading(started_cluster): nested_time = instance.query( f"SELECT event_time_microseconds FROM system.text_log WHERE message like 'Loading table default.{uuid}_nested' and message not like '%like%'" ).strip() - time = instance.query( - f"SELECT event_time_microseconds FROM system.text_log WHERE message like 'Loading table default.{table}' and message not like '%like%'" - ).strip() + time = ( + instance.query( + f"SELECT event_time_microseconds FROM system.text_log WHERE message like 'Loading table default.{table}' and message not like '%like%'" + ) + .strip() + .split("\n")[-1] + ) instance.query( f"SELECT toDateTime64('{nested_time}', 6) < toDateTime64('{time}', 6)" ) From c13d348d1e8a467b9d16fc83214ef574752092e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 19:56:50 +0200 Subject: [PATCH 528/644] Fix test `00900_long_parquet_load` --- tests/queries/0_stateless/00900_long_parquet_load.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00900_long_parquet_load.sh b/tests/queries/0_stateless/00900_long_parquet_load.sh index 1bafb033f56..3a7022ac0cf 100755 --- a/tests/queries/0_stateless/00900_long_parquet_load.sh +++ b/tests/queries/0_stateless/00900_long_parquet_load.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-debug +# Tags: long, no-fasttest, no-debug, no-asan, no-msan, no-tsan # # Load all possible .parquet files found in submodules. From b5afddb1af0a9aeb4738cf3fb7b7242361469028 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 9 Aug 2024 22:56:25 +0200 Subject: [PATCH 529/644] Update optimize_functions_to_subcolumns.xml --- tests/performance/optimize_functions_to_subcolumns.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/performance/optimize_functions_to_subcolumns.xml b/tests/performance/optimize_functions_to_subcolumns.xml index a246aae7950..146af1605c4 100644 --- a/tests/performance/optimize_functions_to_subcolumns.xml +++ b/tests/performance/optimize_functions_to_subcolumns.xml @@ -1,6 +1,5 @@ - 1 4 From c61eef4a7659e4856cc3266d8d7dd28b4e095d2b Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 9 Aug 2024 21:17:49 +0000 Subject: [PATCH 530/644] Reimplement Dynamic type --- docs/en/sql-reference/data-types/dynamic.md | 123 +- src/Columns/ColumnDynamic.cpp | 881 ++++-- src/Columns/ColumnDynamic.h | 164 +- src/Columns/ColumnVariant.cpp | 29 +- src/Columns/ColumnVariant.h | 11 +- src/Columns/tests/gtest_column_dynamic.cpp | 428 ++- src/DataTypes/DataTypeDynamic.cpp | 79 +- src/DataTypes/DataTypeFactory.cpp | 6 + src/DataTypes/DataTypeFactory.h | 1 + src/DataTypes/DataTypeTuple.cpp | 13 +- src/DataTypes/DataTypesBinaryEncoding.cpp | 2 +- .../Serializations/SerializationDynamic.cpp | 361 ++- .../Serializations/SerializationDynamic.h | 8 +- .../SerializationDynamicElement.cpp | 127 +- .../SerializationDynamicElement.h | 8 +- .../SerializationVariantElement.cpp | 41 +- .../SerializationVariantElement.h | 6 +- src/Formats/JSONExtractTree.cpp | 31 +- src/Functions/FunctionsConversion.cpp | 326 +-- src/Functions/dynamicType.cpp | 91 +- tests/queries/0_stateless/00000_test.sql | 43 + ...03033_dynamic_text_serialization.reference | 26 +- .../03033_dynamic_text_serialization.sql | 4 +- .../03034_dynamic_conversions.reference | 18 +- .../0_stateless/03034_dynamic_conversions.sql | 6 +- .../03035_dynamic_sorting.reference | 715 +++-- .../0_stateless/03035_dynamic_sorting.sql | 67 +- ...ed_subcolumns_compact_merge_tree.reference | 20 + ...d_shared_subcolumns_compact_merge_tree.sql | 43 + ...ic_read_shared_subcolumns_memory.reference | 20 + ..._dynamic_read_shared_subcolumns_memory.sql | 43 + ..._read_shared_subcolumns_small.reference.j2 | 2460 +++++++++++++++++ ...ynamic_read_shared_subcolumns_small.sql.j2 | 46 + ...hared_subcolumns_wide_merge_tree.reference | 20 + ...read_shared_subcolumns_wide_merge_tree.sql | 43 + ..._1_horizontal_compact_merge_tree.reference | 94 +- ...merges_1_horizontal_compact_merge_tree.sql | 29 +- ...s_1_horizontal_compact_wide_tree.reference | 94 +- ..._merges_1_horizontal_compact_wide_tree.sql | 28 +- ...es_1_vertical_compact_merge_tree.reference | 94 +- ...c_merges_1_vertical_compact_merge_tree.sql | 31 +- ...erges_1_vertical_wide_merge_tree.reference | 94 +- ...amic_merges_1_vertical_wide_merge_tree.sql | 28 +- .../03037_dynamic_merges_small.reference.j2 | 376 ++- .../03037_dynamic_merges_small.sql.j2 | 28 +- ...ynamic_merges_compact_horizontal.reference | 84 +- ...sted_dynamic_merges_compact_horizontal.sql | 33 +- ..._dynamic_merges_compact_vertical.reference | 84 +- ...nested_dynamic_merges_compact_vertical.sql | 33 +- ...8_nested_dynamic_merges_small.reference.j2 | 340 ++- .../03038_nested_dynamic_merges_small.sql.j2 | 36 +- ...d_dynamic_merges_wide_horizontal.reference | 84 +- ..._nested_dynamic_merges_wide_horizontal.sql | 33 +- ...ted_dynamic_merges_wide_vertical.reference | 84 +- ...38_nested_dynamic_merges_wide_vertical.sql | 33 +- ...type_alters_1_compact_merge_tree.reference | 123 +- ...namic_type_alters_1_compact_merge_tree.sql | 20 +- ...040_dynamic_type_alters_1_memory.reference | 94 +- ...ic_type_alters_1_wide_merge_tree.reference | 157 +- .../03041_dynamic_type_check_table.sh | 2 +- .../03150_dynamic_type_mv_insert.reference | 50 +- .../03150_dynamic_type_mv_insert.sql | 7 + ...151_dynamic_type_scale_max_types.reference | 48 +- .../03151_dynamic_type_scale_max_types.sql | 8 +- .../03152_dynamic_type_simple.reference | 2 +- .../0_stateless/03152_dynamic_type_simple.sql | 8 + .../0_stateless/03153_dynamic_type_empty.sql | 2 + .../03159_dynamic_type_all_types.sql | 4 +- .../03172_dynamic_binary_serialization.sh | 4 +- ...3200_memory_engine_alter_dynamic.reference | 20 +- .../03200_memory_engine_alter_dynamic.sql | 4 +- 71 files changed, 6725 insertions(+), 1878 deletions(-) create mode 100644 tests/queries/0_stateless/00000_test.sql create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.reference create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.sql create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.reference create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.sql create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.reference create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.sql diff --git a/docs/en/sql-reference/data-types/dynamic.md b/docs/en/sql-reference/data-types/dynamic.md index 8be81471377..f9befd166fe 100644 --- a/docs/en/sql-reference/data-types/dynamic.md +++ b/docs/en/sql-reference/data-types/dynamic.md @@ -14,7 +14,7 @@ To declare a column of `Dynamic` type, use the following syntax: Dynamic(max_types=N) ``` -Where `N` is an optional parameter between `1` and `255` indicating how many different data types can be stored inside a column with type `Dynamic` across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all new types will be converted to type `String`. Default value of `max_types` is `32`. +Where `N` is an optional parameter between `0` and `254` indicating how many different data types can be stored as separate subcolumns inside a column with type `Dynamic` across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all values with new types will be stored together in a special shared data structure in binary form. Default value of `max_types` is `32`. :::note The Dynamic data type is an experimental feature. To use it, set `allow_experimental_dynamic_type = 1`. @@ -224,41 +224,43 @@ SELECT d::Dynamic(max_types=5) as d2, dynamicType(d2) FROM test; └───────┴────────────────┘ ``` -If `K < N`, then the values with the rarest types are converted to `String`: +If `K < N`, then the values with the rarest types will be inserted into a single special subcolumn, but still will be accessible: ```text CREATE TABLE test (d Dynamic(max_types=4)) ENGINE = Memory; INSERT INTO test VALUES (NULL), (42), (43), ('42.42'), (true), ([1, 2, 3]); -SELECT d, dynamicType(d), d::Dynamic(max_types=2) as d2, dynamicType(d2) FROM test; +SELECT d, dynamicType(d), d::Dynamic(max_types=2) as d2, dynamicType(d2), isDynamicElementInSharedData(d2) FROM test; ``` ```text -┌─d───────┬─dynamicType(d)─┬─d2──────┬─dynamicType(d2)─┐ -│ ᴺᵁᴸᴸ │ None │ ᴺᵁᴸᴸ │ None │ -│ 42 │ Int64 │ 42 │ Int64 │ -│ 43 │ Int64 │ 43 │ Int64 │ -│ 42.42 │ String │ 42.42 │ String │ -│ true │ Bool │ true │ String │ -│ [1,2,3] │ Array(Int64) │ [1,2,3] │ String │ -└─────────┴────────────────┴─────────┴─────────────────┘ +┌─d───────┬─dynamicType(d)─┬─d2──────┬─dynamicType(d2)─┬─isDynamicElementInSharedData(d2)─┐ +│ ᴺᵁᴸᴸ │ None │ ᴺᵁᴸᴸ │ None │ false │ +│ 42 │ Int64 │ 42 │ Int64 │ false │ +│ 43 │ Int64 │ 43 │ Int64 │ false │ +│ 42.42 │ String │ 42.42 │ String │ false │ +│ true │ Bool │ true │ Bool │ true │ +│ [1,2,3] │ Array(Int64) │ [1,2,3] │ Array(Int64) │ true │ +└─────────┴────────────────┴─────────┴─────────────────┴──────────────────────────────────┘ ``` -If `K=1`, all types are converted to `String`: +Functions `isDynamicElementInSharedData` returns `true` for rows that are stored in a special shared data structure inside `Dynamic` and as we can see, resulting column contains only 2 types that are not stored in shared data structure. + +If `K=0`, all types will be inserted into single special subcolumn: ```text CREATE TABLE test (d Dynamic(max_types=4)) ENGINE = Memory; INSERT INTO test VALUES (NULL), (42), (43), ('42.42'), (true), ([1, 2, 3]); -SELECT d, dynamicType(d), d::Dynamic(max_types=1) as d2, dynamicType(d2) FROM test; +SELECT d, dynamicType(d), d::Dynamic(max_types=0) as d2, dynamicType(d2), isDynamicElementInSharedData(d2) FROM test; ``` ```text -┌─d───────┬─dynamicType(d)─┬─d2──────┬─dynamicType(d2)─┐ -│ ᴺᵁᴸᴸ │ None │ ᴺᵁᴸᴸ │ None │ -│ 42 │ Int64 │ 42 │ String │ -│ 43 │ Int64 │ 43 │ String │ -│ 42.42 │ String │ 42.42 │ String │ -│ true │ Bool │ true │ String │ -│ [1,2,3] │ Array(Int64) │ [1,2,3] │ String │ -└─────────┴────────────────┴─────────┴─────────────────┘ +┌─d───────┬─dynamicType(d)─┬─d2──────┬─dynamicType(d2)─┬─isDynamicElementInSharedData(d2)─┐ +│ ᴺᵁᴸᴸ │ None │ ᴺᵁᴸᴸ │ None │ false │ +│ 42 │ Int64 │ 42 │ Int64 │ true │ +│ 43 │ Int64 │ 43 │ Int64 │ true │ +│ 42.42 │ String │ 42.42 │ String │ true │ +│ true │ Bool │ true │ Bool │ true │ +│ [1,2,3] │ Array(Int64) │ [1,2,3] │ Array(Int64) │ true │ +└─────────┴────────────────┴─────────┴─────────────────┴──────────────────────────────────┘ ``` ## Reading Dynamic type from the data @@ -411,17 +413,17 @@ SELECT d, dynamicType(d) FROM test ORDER by d; ## Reaching the limit in number of different data types stored inside Dynamic -`Dynamic` data type can store only limited number of different data types inside. By default, this limit is 32, but you can change it in type declaration using syntax `Dynamic(max_types=N)` where N is between 1 and 255 (due to implementation details, it's impossible to have more than 255 different data types inside Dynamic). -When the limit is reached, all new data types inserted to `Dynamic` column will be casted to `String` and stored as `String` values. +`Dynamic` data type can store only limited number of different data types as separate subcolumns. By default, this limit is 32, but you can change it in type declaration using syntax `Dynamic(max_types=N)` where N is between 0 and 254 (due to implementation details, it's impossible to have more than 254 different data types that can be stored as separate subcolumns inside Dynamic). +When the limit is reached, all new data types inserted to `Dynamic` column will be inserted into a single shared data structure that stores values with different data types in binary form. Let's see what happens when the limit is reached in different scenarios. ### Reaching the limit during data parsing -During parsing of `Dynamic` values from the data, when the limit is reached for current block of data, all new values will be inserted as `String` values: +During parsing of `Dynamic` values from the data, when the limit is reached for current block of data, all new values will be inserted into shared data structure: ```sql -SELECT d, dynamicType(d) FROM format(JSONEachRow, 'd Dynamic(max_types=3)', ' +SELECT d, dynamicType(d), isDynamicElementInSharedData(d) FROM format(JSONEachRow, 'd Dynamic(max_types=3)', ' {"d" : 42} {"d" : [1, 2, 3]} {"d" : "Hello, World!"} @@ -432,22 +434,22 @@ SELECT d, dynamicType(d) FROM format(JSONEachRow, 'd Dynamic(max_types=3)', ' ``` ```text -┌─d──────────────────────────┬─dynamicType(d)─┐ -│ 42 │ Int64 │ -│ [1,2,3] │ Array(Int64) │ -│ Hello, World! │ String │ -│ 2020-01-01 │ String │ -│ ["str1", "str2", "str3"] │ String │ -│ {"a" : 1, "b" : [1, 2, 3]} │ String │ -└────────────────────────────┴────────────────┘ +┌─d──────────────────────┬─dynamicType(d)─────────────────┬─isDynamicElementInSharedData(d)─┐ +│ 42 │ Int64 │ false │ +│ [1,2,3] │ Array(Int64) │ false │ +│ Hello, World! │ String │ false │ +│ 2020-01-01 │ Date │ true │ +│ ['str1','str2','str3'] │ Array(String) │ true │ +│ (1,[1,2,3]) │ Tuple(a Int64, b Array(Int64)) │ true │ +└────────────────────────┴────────────────────────────────┴─────────────────────────────────┘ ``` -As we can see, after inserting 3 different data types `Int64`, `Array(Int64)` and `String` all new types were converted to `String`. +As we can see, after inserting 3 different data types `Int64`, `Array(Int64)` and `String` all new types were inserted into special shared data structure. ### During merges of data parts in MergeTree table engines -During merge of several data parts in MergeTree table the `Dynamic` column in the resulting data part can reach the limit of different data types inside and won't be able to store all types from source parts. -In this case ClickHouse chooses what types will remain after merge and what types will be casted to `String`. In most cases ClickHouse tries to keep the most frequent types and cast the rarest types to `String`, but it depends on the implementation. +During merge of several data parts in MergeTree table the `Dynamic` column in the resulting data part can reach the limit of different data types that can be stored in separate subcolumns inside and won't be able to store all types as subcolumns from source parts. +In this case ClickHouse chooses what types will remain as separate subcolumns after merge and what types will be inserted into shared data structure. In most cases ClickHouse tries to keep the most frequent types and store the rarest types in shared data structure, but it depends on the implementation. Let's see an example of such merge. First, let's create a table with `Dynamic` column, set the limit of different data types to `3` and insert values with `5` different types: @@ -463,17 +465,17 @@ INSERT INTO test SELECT number, 'str_' || toString(number) FROM numbers(1); Each insert will create a separate data pert with `Dynamic` column containing single type: ```sql -SELECT count(), dynamicType(d), _part FROM test GROUP BY _part, dynamicType(d) ORDER BY _part; +SELECT count(), dynamicType(d), isDynamicElementInSharedData(d), _part FROM test GROUP BY _part, dynamicType(d), isDynamicElementInSharedData(d) ORDER BY _part, count(); ``` ```text -┌─count()─┬─dynamicType(d)──────┬─_part─────┐ -│ 5 │ UInt64 │ all_1_1_0 │ -│ 4 │ Array(UInt64) │ all_2_2_0 │ -│ 3 │ Date │ all_3_3_0 │ -│ 2 │ Map(UInt64, UInt64) │ all_4_4_0 │ -│ 1 │ String │ all_5_5_0 │ -└─────────┴─────────────────────┴───────────┘ +┌─count()─┬─dynamicType(d)──────┬─isDynamicElementInSharedData(d)─┬─_part─────┐ +│ 5 │ UInt64 │ false │ all_1_1_0 │ +│ 4 │ Array(UInt64) │ false │ all_2_2_0 │ +│ 3 │ Date │ false │ all_3_3_0 │ +│ 2 │ Map(UInt64, UInt64) │ false │ all_4_4_0 │ +│ 1 │ String │ false │ all_5_5_0 │ +└─────────┴─────────────────────┴─────────────────────────────────┴───────────┘ ``` Now, let's merge all parts into one and see what will happen: @@ -481,18 +483,20 @@ Now, let's merge all parts into one and see what will happen: ```sql SYSTEM START MERGES test; OPTIMIZE TABLE test FINAL; -SELECT count(), dynamicType(d), _part FROM test GROUP BY _part, dynamicType(d) ORDER BY _part; +SELECT count(), dynamicType(d), isDynamicElementInSharedData(d), _part FROM test GROUP BY _part, dynamicType(d), isDynamicElementInSharedData(d) ORDER BY _part, count() desc; ``` ```text -┌─count()─┬─dynamicType(d)─┬─_part─────┐ -│ 5 │ UInt64 │ all_1_5_2 │ -│ 6 │ String │ all_1_5_2 │ -│ 4 │ Array(UInt64) │ all_1_5_2 │ -└─────────┴────────────────┴───────────┘ +┌─count()─┬─dynamicType(d)──────┬─isDynamicElementInSharedData(d)─┬─_part─────┐ +│ 5 │ UInt64 │ false │ all_1_5_2 │ +│ 4 │ Array(UInt64) │ false │ all_1_5_2 │ +│ 3 │ Date │ false │ all_1_5_2 │ +│ 2 │ Map(UInt64, UInt64) │ true │ all_1_5_2 │ +│ 1 │ String │ true │ all_1_5_2 │ +└─────────┴─────────────────────┴─────────────────────────────────┴───────────┘ ``` -As we can see, ClickHouse kept the most frequent types `UInt64` and `Array(UInt64)` and casted all other types to `String`. +As we can see, ClickHouse kept the most frequent types `UInt64` and `Array(UInt64)` as subcolumns and inserted all other types into shared data. ## JSONExtract functions with Dynamic @@ -509,22 +513,23 @@ SELECT JSONExtract('{"a" : [1, 2, 3]}', 'a', 'Dynamic') AS dynamic, dynamicType( ``` ```sql -SELECT JSONExtract('{"obj" : {"a" : 42, "b" : "Hello", "c" : [1,2,3]}}', 'obj', 'Map(String, Variant(UInt32, String, Array(UInt32)))') AS map_of_dynamics, mapApply((k, v) -> (k, variantType(v)), map_of_dynamics) AS map_of_dynamic_types``` +SELECT JSONExtract('{"obj" : {"a" : 42, "b" : "Hello", "c" : [1,2,3]}}', 'obj', 'Map(String, Dynamic)') AS map_of_dynamics, mapApply((k, v) -> (k, dynamicType(v)), map_of_dynamics) AS map_of_dynamic_types +``` ```text -┌─map_of_dynamics──────────────────┬─map_of_dynamic_types────────────────────────────┐ -│ {'a':42,'b':'Hello','c':[1,2,3]} │ {'a':'UInt32','b':'String','c':'Array(UInt32)'} │ -└──────────────────────────────────┴─────────────────────────────────────────────────┘ +┌─map_of_dynamics──────────────────┬─map_of_dynamic_types────────────────────────────────────┐ +│ {'a':42,'b':'Hello','c':[1,2,3]} │ {'a':'Int64','b':'String','c':'Array(Nullable(Int64))'} │ +└──────────────────────────────────┴─────────────────────────────────────────────────────────┘ ``` ```sql -SELECT JSONExtractKeysAndValues('{"a" : 42, "b" : "Hello", "c" : [1,2,3]}', 'Variant(UInt32, String, Array(UInt32))') AS dynamics, arrayMap(x -> (x.1, variantType(x.2)), dynamics) AS dynamic_types``` +SELECT JSONExtractKeysAndValues('{"a" : 42, "b" : "Hello", "c" : [1,2,3]}', 'Dynamic') AS dynamics, arrayMap(x -> (x.1, dynamicType(x.2)), dynamics) AS dynamic_types``` ``` ```text -┌─dynamics───────────────────────────────┬─dynamic_types─────────────────────────────────────────┐ -│ [('a',42),('b','Hello'),('c',[1,2,3])] │ [('a','UInt32'),('b','String'),('c','Array(UInt32)')] │ -└────────────────────────────────────────┴───────────────────────────────────────────────────────┘ +┌─dynamics───────────────────────────────┬─dynamic_types─────────────────────────────────────────────────┐ +│ [('a',42),('b','Hello'),('c',[1,2,3])] │ [('a','Int64'),('b','String'),('c','Array(Nullable(Int64))')] │ +└────────────────────────────────────────┴───────────────────────────────────────────────────────────────┘ ``` ### Binary output format diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index a92d54dd675..454f7956f48 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -1,16 +1,21 @@ #include #include +#include #include #include #include +#include #include #include #include #include #include #include -#include +#include +#include +#include +#include #include namespace DB @@ -22,31 +27,77 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; } - -ColumnDynamic::ColumnDynamic(size_t max_dynamic_types_) : max_dynamic_types(max_dynamic_types_) +namespace { - /// Create empty Variant. - variant_info.variant_type = std::make_shared(DataTypes{}); - variant_info.variant_name = variant_info.variant_type->getName(); - variant_column = variant_info.variant_type->createColumn(); + +/// Static default format settings to avoid creating it every time. +const FormatSettings & getFormatSettings() +{ + static const FormatSettings settings; + return settings; +} + +} + +/// Shared variant will contain String values but we cannot use usual String type +/// because we can have regular variant with type String. +/// To solve it, we use String type with custom name for shared variant. +DataTypePtr ColumnDynamic::getSharedVariantDataType() +{ + return DataTypeFactory::instance().getCustom("String", std::make_unique(std::make_unique(getSharedVariantTypeName()))); +} + +ColumnDynamic::ColumnDynamic(size_t max_dynamic_types_) : max_dynamic_types(max_dynamic_types_), global_max_dynamic_types(max_dynamic_types) +{ + /// Create Variant with shared variant. + setVariantType(std::make_shared(DataTypes{getSharedVariantDataType()})); } ColumnDynamic::ColumnDynamic( - MutableColumnPtr variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, const Statistics & statistics_) + MutableColumnPtr variant_column_, const DataTypePtr & variant_type_, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_) + : variant_column(std::move(variant_column_)) + , max_dynamic_types(max_dynamic_types_) + , global_max_dynamic_types(global_max_dynamic_types_) + , statistics(statistics_) +{ + createVariantInfo(variant_type_); +} + +ColumnDynamic::ColumnDynamic( + MutableColumnPtr variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_) : variant_column(std::move(variant_column_)) , variant_info(variant_info_) , max_dynamic_types(max_dynamic_types_) + , global_max_dynamic_types(global_max_dynamic_types_) , statistics(statistics_) { } -ColumnDynamic::MutablePtr ColumnDynamic::create(MutableColumnPtr variant_column, const DataTypePtr & variant_type, size_t max_dynamic_types_, const Statistics & statistics_) +void ColumnDynamic::setVariantType(const DataTypePtr & variant_type) +{ + if (variant_column && !empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Setting specific variant type is allowed only for empty dynamic column"); + + variant_column = variant_type->createColumn(); + createVariantInfo(variant_type); +} + +void ColumnDynamic::setMaxDynamicPaths(size_t max_dynamic_type_) +{ + if (variant_column && !empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Setting specific max_dynamic_type parameter is allowed only for empty dynamic column"); + + max_dynamic_types = max_dynamic_type_; +} + +void ColumnDynamic::createVariantInfo(const DataTypePtr & variant_type) { - VariantInfo variant_info; variant_info.variant_type = variant_type; variant_info.variant_name = variant_type->getName(); const auto & variants = assert_cast(*variant_type).getVariants(); + variant_info.variant_names.clear(); variant_info.variant_names.reserve(variants.size()); + variant_info.variant_name_to_discriminator.clear(); variant_info.variant_name_to_discriminator.reserve(variants.size()); for (ColumnVariant::Discriminator discr = 0; discr != variants.size(); ++discr) { @@ -54,30 +105,26 @@ ColumnDynamic::MutablePtr ColumnDynamic::create(MutableColumnPtr variant_column, variant_info.variant_name_to_discriminator[variant_name] = discr; } - return create(std::move(variant_column), variant_info, max_dynamic_types_, statistics_); + if (!variant_info.variant_name_to_discriminator.contains(getSharedVariantTypeName())) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Variant in Dynamic column doesn't contain shared variant"); } -bool ColumnDynamic::addNewVariant(const DB::DataTypePtr & new_variant) +bool ColumnDynamic::addNewVariant(const DataTypePtr & new_variant, const String & new_variant_name) { /// Check if we already have such variant. - if (variant_info.variant_name_to_discriminator.contains(new_variant->getName())) + if (variant_info.variant_name_to_discriminator.contains(new_variant_name)) return true; - /// Check if we reached maximum number of variants. - if (variant_info.variant_names.size() >= max_dynamic_types) + /// Check if we reached maximum number of variants (don't count shared variant). + if (variant_info.variant_names.size() - 1 == max_dynamic_types) { - /// ColumnDynamic can have max_dynamic_types number of variants only when it has String as a variant. - /// Otherwise we won't be able to cast new variants to Strings. - if (!variant_info.variant_name_to_discriminator.contains("String")) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Maximum number of variants reached, but no String variant exists"); + /// Dynamic column should always have shared variant. + if (!variant_info.variant_name_to_discriminator.contains(getSharedVariantTypeName())) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Maximum number of variants reached, but no shared variant exists"); return false; } - /// If we have (max_dynamic_types - 1) number of variants and don't have String variant, we can add only String variant. - if (variant_info.variant_names.size() == max_dynamic_types - 1 && new_variant->getName() != "String" && !variant_info.variant_name_to_discriminator.contains("String")) - return false; - const DataTypes & current_variants = assert_cast(*variant_info.variant_type).getVariants(); DataTypes all_variants = current_variants; all_variants.push_back(new_variant); @@ -86,21 +133,15 @@ bool ColumnDynamic::addNewVariant(const DB::DataTypePtr & new_variant) return true; } -void ColumnDynamic::addStringVariant() +void extendVariantColumn( + IColumn & variant_column, + const DataTypePtr & old_variant_type, + const DataTypePtr & new_variant_type, + std::unordered_map old_variant_name_to_discriminator) { - if (!addNewVariant(std::make_shared())) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add String variant to Dynamic column, it's a bug"); -} - -void ColumnDynamic::updateVariantInfoAndExpandVariantColumn(const DB::DataTypePtr & new_variant_type) -{ - const DataTypes & current_variants = assert_cast(variant_info.variant_type.get())->getVariants(); + const DataTypes & current_variants = assert_cast(old_variant_type.get())->getVariants(); const DataTypes & new_variants = assert_cast(new_variant_type.get())->getVariants(); - Names new_variant_names; - new_variant_names.reserve(new_variants.size()); - std::unordered_map new_variant_name_to_discriminator; - new_variant_name_to_discriminator.reserve(new_variants.size()); std::vector> new_variant_columns_and_discriminators_to_add; new_variant_columns_and_discriminators_to_add.reserve(new_variants.size() - current_variants.size()); std::vector current_to_new_discriminators; @@ -108,26 +149,26 @@ void ColumnDynamic::updateVariantInfoAndExpandVariantColumn(const DB::DataTypePt for (ColumnVariant::Discriminator discr = 0; discr != new_variants.size(); ++discr) { - const auto & name = new_variant_names.emplace_back(new_variants[discr]->getName()); - new_variant_name_to_discriminator[name] = discr; - - auto current_it = variant_info.variant_name_to_discriminator.find(name); - if (current_it == variant_info.variant_name_to_discriminator.end()) + auto current_it = old_variant_name_to_discriminator.find(new_variants[discr]->getName()); + if (current_it == old_variant_name_to_discriminator.end()) new_variant_columns_and_discriminators_to_add.emplace_back(new_variants[discr]->createColumn(), discr); else current_to_new_discriminators[current_it->second] = discr; } - variant_info.variant_type = new_variant_type; - variant_info.variant_name = new_variant_type->getName(); - variant_info.variant_names = new_variant_names; - variant_info.variant_name_to_discriminator = new_variant_name_to_discriminator; - assert_cast(*variant_column).extend(current_to_new_discriminators, std::move(new_variant_columns_and_discriminators_to_add)); + assert_cast(variant_column).extend(current_to_new_discriminators, std::move(new_variant_columns_and_discriminators_to_add)); +} + +void ColumnDynamic::updateVariantInfoAndExpandVariantColumn(const DataTypePtr & new_variant_type) +{ + extendVariantColumn(*variant_column, variant_info.variant_type, new_variant_type, variant_info.variant_name_to_discriminator); + createVariantInfo(new_variant_type); + /// Clear mappings cache because now with new Variant we will have new mappings. variant_mappings_cache.clear(); } -std::vector * ColumnDynamic::combineVariants(const DB::ColumnDynamic::VariantInfo & other_variant_info) +std::vector * ColumnDynamic::combineVariants(const ColumnDynamic::VariantInfo & other_variant_info) { /// Check if we already have global discriminators mapping for other Variant in cache. /// It's used to not calculate the same mapping each call of insertFrom with the same columns. @@ -153,21 +194,14 @@ std::vector * ColumnDynamic::combineVariants(const { const DataTypes & current_variants = assert_cast(*variant_info.variant_type).getVariants(); - /// We cannot combine Variants if total number of variants exceeds max_dynamic_types. - if (current_variants.size() + num_new_variants > max_dynamic_types) + /// We cannot combine Variants if total number of variants exceeds max_dynamic_types (don't count shared variant). + if (current_variants.size() + num_new_variants - 1 > max_dynamic_types) { /// Remember that we cannot combine our variant with this one, so we will not try to do it again. variants_with_failed_combination.insert(other_variant_info.variant_name); return nullptr; } - /// We cannot combine Variants if total number of variants reaches max_dynamic_types and we don't have String variant. - if (current_variants.size() + num_new_variants == max_dynamic_types && !variant_info.variant_name_to_discriminator.contains("String") && !other_variant_info.variant_name_to_discriminator.contains("String")) - { - variants_with_failed_combination.insert(other_variant_info.variant_name); - return nullptr; - } - DataTypes all_variants = current_variants; all_variants.insert(all_variants.end(), other_variants.begin(), other_variants.end()); auto new_variant_type = std::make_shared(all_variants); @@ -185,40 +219,93 @@ std::vector * ColumnDynamic::combineVariants(const return &it->second; } -void ColumnDynamic::insert(const DB::Field & x) +void ColumnDynamic::insert(const Field & x) { - /// Check if we can insert field without Variant extension. - if (variant_column->tryInsert(x)) + if (x.isNull()) + { + insertDefault(); return; + } + + auto & variant_col = getVariantColumn(); + auto shared_variant_discr = getSharedVariantDiscriminator(); + /// Check if we can insert field into existing variants and avoid Variant extension. + for (size_t i = 0; i != variant_col.getNumVariants(); ++i) + { + if (i != shared_variant_discr && variant_col.getVariantByGlobalDiscriminator(i).tryInsert(x)) + { + variant_col.getLocalDiscriminators().push_back(variant_col.localDiscriminatorByGlobal(i)); + variant_col.getOffsets().push_back(variant_col.getVariantByGlobalDiscriminator(i).size() - 1); + return; + } + } /// If we cannot insert field into current variant column, extend it with new variant for this field from its type. - if (addNewVariant(applyVisitor(FieldToDataType(), x))) + auto field_data_type = applyVisitor(FieldToDataType(), x); + auto field_data_type_name = field_data_type->getName(); + if (addNewVariant(field_data_type, field_data_type_name)) { - /// Now we should be able to insert this field into extended variant column. - variant_column->insert(x); + /// Insert this field into newly added variant. + auto discr = variant_info.variant_name_to_discriminator[field_data_type_name]; + variant_col.getVariantByGlobalDiscriminator(discr).insert(x); + variant_col.getLocalDiscriminators().push_back(variant_col.localDiscriminatorByGlobal(discr)); + variant_col.getOffsets().push_back(variant_col.getVariantByGlobalDiscriminator(discr).size() - 1); } else { /// We reached maximum number of variants and couldn't add new variant. - /// This case should be really rare in real use cases. - /// We should always be able to add String variant and cast inserted value to String. - addStringVariant(); - variant_column->insert(toString(x)); + /// In this case we add the value of this new variant into special shared variant. + /// We store values in shared variant in binary form with binary encoded type. + auto & shared_variant = getSharedVariant(); + auto & chars = shared_variant.getChars(); + WriteBufferFromVector value_buf(chars, AppendModeTag()); + encodeDataType(field_data_type, value_buf); + getVariantSerialization(field_data_type, field_data_type_name)->serializeBinary(x, value_buf, getFormatSettings()); + value_buf.finalize(); + chars.push_back(0); + shared_variant.getOffsets().push_back(chars.size()); + variant_col.getLocalDiscriminators().push_back(variant_col.localDiscriminatorByGlobal(shared_variant_discr)); + variant_col.getOffsets().push_back(shared_variant.size() - 1); } } -bool ColumnDynamic::tryInsert(const DB::Field & x) +bool ColumnDynamic::tryInsert(const Field & x) { /// We can insert any value into Dynamic column. insert(x); return true; } +Field ColumnDynamic::operator[](size_t n) const +{ + Field res; + get(n, res); + return res; +} + +void ColumnDynamic::get(size_t n, Field & res) const +{ + const auto & variant_col = getVariantColumn(); + /// Check if value is not in shared variant. + if (variant_col.globalDiscriminatorAt(n) != getSharedVariantDiscriminator()) + { + variant_col.get(n, res); + return; + } + + /// We should deeserialize value from shared variant. + const auto & shared_variant = getSharedVariant(); + auto value_data = shared_variant.getDataAt(variant_col.offsetAt(n)); + ReadBufferFromMemory buf(value_data.data, value_data.size); + auto type = decodeDataType(buf); + getVariantSerialization(type)->deserializeBinary(res, buf, getFormatSettings()); +} + #if !defined(DEBUG_OR_SANITIZER_BUILD) -void ColumnDynamic::insertFrom(const DB::IColumn & src_, size_t n) +void ColumnDynamic::insertFrom(const IColumn & src_, size_t n) #else -void ColumnDynamic::doInsertFrom(const DB::IColumn & src_, size_t n) +void ColumnDynamic::doInsertFrom(const IColumn & src_, size_t n) #endif { const auto & dynamic_src = assert_cast(src_); @@ -231,6 +318,28 @@ void ColumnDynamic::doInsertFrom(const DB::IColumn & src_, size_t n) } auto & variant_col = assert_cast(*variant_column); + const auto & src_variant_col = assert_cast(*dynamic_src.variant_column); + auto src_global_discr = src_variant_col.globalDiscriminatorAt(n); + auto src_offset = src_variant_col.offsetAt(n); + + /// Check if we insert from shared variant and process it separately. + if (src_global_discr == dynamic_src.getSharedVariantDiscriminator()) + { + auto & src_shared_variant = dynamic_src.getSharedVariant(); + auto value = src_shared_variant.getDataAt(src_offset); + /// Decode data type of this value. + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + auto type_name = type->getName(); + /// Check if we have this variant and deserialize value into variant from shared variant data. + if (auto it = variant_info.variant_name_to_discriminator.find(type_name); it != variant_info.variant_name_to_discriminator.end()) + variant_col.deserializeBinaryIntoVariant(it->second, getVariantSerialization(type, type_name), buf, getFormatSettings()); + /// Otherwise just insert it into our shared variant. + else + variant_col.insertIntoVariantFrom(getSharedVariantDiscriminator(), src_shared_variant, src_offset); + + return; + } /// If variants are different, we need to extend our variant with new variants. if (auto * global_discriminators_mapping = combineVariants(dynamic_src.variant_info)) @@ -241,8 +350,6 @@ void ColumnDynamic::doInsertFrom(const DB::IColumn & src_, size_t n) /// We cannot combine 2 Variant types as total number of variants exceeds the limit. /// We need to insert single value, try to add only corresponding variant. - const auto & src_variant_col = assert_cast(*dynamic_src.variant_column); - auto src_global_discr = src_variant_col.globalDiscriminatorAt(n); /// NULL doesn't require Variant extension. if (src_global_discr == ColumnVariant::NULL_DISCRIMINATOR) @@ -260,19 +367,18 @@ void ColumnDynamic::doInsertFrom(const DB::IColumn & src_, size_t n) } /// We reached maximum number of variants and couldn't add new variant. - /// We should always be able to add String variant and cast inserted value to String. - addStringVariant(); - auto tmp_variant_column = src_variant_col.getVariantByGlobalDiscriminator(src_global_discr).cloneEmpty(); - tmp_variant_column->insertFrom(src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), src_variant_col.offsetAt(n)); - auto tmp_string_column = castColumn(ColumnWithTypeAndName(tmp_variant_column->getPtr(), variant_type, ""), std::make_shared()); - auto string_variant_discr = variant_info.variant_name_to_discriminator["String"]; - variant_col.insertIntoVariantFrom(string_variant_discr, *tmp_string_column, 0); + /// Insert this value into shared variant. + insertValueIntoSharedVariant( + src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), + variant_type, + dynamic_src.variant_info.variant_names[src_global_discr], + src_offset); } #if !defined(DEBUG_OR_SANITIZER_BUILD) -void ColumnDynamic::insertRangeFrom(const DB::IColumn & src_, size_t start, size_t length) +void ColumnDynamic::insertRangeFrom(const IColumn & src_, size_t start, size_t length) #else -void ColumnDynamic::doInsertRangeFrom(const DB::IColumn & src_, size_t start, size_t length) +void ColumnDynamic::doInsertRangeFrom(const IColumn & src_, size_t start, size_t length) #endif { if (start + length > src_.size()) @@ -293,156 +399,206 @@ void ColumnDynamic::doInsertRangeFrom(const DB::IColumn & src_, size_t start, si /// If variants are different, we need to extend our variant with new variants. if (auto * global_discriminators_mapping = combineVariants(dynamic_src.variant_info)) { - variant_col.insertRangeFrom(*dynamic_src.variant_column, start, length, *global_discriminators_mapping); + size_t prev_size = variant_col.size(); + auto shared_variant_discr = getSharedVariantDiscriminator(); + variant_col.insertRangeFrom(*dynamic_src.variant_column, start, length, *global_discriminators_mapping, shared_variant_discr); + + /// We should process insertion from srs shared variant separately, because it can contain + /// values that should be extracted into our variants. insertRangeFrom above didn't insert + /// values into our shared variant (we specified shared_variant_discr as special skip discriminator). + + /// Check if srs shared variant is empty, nothing to do in this case. + if (dynamic_src.getSharedVariant().empty()) + return; + + /// Iterate over src discriminators and process insertion from src shared variant. + const auto & src_variant_column = dynamic_src.getVariantColumn(); + const auto src_shared_variant_discr = dynamic_src.getSharedVariantDiscriminator(); + const auto src_shared_variant_local_discr = src_variant_column.localDiscriminatorByGlobal(src_shared_variant_discr); + const auto & src_local_discriminators = src_variant_column.getLocalDiscriminators(); + const auto & src_offsets = src_variant_column.getOffsets(); + const auto & src_shared_variant = assert_cast(src_variant_column.getVariantByLocalDiscriminator(src_shared_variant_local_discr)); + + auto & local_discriminators = variant_col.getLocalDiscriminators(); + auto & offsets = variant_col.getOffsets(); + const auto shared_variant_local_discr = variant_col.localDiscriminatorByGlobal(shared_variant_discr); + auto & shared_variant = assert_cast(variant_col.getVariantByLocalDiscriminator(shared_variant_local_discr)); + for (size_t i = 0; i != length; ++i) + { + if (src_local_discriminators[start + i] == src_shared_variant_local_discr) + { + chassert(local_discriminators[prev_size + i] == shared_variant_local_discr); + auto value = src_shared_variant.getDataAt(src_offsets[start + i]); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + auto type_name = type->getName(); + /// Check if we have variant with this type. In this case we should extract + /// the value from src shared variant and insert it into this variant. + if (auto it = variant_info.variant_name_to_discriminator.find(type_name); it != variant_info.variant_name_to_discriminator.end()) + { + auto local_discr = variant_col.localDiscriminatorByGlobal(it->second); + auto & variant = variant_col.getVariantByLocalDiscriminator(local_discr); + getVariantSerialization(type, type_name)->deserializeBinary(variant, buf, getFormatSettings()); + /// Local discriminators were already filled in ColumnVariant::insertRangeFrom and this row should contain + /// shared_variant_local_discr. Change it to local discriminator of the found variant and update offsets. + local_discriminators[prev_size + i] = local_discr; + offsets[prev_size + i] = variant.size() - 1; + } + /// Otherwise, insert this value into shared variant. + else + { + shared_variant.insertData(value.data, value.size); + /// Update variant offset. + offsets[prev_size + i] = shared_variant.size() - 1; + } + } + } + return; } /// We cannot combine 2 Variant types as total number of variants exceeds the limit. - /// In this case we will add most frequent variants from this range and insert them as usual, - /// all other variants will be converted to String. - /// TODO: instead of keeping all current variants and just adding new most frequent variants - /// from source columns we can also try to replace rarest existing variants with frequent - /// variants from source column (so we will avoid casting new frequent variants to String - /// and keeping rare existing ones). It will require rewriting of existing data in Variant - /// column but will improve usability of Dynamic column for example during squashing blocks - /// during insert. - - const auto & src_variant_column = dynamic_src.getVariantColumn(); - - /// Calculate ranges for each variant in current range. - std::vector> variants_ranges(dynamic_src.variant_info.variant_names.size(), {0, 0}); - /// If we insert the whole column, no need to iterate through the range, we can just take variant sizes. - if (start == 0 && length == dynamic_src.size()) - { - for (size_t i = 0; i != dynamic_src.variant_info.variant_names.size(); ++i) - variants_ranges[i] = {0, src_variant_column.getVariantByGlobalDiscriminator(i).size()}; - } - /// Otherwise we need to iterate through discriminators and calculate the range for each variant. - else - { - const auto & local_discriminators = src_variant_column.getLocalDiscriminators(); - const auto & offsets = src_variant_column.getOffsets(); - size_t end = start + length; - for (size_t i = start; i != end; ++i) - { - auto discr = src_variant_column.globalDiscriminatorByLocal(local_discriminators[i]); - if (discr != ColumnVariant::NULL_DISCRIMINATOR) - { - if (!variants_ranges[discr].second) - variants_ranges[discr].first = offsets[i]; - ++variants_ranges[discr].second; - } - } - } - + /// In this case we will add most frequent variants and insert them as usual, + /// all other variants will be inserted into shared variant. const auto & src_variants = assert_cast(*dynamic_src.variant_info.variant_type).getVariants(); - /// List of variants that will be converted to String. - std::vector variants_to_convert_to_string; /// Mapping from global discriminators of src_variant to the new variant we will create. std::vector other_to_new_discriminators; other_to_new_discriminators.reserve(dynamic_src.variant_info.variant_names.size()); - /// Check if we cannot add any more new variants. In this case we will convert all new variants to String. - if (variant_info.variant_names.size() == max_dynamic_types || (variant_info.variant_names.size() == max_dynamic_types - 1 && !variant_info.variant_name_to_discriminator.contains("String"))) + /// Check if we cannot add any more new variants. In this case we will insert all new variants into shared variant. + if (variant_info.variant_names.size() - 1 == max_dynamic_types) { - addStringVariant(); + auto shared_variant_discr = getSharedVariantDiscriminator(); for (size_t i = 0; i != dynamic_src.variant_info.variant_names.size(); ++i) { auto it = variant_info.variant_name_to_discriminator.find(dynamic_src.variant_info.variant_names[i]); if (it == variant_info.variant_name_to_discriminator.end()) - { - variants_to_convert_to_string.push_back(i); - other_to_new_discriminators.push_back(variant_info.variant_name_to_discriminator["String"]); - } + other_to_new_discriminators.push_back(shared_variant_discr); else - { other_to_new_discriminators.push_back(it->second); - } } } - /// We still can add some new variants, but not all of them. Let's choose the most frequent variants in specified range. + /// We still can add some new variants, but not all of them. Let's choose the most frequent variants. else { + /// Create list of pairs and sort it. std::vector> new_variants_with_sizes; new_variants_with_sizes.reserve(dynamic_src.variant_info.variant_names.size()); - for (size_t i = 0; i != dynamic_src.variant_info.variant_names.size(); ++i) + const auto & src_variant_column = dynamic_src.getVariantColumn(); + for (const auto & [name, discr] : dynamic_src.variant_info.variant_name_to_discriminator) { - const auto & variant_name = dynamic_src.variant_info.variant_names[i]; - if (variant_name != "String" && !variant_info.variant_name_to_discriminator.contains(variant_name)) - new_variants_with_sizes.emplace_back(variants_ranges[i].second, i); + if (!variant_info.variant_name_to_discriminator.contains(name)) + new_variants_with_sizes.emplace_back(src_variant_column.getVariantByGlobalDiscriminator(discr).size(), discr); } std::sort(new_variants_with_sizes.begin(), new_variants_with_sizes.end(), std::greater()); DataTypes new_variants = assert_cast(*variant_info.variant_type).getVariants(); - if (!variant_info.variant_name_to_discriminator.contains("String")) - new_variants.push_back(std::make_shared()); - + /// Add new variants from sorted list until we reach max_dynamic_types. for (const auto & [_, discr] : new_variants_with_sizes) { - if (new_variants.size() != max_dynamic_types) - new_variants.push_back(src_variants[discr]); - else - variants_to_convert_to_string.push_back(discr); + if (new_variants.size() - 1 == max_dynamic_types) + break; + new_variants.push_back(src_variants[discr]); } auto new_variant_type = std::make_shared(new_variants); updateVariantInfoAndExpandVariantColumn(new_variant_type); - auto string_variant_discriminator = variant_info.variant_name_to_discriminator.at("String"); + auto shared_variant_discr = getSharedVariantDiscriminator(); for (const auto & variant_name : dynamic_src.variant_info.variant_names) { auto it = variant_info.variant_name_to_discriminator.find(variant_name); if (it == variant_info.variant_name_to_discriminator.end()) - other_to_new_discriminators.push_back(string_variant_discriminator); + other_to_new_discriminators.push_back(shared_variant_discr); else other_to_new_discriminators.push_back(it->second); } } - /// Convert to String all variants that couldn't be added. - std::unordered_map variants_converted_to_string; - variants_converted_to_string.reserve(variants_to_convert_to_string.size()); - for (auto discr : variants_to_convert_to_string) - { - auto [variant_start, variant_length] = variants_ranges[discr]; - const auto & variant = src_variant_column.getVariantPtrByGlobalDiscriminator(discr); - if (variant_start == 0 && variant_length == variant->size()) - variants_converted_to_string[discr] = castColumn(ColumnWithTypeAndName(variant, src_variants[discr], ""), std::make_shared()); - else - variants_converted_to_string[discr] = castColumn(ColumnWithTypeAndName(variant->cut(variant_start, variant_length), src_variants[discr], ""), std::make_shared()); - } - + /// Iterate over the range and perform insertion. + const auto & src_variant_column = dynamic_src.getVariantColumn(); const auto & src_local_discriminators = src_variant_column.getLocalDiscriminators(); const auto & src_offsets = src_variant_column.getOffsets(); const auto & src_variant_columns = src_variant_column.getVariants(); + const auto src_shared_variant_discr = dynamic_src.getSharedVariantDiscriminator(); + const auto src_shared_variant_local_discr = src_variant_column.localDiscriminatorByGlobal(src_shared_variant_discr); + const auto & src_shared_variant = assert_cast(*src_variant_columns[src_shared_variant_local_discr]); + auto & local_discriminators = variant_col.getLocalDiscriminators(); + local_discriminators.reserve(local_discriminators.size() + length); + auto & offsets = variant_col.getOffsets(); + offsets.reserve(offsets.size() + length); + auto & variant_columns = variant_col.getVariants(); + const auto shared_variant_discr = getSharedVariantDiscriminator(); + const auto shared_variant_local_discr = variant_col.localDiscriminatorByGlobal(shared_variant_discr); + auto & shared_variant = assert_cast(*variant_columns[shared_variant_local_discr]); size_t end = start + length; for (size_t i = start; i != end; ++i) { - auto local_discr = src_local_discriminators[i]; - if (local_discr == ColumnVariant::NULL_DISCRIMINATOR) + auto src_local_discr = src_local_discriminators[i]; + auto src_offset = src_offsets[i]; + if (src_local_discr == ColumnVariant::NULL_DISCRIMINATOR) { - variant_col.insertDefault(); + local_discriminators.push_back(ColumnVariant::NULL_DISCRIMINATOR); + offsets.emplace_back(); } else { - auto global_discr = src_variant_column.globalDiscriminatorByLocal(local_discr); - auto to_global_discr = other_to_new_discriminators[global_discr]; - auto it = variants_converted_to_string.find(global_discr); - if (it == variants_converted_to_string.end()) + /// Process insertion from src shared variant separately. + if (src_local_discr == src_shared_variant_local_discr) { - variant_col.insertIntoVariantFrom(to_global_discr, *src_variant_columns[local_discr], src_offsets[i]); + auto value = src_shared_variant.getDataAt(src_offset); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + auto type_name = type->getName(); + /// Check if we have variant with this type. In this case we should extract + /// the value from src shared variant and insert it into this variant. + if (auto it = variant_info.variant_name_to_discriminator.find(type_name); it != variant_info.variant_name_to_discriminator.end()) + { + auto local_discr = variant_col.localDiscriminatorByGlobal(it->second); + getVariantSerialization(type, type_name)->deserializeBinary(*variant_columns[local_discr], buf, getFormatSettings()); + local_discriminators.push_back(local_discr); + offsets.push_back(variant_columns[local_discr]->size() - 1); + } + /// Otherwise, insert this value into shared variant. + else + { + shared_variant.insertData(value.data, value.size); + local_discriminators.push_back(shared_variant_local_discr); + offsets.push_back(shared_variant.size() - 1); + } } + /// Insertion from usual variant. else { - variant_col.insertIntoVariantFrom(to_global_discr, *it->second, src_offsets[i] - variants_ranges[global_discr].first); + auto src_global_discr = src_variant_column.globalDiscriminatorByLocal(src_local_discr); + auto global_discr = other_to_new_discriminators[src_global_discr]; + /// Check if we need to insert this value into shared variant. + if (global_discr == shared_variant_discr) + { + serializeValueIntoSharedVariant( + shared_variant, + *src_variant_columns[src_local_discr], + src_variants[src_global_discr], + getVariantSerialization(src_variants[src_global_discr], dynamic_src.variant_info.variant_names[src_global_discr]), + src_offset); + local_discriminators.push_back(shared_variant_local_discr); + offsets.push_back(shared_variant.size() - 1); + } + else + { + auto local_discr = variant_col.localDiscriminatorByGlobal(global_discr); + variant_columns[local_discr]->insertFrom(*src_variant_columns[src_local_discr], src_offset); + local_discriminators.push_back(local_discr); + offsets.push_back(variant_columns[local_discr]->size() - 1); + } } } } } #if !defined(DEBUG_OR_SANITIZER_BUILD) -void ColumnDynamic::insertManyFrom(const DB::IColumn & src_, size_t position, size_t length) +void ColumnDynamic::insertManyFrom(const IColumn & src_, size_t position, size_t length) #else -void ColumnDynamic::doInsertManyFrom(const DB::IColumn & src_, size_t position, size_t length) +void ColumnDynamic::doInsertManyFrom(const IColumn & src_, size_t position, size_t length) #endif { const auto & dynamic_src = assert_cast(src_); @@ -455,6 +611,36 @@ void ColumnDynamic::doInsertManyFrom(const DB::IColumn & src_, size_t position, } auto & variant_col = assert_cast(*variant_column); + const auto & src_variant_col = assert_cast(*dynamic_src.variant_column); + auto src_global_discr = src_variant_col.globalDiscriminatorAt(position); + auto src_offset = src_variant_col.offsetAt(position); + + /// Check if we insert from shared variant and process it separately. + if (src_global_discr == dynamic_src.getSharedVariantDiscriminator()) + { + auto & src_shared_variant = dynamic_src.getSharedVariant(); + auto value = src_shared_variant.getDataAt(src_offset); + /// Decode data type of this value. + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + auto type_name = type->getName(); + /// Check if we have this variant and deserialize value into variant from shared variant data. + if (auto it = variant_info.variant_name_to_discriminator.find(type_name); it != variant_info.variant_name_to_discriminator.end()) + { + /// Deserialize value into temporary column and use it in insertManyIntoVariantFrom. + auto tmp_column = type->createColumn(); + tmp_column->reserve(1); + getVariantSerialization(type, type_name)->deserializeBinary(*tmp_column, buf, getFormatSettings()); + variant_col.insertManyIntoVariantFrom(it->second, *tmp_column, 0, length); + } + /// Otherwise just insert it into our shared variant. + else + { + variant_col.insertManyIntoVariantFrom(getSharedVariantDiscriminator(), src_shared_variant, src_offset, length); + } + + return; + } /// If variants are different, we need to extend our variant with new variants. if (auto * global_discriminators_mapping = combineVariants(dynamic_src.variant_info)) @@ -465,8 +651,6 @@ void ColumnDynamic::doInsertManyFrom(const DB::IColumn & src_, size_t position, /// We cannot combine 2 Variant types as total number of variants exceeds the limit. /// We need to insert single value, try to add only corresponding variant. - const auto & src_variant_col = assert_cast(*dynamic_src.variant_column); - auto src_global_discr = src_variant_col.globalDiscriminatorAt(position); if (src_global_discr == ColumnVariant::NULL_DISCRIMINATOR) { insertDefault(); @@ -481,21 +665,51 @@ void ColumnDynamic::doInsertManyFrom(const DB::IColumn & src_, size_t position, return; } - addStringVariant(); - auto tmp_variant_column = src_variant_col.getVariantByGlobalDiscriminator(src_global_discr).cloneEmpty(); - tmp_variant_column->insertFrom(src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), src_variant_col.offsetAt(position)); - auto tmp_string_column = castColumn(ColumnWithTypeAndName(tmp_variant_column->getPtr(), variant_type, ""), std::make_shared()); - auto string_variant_discr = variant_info.variant_name_to_discriminator["String"]; - variant_col.insertManyIntoVariantFrom(string_variant_discr, *tmp_string_column, 0, length); + /// We reached maximum number of variants and couldn't add new variant. + /// Insert this value into shared variant. + /// Create temporary string column, serialize value into it and use it in insertManyIntoVariantFrom. + auto tmp_shared_variant = ColumnString::create(); + serializeValueIntoSharedVariant( + *tmp_shared_variant, + src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), + variant_type, + getVariantSerialization(variant_type, dynamic_src.variant_info.variant_names[src_global_discr]), + src_offset); + + variant_col.insertManyIntoVariantFrom(getSharedVariantDiscriminator(), *tmp_shared_variant, 0, length); } +void ColumnDynamic::insertValueIntoSharedVariant(const IColumn & src, const DataTypePtr & type, const String & type_name, size_t n) +{ + auto & variant_col = getVariantColumn(); + auto & shared_variant = getSharedVariant(); + serializeValueIntoSharedVariant(shared_variant, src, type, getVariantSerialization(type, type_name), n); + variant_col.getLocalDiscriminators().push_back(variant_col.localDiscriminatorByGlobal(getSharedVariantDiscriminator())); + variant_col.getOffsets().push_back(shared_variant.size() - 1); +} -StringRef ColumnDynamic::serializeValueIntoArena(size_t n, DB::Arena & arena, const char *& begin) const +void ColumnDynamic::serializeValueIntoSharedVariant( + ColumnString & shared_variant, + const IColumn & src, + const DataTypePtr & type, + const SerializationPtr & serialization, + size_t n) +{ + auto & chars = shared_variant.getChars(); + WriteBufferFromVector value_buf(chars, AppendModeTag()); + encodeDataType(type, value_buf); + serialization->serializeBinary(src, n, value_buf, getFormatSettings()); + value_buf.finalize(); + chars.push_back(0); + shared_variant.getOffsets().push_back(chars.size()); +} + +StringRef ColumnDynamic::serializeValueIntoArena(size_t n, Arena & arena, const char *& begin) const { /// We cannot use Variant serialization here as it serializes discriminator + value, /// but Dynamic doesn't have fixed mapping discriminator <-> variant type /// as different Dynamic column can have different Variants. - /// Instead, we serialize null bit + variant type in binary format (size + bytes) + value. + /// Instead, we serialize null bit + variant type and value in binary format (size + data). const auto & variant_col = assert_cast(*variant_column); auto discr = variant_col.globalDiscriminatorAt(n); StringRef res; @@ -509,19 +723,29 @@ StringRef ColumnDynamic::serializeValueIntoArena(size_t n, DB::Arena & arena, co return res; } - const auto & variant_type = assert_cast(*variant_info.variant_type).getVariant(discr); - String variant_type_binary_data = encodeDataType(variant_type); - size_t variant_type_binary_data_size = variant_type_binary_data.size(); - char * pos = arena.allocContinue(sizeof(UInt8) + sizeof(size_t) + variant_type_binary_data.size(), begin); - memcpy(pos, &null_bit, sizeof(UInt8)); - memcpy(pos + sizeof(UInt8), &variant_type_binary_data_size, sizeof(size_t)); - memcpy(pos + sizeof(UInt8) + sizeof(size_t), variant_type_binary_data.data(), variant_type_binary_data.size()); - res.data = pos; - res.size = sizeof(UInt8) + sizeof(size_t) + variant_type_binary_data.size(); + WriteBufferFromOwnString buf; + StringRef type_and_value; + /// If we have value from shared variant, it's already stored in the desired format. + if (discr == getSharedVariantDiscriminator()) + { + type_and_value = getSharedVariant().getDataAt(variant_col.offsetAt(n)); + } + /// For regular variants serialize its type and value in binary format. + else + { + const auto & variant_type = assert_cast(*variant_info.variant_type).getVariant(discr); + encodeDataType(variant_type, buf); + getVariantSerialization(variant_type, variant_info.variant_names[discr]) + ->serializeBinary(variant_col.getVariantByGlobalDiscriminator(discr), variant_col.offsetAt(n), buf, getFormatSettings()); + type_and_value = buf.str(); + } - auto value_ref = variant_col.getVariantByGlobalDiscriminator(discr).serializeValueIntoArena(variant_col.offsetAt(n), arena, begin); - res.data = value_ref.data - res.size; - res.size += value_ref.size; + char * pos = arena.allocContinue(sizeof(UInt8) + sizeof(size_t) + type_and_value.size, begin); + memcpy(pos, &null_bit, sizeof(UInt8)); + memcpy(pos + sizeof(UInt8), &type_and_value.size, sizeof(size_t)); + memcpy(pos + sizeof(UInt8) + sizeof(size_t), type_and_value.data, type_and_value.size); + res.data = pos; + res.size = sizeof(UInt8) + sizeof(size_t) + type_and_value.size; return res; } @@ -536,39 +760,36 @@ const char * ColumnDynamic::deserializeAndInsertFromArena(const char * pos) return pos; } - /// Read variant type in binary format. - const size_t variant_type_binary_data_size = unalignedLoad(pos); - pos += sizeof(variant_type_binary_data_size); - String variant_type_binary_data; - variant_type_binary_data.resize(variant_type_binary_data_size); - memcpy(variant_type_binary_data.data(), pos, variant_type_binary_data_size); - pos += variant_type_binary_data_size; - auto variant_type = decodeDataType(variant_type_binary_data); + /// Read variant type and value in binary format. + const size_t type_and_value_size = unalignedLoad(pos); + pos += sizeof(type_and_value_size); + std::string_view type_and_value(pos, type_and_value_size); + pos += type_and_value_size; + + ReadBufferFromMemory buf(type_and_value.data(), type_and_value.size()); + auto variant_type = decodeDataType(buf); auto variant_name = variant_type->getName(); /// If we already have such variant, just deserialize it into corresponding variant column. auto it = variant_info.variant_name_to_discriminator.find(variant_name); if (it != variant_info.variant_name_to_discriminator.end()) { - auto discr = it->second; - return variant_col.deserializeVariantAndInsertFromArena(discr, pos); + variant_col.deserializeBinaryIntoVariant(it->second, getVariantSerialization(variant_type, variant_name), buf, getFormatSettings()); } - - /// If we don't have such variant, add it. - if (likely(addNewVariant(variant_type))) + /// If we don't have such variant, try to add it. + else if (likely(addNewVariant(variant_type))) { auto discr = variant_info.variant_name_to_discriminator[variant_name]; - return variant_col.deserializeVariantAndInsertFromArena(discr, pos); + variant_col.deserializeBinaryIntoVariant(discr, getVariantSerialization(variant_type, variant_name), buf, getFormatSettings()); + } + /// Otherwise insert this value into shared variant. + else + { + auto & shared_variant = getSharedVariant(); + shared_variant.insertData(type_and_value.data(), type_and_value.size()); + variant_col.getLocalDiscriminators().push_back(variant_col.localDiscriminatorByGlobal(getSharedVariantDiscriminator())); + variant_col.getOffsets().push_back(shared_variant.size() - 1); } - /// We reached maximum number of variants and couldn't add new variant. - /// We should always be able to add String variant and cast inserted value to String. - addStringVariant(); - /// Create temporary column of this variant type and deserialize value into it. - auto tmp_variant_column = variant_type->createColumn(); - pos = tmp_variant_column->deserializeAndInsertFromArena(pos); - /// Cast temporary column to String and insert this value into String variant. - auto str_column = castColumn(ColumnWithTypeAndName(tmp_variant_column->getPtr(), variant_type, ""), std::make_shared()); - variant_col.insertIntoVariantFrom(variant_info.variant_name_to_discriminator["String"], *str_column, 0); return pos; } @@ -579,14 +800,10 @@ const char * ColumnDynamic::skipSerializedInArena(const char * pos) const if (null_bit) return pos; - const size_t variant_type_binary_data_size = unalignedLoad(pos); - pos += sizeof(variant_type_binary_data_size); - String variant_type_binary_data; - variant_type_binary_data.resize(variant_type_binary_data_size); - memcpy(variant_type_binary_data.data(), pos, variant_type_binary_data_size); - pos += variant_type_binary_data_size; - auto tmp_variant_column = decodeDataType(variant_type_binary_data)->createColumn(); - return tmp_variant_column->skipSerializedInArena(pos); + const size_t type_and_value_size = unalignedLoad(pos); + pos += sizeof(type_and_value_size); + pos += type_and_value_size; + return pos; } void ColumnDynamic::updateHashWithValue(size_t n, SipHash & hash) const @@ -604,9 +821,9 @@ void ColumnDynamic::updateHashWithValue(size_t n, SipHash & hash) const } #if !defined(DEBUG_OR_SANITIZER_BUILD) -int ColumnDynamic::compareAt(size_t n, size_t m, const DB::IColumn & rhs, int nan_direction_hint) const +int ColumnDynamic::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const #else -int ColumnDynamic::doCompareAt(size_t n, size_t m, const DB::IColumn & rhs, int nan_direction_hint) const +int ColumnDynamic::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const #endif { const auto & left_variant = assert_cast(*variant_column); @@ -614,7 +831,9 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const DB::IColumn & rhs, int const auto & right_variant = assert_cast(*right_dynamic.variant_column); auto left_discr = left_variant.globalDiscriminatorAt(n); + auto left_shared_variant_discr = getSharedVariantDiscriminator(); auto right_discr = right_variant.globalDiscriminatorAt(m); + auto right_shared_variant_discr = right_dynamic.getSharedVariantDiscriminator(); /// Check if we have NULLs and return result based on nan_direction_hint. if (left_discr == ColumnVariant::NULL_DISCRIMINATOR && right_discr == ColumnVariant::NULL_DISCRIMINATOR) @@ -624,12 +843,125 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const DB::IColumn & rhs, int else if (right_discr == ColumnVariant::NULL_DISCRIMINATOR) return -nan_direction_hint; - /// If rows have different types, we compare type names. - if (variant_info.variant_names[left_discr] != right_dynamic.variant_info.variant_names[right_discr]) - return variant_info.variant_names[left_discr] < right_dynamic.variant_info.variant_names[right_discr] ? -1 : 1; + /// Check if both values are in shared variant. + if (left_discr == left_shared_variant_discr && right_discr == right_shared_variant_discr) + { + /// Extract type names from both values. + auto left_value = getSharedVariant().getDataAt(left_variant.offsetAt(n)); + ReadBufferFromMemory buf_left(left_value.data, left_value.size); + auto left_data_type = decodeDataType(buf_left); + auto left_data_type_name = left_data_type->getName(); - /// If rows have the same types, compare actual values from corresponding variants. - return left_variant.getVariantByGlobalDiscriminator(left_discr).compareAt(left_variant.offsetAt(n), right_variant.offsetAt(m), right_variant.getVariantByGlobalDiscriminator(right_discr), nan_direction_hint); + auto right_value = right_dynamic.getSharedVariant().getDataAt(right_variant.offsetAt(m)); + ReadBufferFromMemory buf_right(right_value.data, right_value.size); + auto right_data_type = decodeDataType(buf_right); + auto right_data_type_name = right_data_type->getName(); + + /// If rows have different types, we compare type names. + if (left_data_type_name != right_data_type_name) + return left_data_type_name < right_data_type_name ? -1 : 1; + + /// If rows have the same type, we compare actual values. + /// We have both values serialized in binary format, so we need to + /// create temporary column, insert both values into it and compare. + auto tmp_column = left_data_type->createColumn(); + const auto & serialization = getVariantSerialization(left_data_type, left_data_type_name); + serialization->deserializeBinary(*tmp_column, buf_left, getFormatSettings()); + serialization->deserializeBinary(*tmp_column, buf_right, getFormatSettings()); + return tmp_column->compareAt(0, 1, *tmp_column, nan_direction_hint); + } + /// Check if only left value is in shared data. + else if (left_discr == left_shared_variant_discr) + { + /// Extract left type name from the value. + auto left_value = getSharedVariant().getDataAt(left_variant.offsetAt(n)); + ReadBufferFromMemory buf_left(left_value.data, left_value.size); + auto left_data_type = decodeDataType(buf_left); + auto left_data_type_name = left_data_type->getName(); + + /// If rows have different types, we compare type names. + if (left_data_type_name != right_dynamic.variant_info.variant_names[right_discr]) + return left_data_type_name < right_dynamic.variant_info.variant_names[right_discr] ? -1 : 1; + + /// If rows have the same type, we compare actual values. + /// We have left value serialized in binary format, we need to + /// create temporary column, insert the value into it and compare. + auto tmp_column = left_data_type->createColumn(); + getVariantSerialization(left_data_type, left_data_type_name)->deserializeBinary(*tmp_column, buf_left, getFormatSettings()); + return tmp_column->compareAt(0, right_variant.offsetAt(m), right_variant.getVariantByGlobalDiscriminator(right_discr), nan_direction_hint); + } + /// Check if only right value is in shared data. + else if (right_discr == right_shared_variant_discr) + { + /// Extract right type name from the value. + auto right_value = right_dynamic.getSharedVariant().getDataAt(right_variant.offsetAt(m)); + ReadBufferFromMemory buf_right(right_value.data, right_value.size); + auto right_data_type = decodeDataType(buf_right); + auto right_data_type_name = right_data_type->getName(); + + /// If rows have different types, we compare type names. + if (variant_info.variant_names[left_discr] != right_data_type_name) + return variant_info.variant_names[left_discr] < right_data_type_name ? -1 : 1; + + /// If rows have the same type, we compare actual values. + /// We have right value serialized in binary format, we need to + /// create temporary column, insert the value into it and compare. + auto tmp_column = right_data_type->createColumn(); + getVariantSerialization(right_data_type, right_data_type_name)->deserializeBinary(*tmp_column, buf_right, getFormatSettings()); + return left_variant.getVariantByGlobalDiscriminator(left_discr).compareAt(left_variant.offsetAt(n), 0, *tmp_column, nan_direction_hint); + } + /// Otherwise both values are regular variants. + else + { + /// If rows have different types, we compare type names. + if (variant_info.variant_names[left_discr] != right_dynamic.variant_info.variant_names[right_discr]) + return variant_info.variant_names[left_discr] < right_dynamic.variant_info.variant_names[right_discr] ? -1 : 1; + + /// If rows have the same types, compare actual values from corresponding variants. + return left_variant.getVariantByGlobalDiscriminator(left_discr).compareAt(left_variant.offsetAt(n), right_variant.offsetAt(m), right_variant.getVariantByGlobalDiscriminator(right_discr), nan_direction_hint); + } +} + +struct ColumnDynamic::ComparatorBase +{ + const ColumnDynamic & parent; + int nan_direction_hint; + + ComparatorBase(const ColumnDynamic & parent_, int nan_direction_hint_) + : parent(parent_), nan_direction_hint(nan_direction_hint_) + { + } + + ALWAYS_INLINE int compare(size_t lhs, size_t rhs) const + { + return parent.compareAt(lhs, rhs, parent, nan_direction_hint); + } +}; + +void ColumnDynamic::getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const +{ + if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) + getPermutationImpl(limit, res, ComparatorAscendingUnstable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); + else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) + getPermutationImpl(limit, res, ComparatorAscendingStable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); + else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) + getPermutationImpl(limit, res, ComparatorDescendingUnstable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); + else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable) + getPermutationImpl(limit, res, ComparatorDescendingStable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); +} + +void ColumnDynamic::updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, DB::EqualRanges & equal_ranges) const +{ + auto comparator_equal = ComparatorEqual(*this, nan_direction_hint); + + if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) + updatePermutationImpl(limit, res, equal_ranges, ComparatorAscendingUnstable(*this, nan_direction_hint), comparator_equal, DefaultSort(), DefaultPartialSort()); + else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) + updatePermutationImpl(limit, res, equal_ranges, ComparatorAscendingStable(*this, nan_direction_hint), comparator_equal, DefaultSort(), DefaultPartialSort()); + else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) + updatePermutationImpl(limit, res, equal_ranges, ComparatorDescendingUnstable(*this, nan_direction_hint), comparator_equal, DefaultSort(), DefaultPartialSort()); + else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable) + updatePermutationImpl(limit, res, equal_ranges, ComparatorDescendingStable(*this, nan_direction_hint), comparator_equal, DefaultSort(), DefaultPartialSort()); } ColumnPtr ColumnDynamic::compress() const @@ -637,14 +969,16 @@ ColumnPtr ColumnDynamic::compress() const ColumnPtr variant_compressed = variant_column->compress(); size_t byte_size = variant_compressed->byteSize(); return ColumnCompressed::create(size(), byte_size, - [my_variant_compressed = std::move(variant_compressed), my_variant_info = variant_info, my_max_dynamic_types = max_dynamic_types, my_statistics = statistics]() mutable + [my_variant_compressed = std::move(variant_compressed), my_variant_info = variant_info, my_max_dynamic_types = max_dynamic_types, my_global_max_dynamic_types = global_max_dynamic_types, my_statistics = statistics]() mutable { - return ColumnDynamic::create(my_variant_compressed->decompress(), my_variant_info, my_max_dynamic_types, my_statistics); + return ColumnDynamic::create(my_variant_compressed->decompress(), my_variant_info, my_max_dynamic_types, my_global_max_dynamic_types, my_statistics); }); } void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source_columns) { + LOG_DEBUG(getLogger("ColumnDynamic"), "takeDynamicStructureFromSourceColumns"); + if (!empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "takeDynamicStructureFromSourceColumns should be called only on empty Dynamic column"); @@ -663,6 +997,9 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source /// First, collect all variants from all source columns and calculate total sizes. std::unordered_map total_sizes; DataTypes all_variants; + /// Add shared variant type in advance; + all_variants.push_back(getSharedVariantDataType()); + total_sizes[getSharedVariantTypeName()] = 0; for (const auto & source_column : source_columns) { @@ -671,7 +1008,7 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source const auto & source_variant_info = source_dynamic.getVariantInfo(); const auto & source_variants = assert_cast(*source_variant_info.variant_type).getVariants(); /// During deserialization from MergeTree we will have variant sizes statistics from the whole data part. - const auto & source_statistics = source_dynamic.getStatistics(); + const auto & source_statistics = source_dynamic.getStatistics(); for (size_t i = 0; i != source_variants.size(); ++i) { const auto & variant_name = source_variant_info.variant_names[i]; @@ -682,37 +1019,67 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source all_variants.push_back(source_variants[i]); it = total_sizes.emplace(variant_name, 0).first; } - auto statistics_it = source_statistics.data.find(variant_name); - size_t size = statistics_it == source_statistics.data.end() ? source_variant_column.getVariantByGlobalDiscriminator(i).size() : statistics_it->second; + size_t size = source_variant_column.getVariantByGlobalDiscriminator(i).size(); + if (source_statistics) + { + auto statistics_it = source_statistics->variants_statistics.find(variant_name); + if (statistics_it != source_statistics->variants_statistics.end()) + size = statistics_it->second; + } + it->second += size; } + + /// Use add variants from shared variant statistics. It can help extracting + /// frequent variants from shared variant to usual variants. + if (source_statistics) + { + for (const auto & [variant_name, size] : source_statistics->shared_variants_statistics) + { + auto it = total_sizes.find(variant_name); + /// Add this variant to the list of all variants if we didn't see it yet. + if (it == total_sizes.end()) + { + all_variants.push_back(DataTypeFactory::instance().get(variant_name)); + it = total_sizes.emplace(variant_name, 0).first; + } + it->second += size; + } + } } DataTypePtr result_variant_type; - /// Check if the number of all variants exceeds the limit. - if (all_variants.size() > max_dynamic_types || (all_variants.size() == max_dynamic_types && !total_sizes.contains("String"))) + Statistics new_statistics(Statistics::Source::MERGE); + /// Check if the number of all dynamic types exceeds the limit. + if (all_variants.size() - 1 > global_max_dynamic_types) { /// Create list of variants with their sizes and sort it. std::vector> variants_with_sizes; variants_with_sizes.reserve(all_variants.size()); for (const auto & variant : all_variants) - variants_with_sizes.emplace_back(total_sizes[variant->getName()], variant); + { + if (variant->getName() != getSharedVariantTypeName()) + variants_with_sizes.emplace_back(total_sizes[variant->getName()], variant); + } std::sort(variants_with_sizes.begin(), variants_with_sizes.end(), std::greater()); - /// Take first max_dynamic_types variants from sorted list. + /// Take first max_dynamic_types variants from sorted list and fill shared_variants_statistics with the rest. DataTypes result_variants; - result_variants.reserve(max_dynamic_types); - /// Add String variant in advance. - result_variants.push_back(std::make_shared()); - for (const auto & [_, variant] : variants_with_sizes) + result_variants.reserve(global_max_dynamic_types + 1); + for (const auto & [size, variant] : variants_with_sizes) { - if (result_variants.size() == max_dynamic_types) - break; - - if (variant->getName() != "String") + /// Add variant to the resulting variants list until we reach max_dynamic_types. + if (result_variants.size() < global_max_dynamic_types) result_variants.push_back(variant); + /// Add all remaining variants into shared_variants_statistics until we reach its max size. + else if (new_statistics.shared_variants_statistics.size() < Statistics::MAX_SHARED_VARIANT_STATISTICS_SIZE) + new_statistics.shared_variants_statistics[variant->getName()] = size; + else + break; } + /// Add shared variant. + result_variants.push_back(getSharedVariantDataType()); result_variant_type = std::make_shared(result_variants); } else @@ -720,26 +1087,16 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source result_variant_type = std::make_shared(all_variants); } - /// Now we have resulting Variant and can fill variant info. - variant_info.variant_type = result_variant_type; - variant_info.variant_name = result_variant_type->getName(); - const auto & result_variants = assert_cast(*result_variant_type).getVariants(); - variant_info.variant_names.clear(); - variant_info.variant_names.reserve(result_variants.size()); - variant_info.variant_name_to_discriminator.clear(); - variant_info.variant_name_to_discriminator.reserve(result_variants.size()); - statistics.data.clear(); - statistics.data.reserve(result_variants.size()); - statistics.source = Statistics::Source::MERGE; - for (size_t i = 0; i != result_variants.size(); ++i) - { - auto variant_name = result_variants[i]->getName(); - variant_info.variant_names.push_back(variant_name); - variant_info.variant_name_to_discriminator[variant_name] = i; - statistics.data[variant_name] = total_sizes[variant_name]; - } + /// Now we have resulting Variant and can fill variant info and create merge statistics. + setVariantType(result_variant_type); + new_statistics.variants_statistics.reserve(variant_info.variant_names.size()); + for (const auto & variant_name : variant_info.variant_names) + new_statistics.variants_statistics[variant_name] = total_sizes[variant_name]; + statistics = std::make_shared(std::move(new_statistics)); - variant_column = variant_info.variant_type->createColumn(); + /// Reduce max_dynamic_types to the number of selected variants (without shared variant), so there will be no possibility + /// to extend selected variants on inerts into this column during merges. + max_dynamic_types = variant_info.variant_names.size() - 1; /// Now we have the resulting Variant that will be used in all merged columns. /// Variants can also contain Dynamic columns inside, we should collect diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index e92cabd3db9..8b815e2b015 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -19,11 +20,15 @@ namespace DB * * When new values are inserted into Dynamic column, the internal Variant * type and column are extended if the inserted value has new type. + * When the limit on number of dynamic types is exceeded, all values + * with new types are inserted into special shared variant with type String + * that contains values and their types in binary format. */ class ColumnDynamic final : public COWHelper, ColumnDynamic> { public: - /// + static constexpr const char * SHARED_VARIANT_TYPE_NAME = "SharedVariant"; + struct Statistics { enum class Source @@ -32,12 +37,27 @@ public: MERGE, /// Statistics were calculated during merge of several MergeTree parts. }; + Statistics(Source source_) : source(source_) {} + /// Source of the statistics. Source source; - /// Statistics data: (variant name) -> (total variant size in data part). - std::unordered_map data; + /// Statistics data for usual variants: (variant name) -> (total variant size in data part). + std::unordered_map variants_statistics; + /// Statistics data for variants from shared variant: (variant name) -> (total variant size in data part). + /// For shared variant we store statistics only for first 256 variants (should cover almost all cases and it's not expensive). + static constexpr const size_t MAX_SHARED_VARIANT_STATISTICS_SIZE = 256; + std::unordered_map shared_variants_statistics; }; + using StatisticsPtr = std::shared_ptr; + + struct ComparatorBase; + using ComparatorAscendingUnstable = ComparatorAscendingUnstableImpl; + using ComparatorAscendingStable = ComparatorAscendingStableImpl; + using ComparatorDescendingUnstable = ComparatorDescendingUnstableImpl; + using ComparatorDescendingStable = ComparatorDescendingStableImpl; + using ComparatorEqual = ComparatorEqualImpl; + private: friend class COWHelper, ColumnDynamic>; @@ -54,28 +74,32 @@ private: }; explicit ColumnDynamic(size_t max_dynamic_types_); - ColumnDynamic(MutableColumnPtr variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, const Statistics & statistics_ = {}); + ColumnDynamic(MutableColumnPtr variant_column_, const DataTypePtr & variant_type_, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_ = {}); + ColumnDynamic(MutableColumnPtr variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_ = {}); public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. */ using Base = COWHelper, ColumnDynamic>; - static Ptr create(const ColumnPtr & variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, const Statistics & statistics_ = {}) + static Ptr create(const ColumnPtr & variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_ = {}) { - return ColumnDynamic::create(variant_column_->assumeMutable(), variant_info_, max_dynamic_types_, statistics_); + return ColumnDynamic::create(variant_column_->assumeMutable(), variant_info_, max_dynamic_types_, global_max_dynamic_types_, statistics_); } - static MutablePtr create(MutableColumnPtr variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, const Statistics & statistics_ = {}) + static MutablePtr create(MutableColumnPtr variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_ = {}) { - return Base::create(std::move(variant_column_), variant_info_, max_dynamic_types_, statistics_); + return Base::create(std::move(variant_column_), variant_info_, max_dynamic_types_, global_max_dynamic_types_, statistics_); } - static MutablePtr create(MutableColumnPtr variant_column_, const DataTypePtr & variant_type, size_t max_dynamic_types_, const Statistics & statistics_ = {}); - - static ColumnPtr create(ColumnPtr variant_column_, const DataTypePtr & variant_type, size_t max_dynamic_types_, const Statistics & statistics_ = {}) + static MutablePtr create(MutableColumnPtr variant_column_, const DataTypePtr & variant_type_, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_ = {}) { - return create(variant_column_->assumeMutable(), variant_type, max_dynamic_types_, statistics_); + return Base::create(std::move(variant_column_), variant_type_, max_dynamic_types_, global_max_dynamic_types_, statistics_); + } + + static ColumnPtr create(ColumnPtr variant_column_, const DataTypePtr & variant_type, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_ = {}) + { + return create(variant_column_->assumeMutable(), variant_type, max_dynamic_types_, global_max_dynamic_types_, statistics_); } static MutablePtr create(size_t max_dynamic_types_) @@ -83,7 +107,7 @@ public: return Base::create(max_dynamic_types_); } - std::string getName() const override { return "Dynamic(max_types=" + std::to_string(max_dynamic_types) + ")"; } + std::string getName() const override { return "Dynamic(max_types=" + std::to_string(global_max_dynamic_types) + ")"; } const char * getFamilyName() const override { @@ -98,12 +122,12 @@ public: MutableColumnPtr cloneEmpty() const override { /// Keep current dynamic structure - return Base::create(variant_column->cloneEmpty(), variant_info, max_dynamic_types, statistics); + return Base::create(variant_column->cloneEmpty(), variant_info, max_dynamic_types, global_max_dynamic_types, statistics); } MutableColumnPtr cloneResized(size_t size) const override { - return Base::create(variant_column->cloneResized(size), variant_info, max_dynamic_types, statistics); + return Base::create(variant_column->cloneResized(size), variant_info, max_dynamic_types, global_max_dynamic_types, statistics); } size_t size() const override @@ -111,15 +135,9 @@ public: return variant_column->size(); } - Field operator[](size_t n) const override - { - return (*variant_column)[n]; - } + Field operator[](size_t n) const override; - void get(size_t n, Field & res) const override - { - variant_column->get(n, res); - } + void get(size_t n, Field & res) const override; bool isDefaultAt(size_t n) const override { @@ -187,7 +205,7 @@ public: ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override { - return create(variant_column->filter(filt, result_size_hint), variant_info, max_dynamic_types); + return create(variant_column->filter(filt, result_size_hint), variant_info, max_dynamic_types, global_max_dynamic_types); } void expand(const Filter & mask, bool inverted) override @@ -197,17 +215,17 @@ public: ColumnPtr permute(const Permutation & perm, size_t limit) const override { - return create(variant_column->permute(perm, limit), variant_info, max_dynamic_types); + return create(variant_column->permute(perm, limit), variant_info, max_dynamic_types, global_max_dynamic_types); } ColumnPtr index(const IColumn & indexes, size_t limit) const override { - return create(variant_column->index(indexes, limit), variant_info, max_dynamic_types); + return create(variant_column->index(indexes, limit), variant_info, max_dynamic_types, global_max_dynamic_types); } ColumnPtr replicate(const Offsets & replicate_offsets) const override { - return create(variant_column->replicate(replicate_offsets), variant_info, max_dynamic_types); + return create(variant_column->replicate(replicate_offsets), variant_info, max_dynamic_types, global_max_dynamic_types); } MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override @@ -216,7 +234,7 @@ public: MutableColumns scattered_columns; scattered_columns.reserve(num_columns); for (auto & scattered_variant_column : scattered_variant_columns) - scattered_columns.emplace_back(create(std::move(scattered_variant_column), variant_info, max_dynamic_types)); + scattered_columns.emplace_back(create(std::move(scattered_variant_column), variant_info, max_dynamic_types, global_max_dynamic_types)); return scattered_columns; } @@ -238,16 +256,10 @@ public: } void getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, - size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override - { - variant_column->getPermutation(direction, stability, limit, nan_direction_hint, res); - } + size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override; void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, - size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override - { - variant_column->updatePermutation(direction, stability, limit, nan_direction_hint, res, equal_ranges); - } + size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override; void reserve(size_t n) override { @@ -293,7 +305,7 @@ public: bool structureEquals(const IColumn & rhs) const override { if (const auto * rhs_concrete = typeid_cast(&rhs)) - return max_dynamic_types == rhs_concrete->max_dynamic_types; + return global_max_dynamic_types == rhs_concrete->global_max_dynamic_types; return false; } @@ -336,17 +348,67 @@ public: const ColumnVariant & getVariantColumn() const { return assert_cast(*variant_column); } ColumnVariant & getVariantColumn() { return assert_cast(*variant_column); } - bool addNewVariant(const DataTypePtr & new_variant); - void addStringVariant(); + bool addNewVariant(const DataTypePtr & new_variant, const String & new_variant_name); + bool addNewVariant(const DataTypePtr & new_variant) { return addNewVariant(new_variant, new_variant->getName()); } bool hasDynamicStructure() const override { return true; } void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; - const Statistics & getStatistics() const { return statistics; } + const StatisticsPtr & getStatistics() const { return statistics; } + void setStatistics(const StatisticsPtr & statistics_) { statistics = statistics_; } size_t getMaxDynamicTypes() const { return max_dynamic_types; } + void setVariantType(const DataTypePtr & variant_type); + void setMaxDynamicPaths(size_t max_dynamic_type_); + + static const String & getSharedVariantTypeName() + { + static const String name = SHARED_VARIANT_TYPE_NAME; + return name; + } + + static DataTypePtr getSharedVariantDataType(); + + ColumnVariant::Discriminator getSharedVariantDiscriminator() const + { + return variant_info.variant_name_to_discriminator.at(getSharedVariantTypeName()); + } + + ColumnString & getSharedVariant() + { + return assert_cast(getVariantColumn().getVariantByGlobalDiscriminator(getSharedVariantDiscriminator())); + } + + const ColumnString & getSharedVariant() const + { + return assert_cast(getVariantColumn().getVariantByGlobalDiscriminator(getSharedVariantDiscriminator())); + } + + /// Serializes type and value in binary format into provided shared variant. Doesn't update Variant discriminators and offsets. + static void serializeValueIntoSharedVariant(ColumnString & shared_variant, const IColumn & src, const DataTypePtr & type, const SerializationPtr & serialization, size_t n); + + /// Insert value into shared variant. Also updates Variant discriminators and offsets. + void insertValueIntoSharedVariant(const IColumn & src, const DataTypePtr & type, const String & type_name, size_t n); + + const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type, const String & variant_name) const + { + /// Get serialization for provided data type. + /// To avoid calling type->getDefaultSerialization() every time we use simple cache with max size. + /// When max size is reached, just clear the cache. + if (serialization_cache.size() == SERIALIZATION_CACHE_MAX_SIZE) + serialization_cache.clear(); + + if (auto it = serialization_cache.find(variant_name); it != serialization_cache.end()) + return it->second; + + return serialization_cache.emplace(variant_name, variant_type->getDefaultSerialization()).first->second; + } + + const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type) const { return getVariantSerialization(variant_type, variant_type->getName()); } private: + void createVariantInfo(const DataTypePtr & variant_type); + /// Combine current variant with the other variant and return global discriminators mapping /// from other variant to the combined one. It's used for inserting from /// different variants. @@ -359,12 +421,19 @@ private: /// Store the type of current variant with some additional information. VariantInfo variant_info; /// The maximum number of different types that can be stored in this Dynamic column. - /// If exceeded, all new variants will be converted to String. + /// If exceeded, all new variants will be added to a special shared variant with type String + /// in binary format. This limit can be different for different instances of Dynamic column. + /// When max_dynamic_types = 0, we will have only shared variant and insert all values into it. size_t max_dynamic_types; + /// The types limit specified in the data type by the user Dynamic(max_types=N). + /// max_dynamic_types in all column instances of this Dynamic type can be only smaller + /// (for example, max_dynamic_types can be reduced in takeDynamicStructureFromSourceColumns + /// before merge of different Dynamic columns). + size_t global_max_dynamic_types; /// Size statistics of each variants from MergeTree data part. /// Used in takeDynamicStructureFromSourceColumns and set during deserialization. - Statistics statistics; + StatisticsPtr statistics; /// Cache (Variant name) -> (global discriminators mapping from this variant to current variant in Dynamic column). /// Used to avoid mappings recalculation in combineVariants for the same Variant types. @@ -372,6 +441,17 @@ private: /// Cache of Variant types that couldn't be combined with current variant in Dynamic column. /// Used to avoid checking if combination is possible for the same Variant types. std::unordered_set variants_with_failed_combination; + + /// We can use serializations of different data types to serialize values into shared variant. + /// To avoid creating the same serialization multiple times, use simple cache. + static const size_t SERIALIZATION_CACHE_MAX_SIZE = 256; + mutable std::unordered_map serialization_cache; }; +void extendVariantColumn( + IColumn & variant_column, + const DataTypePtr & old_variant_type, + const DataTypePtr & new_variant_type, + std::unordered_map old_variant_name_to_discriminator); + } diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index de7efb41d19..7531e976926 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -476,7 +476,7 @@ void ColumnVariant::insertFromImpl(const DB::IColumn & src_, size_t n, const std } } -void ColumnVariant::insertRangeFromImpl(const DB::IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping) +void ColumnVariant::insertRangeFromImpl(const DB::IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping, Discriminator * skip_discriminator) { const size_t num_variants = variants.size(); const auto & src = assert_cast(src_); @@ -557,9 +557,12 @@ void ColumnVariant::insertRangeFromImpl(const DB::IColumn & src_, size_t start, Discriminator global_discr = src_global_discr; if (global_discriminators_mapping && src_global_discr != NULL_DISCRIMINATOR) global_discr = (*global_discriminators_mapping)[src_global_discr]; - Discriminator local_discr = localDiscriminatorByGlobal(global_discr); - if (nested_length) - variants[local_discr]->insertRangeFrom(*src.variants[src_local_discr], nested_start, nested_length); + if (!skip_discriminator || global_discr != *skip_discriminator) + { + Discriminator local_discr = localDiscriminatorByGlobal(global_discr); + if (nested_length) + variants[local_discr]->insertRangeFrom(*src.variants[src_local_discr], nested_start, nested_length); + } } } @@ -610,7 +613,7 @@ void ColumnVariant::insertRangeFrom(const IColumn & src_, size_t start, size_t l void ColumnVariant::doInsertRangeFrom(const IColumn & src_, size_t start, size_t length) #endif { - insertRangeFromImpl(src_, start, length, nullptr); + insertRangeFromImpl(src_, start, length, nullptr, nullptr); } #if !defined(DEBUG_OR_SANITIZER_BUILD) @@ -627,9 +630,9 @@ void ColumnVariant::insertFrom(const DB::IColumn & src_, size_t n, const std::ve insertFromImpl(src_, n, &global_discriminators_mapping); } -void ColumnVariant::insertRangeFrom(const IColumn & src_, size_t start, size_t length, const std::vector & global_discriminators_mapping) +void ColumnVariant::insertRangeFrom(const IColumn & src_, size_t start, size_t length, const std::vector & global_discriminators_mapping, Discriminator skip_discriminator) { - insertRangeFromImpl(src_, start, length, &global_discriminators_mapping); + insertRangeFromImpl(src_, start, length, &global_discriminators_mapping, &skip_discriminator); } void ColumnVariant::insertManyFrom(const DB::IColumn & src_, size_t position, size_t length, const std::vector & global_discriminators_mapping) @@ -673,6 +676,14 @@ void ColumnVariant::insertManyIntoVariantFrom(DB::ColumnVariant::Discriminator g variants[local_discr]->insertManyFrom(src_, position, length); } +void ColumnVariant::deserializeBinaryIntoVariant(ColumnVariant::Discriminator global_discr, const SerializationPtr & serialization, ReadBuffer & buf, const FormatSettings & format_settings) +{ + auto local_discr = localDiscriminatorByGlobal(global_discr); + serialization->deserializeBinary(*variants[local_discr], buf, format_settings); + getLocalDiscriminators().push_back(local_discr); + getOffsets().push_back(variants[local_discr]->size() - 1); +} + void ColumnVariant::insertDefault() { getLocalDiscriminators().push_back(NULL_DISCRIMINATOR); @@ -1213,9 +1224,7 @@ struct ColumnVariant::ComparatorBase ALWAYS_INLINE int compare(size_t lhs, size_t rhs) const { - int res = parent.compareAt(lhs, rhs, parent, nan_direction_hint); - - return res; + return parent.compareAt(lhs, rhs, parent, nan_direction_hint); } }; diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 34c24b5428d..571a843d113 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -2,6 +2,8 @@ #include #include +#include +#include namespace DB @@ -196,13 +198,15 @@ public: /// Methods for insertion from another Variant but with known mapping between global discriminators. void insertFrom(const IColumn & src_, size_t n, const std::vector & global_discriminators_mapping); - void insertRangeFrom(const IColumn & src_, size_t start, size_t length, const std::vector & global_discriminators_mapping); + /// Don't insert data into variant with skip_discriminator global discriminator, it will be processed separately. + void insertRangeFrom(const IColumn & src_, size_t start, size_t length, const std::vector & global_discriminators_mapping, Discriminator skip_discriminator); void insertManyFrom(const IColumn & src_, size_t position, size_t length, const std::vector & global_discriminators_mapping); /// Methods for insertion into a specific variant. void insertIntoVariantFrom(Discriminator global_discr, const IColumn & src_, size_t n); void insertRangeIntoVariantFrom(Discriminator global_discr, const IColumn & src_, size_t start, size_t length); void insertManyIntoVariantFrom(Discriminator global_discr, const IColumn & src_, size_t position, size_t length); + void deserializeBinaryIntoVariant(Discriminator global_discr, const SerializationPtr & serialization, ReadBuffer & buf, const FormatSettings & format_settings); void insertDefault() override; void insertManyDefaults(size_t length) override; @@ -263,6 +267,7 @@ public: ColumnPtr & getVariantPtrByGlobalDiscriminator(size_t discr) { return variants[global_to_local_discriminators.at(discr)]; } const NestedColumns & getVariants() const { return variants; } + NestedColumns & getVariants() { return variants; } const IColumn & getLocalDiscriminatorsColumn() const { return *local_discriminators; } IColumn & getLocalDiscriminatorsColumn() { return *local_discriminators; } @@ -302,6 +307,8 @@ public: return true; } + std::vector getLocalToGlobalDiscriminatorsMapping() const { return local_to_global_discriminators; } + /// Check if we have only 1 non-empty variant and no NULL values, /// and if so, return the discriminator of this non-empty column. std::optional getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls() const; @@ -322,7 +329,7 @@ public: private: void insertFromImpl(const IColumn & src_, size_t n, const std::vector * global_discriminators_mapping); - void insertRangeFromImpl(const IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping); + void insertRangeFromImpl(const IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping, Discriminator * skip_discriminator); void insertManyFromImpl(const IColumn & src_, size_t position, size_t length, const std::vector * global_discriminators_mapping); void initIdentityGlobalToLocalDiscriminatorsMapping(); diff --git a/src/Columns/tests/gtest_column_dynamic.cpp b/src/Columns/tests/gtest_column_dynamic.cpp index a2862b09de1..5445bd525d9 100644 --- a/src/Columns/tests/gtest_column_dynamic.cpp +++ b/src/Columns/tests/gtest_column_dynamic.cpp @@ -9,9 +9,12 @@ TEST(ColumnDynamic, CreateEmpty) { auto column = ColumnDynamic::create(255); ASSERT_TRUE(column->empty()); - ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant()"); - ASSERT_TRUE(column->getVariantInfo().variant_names.empty()); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.empty()); + ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant(SharedVariant)"); + ASSERT_EQ(column->getVariantInfo().variant_names.size(), 1); + ASSERT_EQ(column->getVariantInfo().variant_names[0], "SharedVariant"); + ASSERT_EQ(column->getVariantInfo().variant_name_to_discriminator.size(), 1); + ASSERT_EQ(column->getVariantInfo().variant_name_to_discriminator.at("SharedVariant"), 0); + ASSERT_TRUE(column->getVariantColumn().getVariantByGlobalDiscriminator(0).empty()); } TEST(ColumnDynamic, InsertDefault) @@ -19,9 +22,12 @@ TEST(ColumnDynamic, InsertDefault) auto column = ColumnDynamic::create(255); column->insertDefault(); ASSERT_TRUE(column->size() == 1); - ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant()"); - ASSERT_TRUE(column->getVariantInfo().variant_names.empty()); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.empty()); + ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant(SharedVariant)"); + ASSERT_EQ(column->getVariantInfo().variant_names.size(), 1); + ASSERT_EQ(column->getVariantInfo().variant_names[0], "SharedVariant"); + ASSERT_EQ(column->getVariantInfo().variant_name_to_discriminator.size(), 1); + ASSERT_EQ(column->getVariantInfo().variant_name_to_discriminator.at("SharedVariant"), 0); + ASSERT_TRUE(column->getVariantColumn().getVariantByGlobalDiscriminator(0).empty()); ASSERT_TRUE(column->isNullAt(0)); ASSERT_EQ((*column)[0], Field(Null())); } @@ -41,10 +47,10 @@ TEST(ColumnDynamic, InsertFields) column->insert(Field(43.43)); ASSERT_TRUE(column->size() == 10); - ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant(Float64, Int8, String)"); - std::vector expected_names = {"Float64", "Int8", "String"}; + ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant(Float64, Int8, SharedVariant, String)"); + std::vector expected_names = {"Float64", "Int8", "SharedVariant", "String"}; ASSERT_EQ(column->getVariantInfo().variant_names, expected_names); - std::unordered_map expected_variant_name_to_discriminator = {{"Float64", 0}, {"Int8", 1}, {"String", 2}}; + std::unordered_map expected_variant_name_to_discriminator = {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}; ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator == expected_variant_name_to_discriminator); } @@ -66,56 +72,66 @@ TEST(ColumnDynamic, InsertFieldsOverflow1) { auto column = getDynamicWithManyVariants(253); - ASSERT_EQ(column->getVariantInfo().variant_names.size(), 253); + ASSERT_EQ(column->getVariantInfo().variant_names.size(), 254); column->insert(Field(42.42)); - ASSERT_EQ(column->getVariantInfo().variant_names.size(), 254); + ASSERT_EQ(column->size(), 254); + ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.contains("Float64")); column->insert(Field(42)); + ASSERT_EQ(column->size(), 255); ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("Int8")); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column->getSharedVariant().size(), 1); Field field = (*column)[column->size() - 1]; - ASSERT_EQ(field, "42"); + ASSERT_EQ(field, 42); column->insert(Field(43)); + ASSERT_EQ(column->size(), 256); ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("Int8")); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column->getSharedVariant().size(), 2); field = (*column)[column->size() - 1]; - ASSERT_EQ(field, "43"); + ASSERT_EQ(field, 43); column->insert(Field("str1")); + ASSERT_EQ(column->size(), 257); ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("Int8")); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column->getSharedVariant().size(), 3); field = (*column)[column->size() - 1]; ASSERT_EQ(field, "str1"); column->insert(Field(Array({Field(42), Field(43)}))); ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("Array(Int8)")); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column->getSharedVariant().size(), 4); field = (*column)[column->size() - 1]; - ASSERT_EQ(field, "[42, 43]"); + ASSERT_EQ(field, Field(Array({Field(42), Field(43)}))); } TEST(ColumnDynamic, InsertFieldsOverflow2) { auto column = getDynamicWithManyVariants(254); - ASSERT_EQ(column->getVariantInfo().variant_names.size(), 254); + ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); column->insert(Field("str1")); ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column->getSharedVariant().size(), 1); + Field field = (*column)[column->size() - 1]; + ASSERT_EQ(field, "str1"); column->insert(Field(42)); ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("Int8")); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); - Field field = (*column)[column->size() - 1]; - ASSERT_EQ(field, "42"); + ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column->getSharedVariant().size(), 2); + field = (*column)[column->size() - 1]; + ASSERT_EQ(field, 42); } ColumnDynamic::MutablePtr getInsertFromColumn(size_t num = 1) @@ -155,7 +171,7 @@ void checkInsertFrom(const ColumnDynamic::MutablePtr & column_from, ColumnDynami TEST(ColumnDynamic, InsertFrom1) { auto column_to = ColumnDynamic::create(255); - checkInsertFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, String)", {"Float64", "Int8", "String"}, {{"Float64", 0}, {"Int8", 1}, {"String", 2}}); + checkInsertFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertFrom2) @@ -165,7 +181,7 @@ TEST(ColumnDynamic, InsertFrom2) column_to->insert(Field(42.42)); column_to->insert(Field("str")); - checkInsertFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, String)", {"Float64", "Int8", "String"}, {{"Float64", 0}, {"Int8", 1}, {"String", 2}}); + checkInsertFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertFrom3) @@ -176,7 +192,7 @@ TEST(ColumnDynamic, InsertFrom3) column_to->insert(Field("str")); column_to->insert(Array({Field(42)})); - checkInsertFrom(getInsertFromColumn(), column_to, "Variant(Array(Int8), Float64, Int8, String)", {"Array(Int8)", "Float64", "Int8", "String"}, {{"Array(Int8)", 0}, {"Float64", 1}, {"Int8", 2}, {"String", 3}}); + checkInsertFrom(getInsertFromColumn(), column_to, "Variant(Array(Int8), Float64, Int8, SharedVariant, String)", {"Array(Int8)", "Float64", "Int8", "SharedVariant", "String"}, {{"Array(Int8)", 0}, {"Float64", 1}, {"Int8", 2}, {"SharedVariant", 3}, {"String", 4}}); } TEST(ColumnDynamic, InsertFromOverflow1) @@ -188,7 +204,7 @@ TEST(ColumnDynamic, InsertFromOverflow1) auto column_to = getDynamicWithManyVariants(253); column_to->insertFrom(*column_from, 0); - ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 254); + ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); auto field = (*column_to)[column_to->size() - 1]; ASSERT_EQ(field, 42); @@ -196,13 +212,15 @@ TEST(ColumnDynamic, InsertFromOverflow1) column_to->insertFrom(*column_from, 1); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column_to->getSharedVariant().size(), 1); field = (*column_to)[column_to->size() - 1]; - ASSERT_EQ(field, "42.42"); + ASSERT_EQ(field, 42.42); column_to->insertFrom(*column_from, 2); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); field = (*column_to)[column_to->size() - 1]; ASSERT_EQ(field, "str"); } @@ -221,9 +239,32 @@ TEST(ColumnDynamic, InsertFromOverflow2) column_to->insertFrom(*column_from, 1); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column_to->getSharedVariant().size(), 1); field = (*column_to)[column_to->size() - 1]; - ASSERT_EQ(field, "42.42"); + ASSERT_EQ(field, 42.42); +} + +TEST(ColumnDynamic, InsertFromOverflow3) +{ + auto column_from = ColumnDynamic::create(1); + column_from->insert(Field(42)); + column_from->insert(Field(42.42)); + + auto column_to = ColumnDynamic::create(255); + column_to->insert(Field(41)); + + column_to->insertFrom(*column_from, 0); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_EQ(column_to->getSharedVariant().size(), 0); + auto field = (*column_to)[column_to->size() - 1]; + ASSERT_EQ(field, 42); + + column_to->insertFrom(*column_from, 1); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_EQ(column_to->getSharedVariant().size(), 1); + field = (*column_to)[column_to->size() - 1]; + ASSERT_EQ(field, 42.42); } void checkInsertManyFrom(const ColumnDynamic::MutablePtr & column_from, ColumnDynamic::MutablePtr & column_to, const std::string & expected_variant, const std::vector & expected_names, const std::unordered_map & expected_variant_name_to_discriminator) @@ -257,7 +298,7 @@ void checkInsertManyFrom(const ColumnDynamic::MutablePtr & column_from, ColumnDy TEST(ColumnDynamic, InsertManyFrom1) { auto column_to = ColumnDynamic::create(255); - checkInsertManyFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, String)", {"Float64", "Int8", "String"}, {{"Float64", 0}, {"Int8", 1}, {"String", 2}}); + checkInsertManyFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertManyFrom2) @@ -267,7 +308,7 @@ TEST(ColumnDynamic, InsertManyFrom2) column_to->insert(Field(42.42)); column_to->insert(Field("str")); - checkInsertManyFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, String)", {"Float64", "Int8", "String"}, {{"Float64", 0}, {"Int8", 1}, {"String", 2}}); + checkInsertManyFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertManyFrom3) @@ -278,7 +319,7 @@ TEST(ColumnDynamic, InsertManyFrom3) column_to->insert(Field("str")); column_to->insert(Array({Field(42)})); - checkInsertManyFrom(getInsertFromColumn(), column_to, "Variant(Array(Int8), Float64, Int8, String)", {"Array(Int8)", "Float64", "Int8", "String"}, {{"Array(Int8)", 0}, {"Float64", 1}, {"Int8", 2}, {"String", 3}}); + checkInsertManyFrom(getInsertFromColumn(), column_to, "Variant(Array(Int8), Float64, Int8, SharedVariant, String)", {"Array(Int8)", "Float64", "Int8", "SharedVariant", "String"}, {{"Array(Int8)", 0}, {"Float64", 1}, {"Int8", 2}, {"SharedVariant", 3}, {"String", 4}}); } TEST(ColumnDynamic, InsertManyFromOverflow1) @@ -290,8 +331,9 @@ TEST(ColumnDynamic, InsertManyFromOverflow1) auto column_to = getDynamicWithManyVariants(253); column_to->insertManyFrom(*column_from, 0, 2); - ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 254); + ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_EQ(column_to->getSharedVariant().size(), 0); auto field = (*column_to)[column_to->size() - 2]; ASSERT_EQ(field, 42); field = (*column_to)[column_to->size() - 1]; @@ -300,15 +342,17 @@ TEST(ColumnDynamic, InsertManyFromOverflow1) column_to->insertManyFrom(*column_from, 1, 2); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); field = (*column_to)[column_to->size() - 2]; - ASSERT_EQ(field, "42.42"); + ASSERT_EQ(field, 42.42); field = (*column_to)[column_to->size() - 1]; - ASSERT_EQ(field, "42.42"); + ASSERT_EQ(field, 42.42); column_to->insertManyFrom(*column_from, 2, 2); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column_to->getSharedVariant().size(), 4); field = (*column_to)[column_to->size() - 1]; ASSERT_EQ(field, "str"); field = (*column_to)[column_to->size() - 2]; @@ -323,8 +367,9 @@ TEST(ColumnDynamic, InsertManyFromOverflow2) auto column_to = getDynamicWithManyVariants(253); column_to->insertManyFrom(*column_from, 0, 2); - ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 254); + ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_EQ(column_to->getSharedVariant().size(), 0); auto field = (*column_to)[column_to->size() - 2]; ASSERT_EQ(field, 42); field = (*column_to)[column_to->size() - 1]; @@ -333,11 +378,39 @@ TEST(ColumnDynamic, InsertManyFromOverflow2) column_to->insertManyFrom(*column_from, 1, 2); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); field = (*column_to)[column_to->size() - 2]; - ASSERT_EQ(field, "42.42"); + ASSERT_EQ(field, 42.42); field = (*column_to)[column_to->size() - 1]; - ASSERT_EQ(field, "42.42"); + ASSERT_EQ(field, 42.42); +} + + +TEST(ColumnDynamic, InsertManyFromOverflow3) +{ + auto column_from = ColumnDynamic::create(1); + column_from->insert(Field(42)); + column_from->insert(Field(42.42)); + + auto column_to = ColumnDynamic::create(255); + column_to->insert(Field(41)); + + column_to->insertManyFrom(*column_from, 0, 2); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_EQ(column_to->getSharedVariant().size(), 0); + auto field = (*column_to)[column_to->size() - 2]; + ASSERT_EQ(field, 42); + field = (*column_to)[column_to->size() - 1]; + ASSERT_EQ(field, 42); + + column_to->insertManyFrom(*column_from, 1, 2); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); + field = (*column_to)[column_to->size() - 2]; + ASSERT_EQ(field, 42.42); + field = (*column_to)[column_to->size() - 1]; + ASSERT_EQ(field, 42.42); } void checkInsertRangeFrom(const ColumnDynamic::MutablePtr & column_from, ColumnDynamic::MutablePtr & column_to, const std::string & expected_variant, const std::vector & expected_names, const std::unordered_map & expected_variant_name_to_discriminator) @@ -369,7 +442,7 @@ void checkInsertRangeFrom(const ColumnDynamic::MutablePtr & column_from, ColumnD TEST(ColumnDynamic, InsertRangeFrom1) { auto column_to = ColumnDynamic::create(255); - checkInsertRangeFrom(getInsertFromColumn(2), column_to, "Variant(Float64, Int8, String)", {"Float64", "Int8", "String"}, {{"Float64", 0}, {"Int8", 1}, {"String", 2}}); + checkInsertRangeFrom(getInsertFromColumn(2), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertRangeFrom2) @@ -379,7 +452,7 @@ TEST(ColumnDynamic, InsertRangeFrom2) column_to->insert(Field(42.42)); column_to->insert(Field("str1")); - checkInsertRangeFrom(getInsertFromColumn(2), column_to, "Variant(Float64, Int8, String)", {"Float64", "Int8", "String"}, {{"Float64", 0}, {"Int8", 1}, {"String", 2}}); + checkInsertRangeFrom(getInsertFromColumn(2), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertRangeFrom3) @@ -390,7 +463,7 @@ TEST(ColumnDynamic, InsertRangeFrom3) column_to->insert(Field("str1")); column_to->insert(Array({Field(42)})); - checkInsertRangeFrom(getInsertFromColumn(2), column_to, "Variant(Array(Int8), Float64, Int8, String)", {"Array(Int8)", "Float64", "Int8", "String"}, {{"Array(Int8)", 0}, {"Float64", 1}, {"Int8", 2}, {"String", 3}}); + checkInsertRangeFrom(getInsertFromColumn(2), column_to, "Variant(Array(Int8), Float64, Int8, SharedVariant, String)", {"Array(Int8)", "Float64", "Int8", "SharedVariant", "String"}, {{"Array(Int8)", 0}, {"Float64", 1}, {"Int8", 2}, {"SharedVariant", 3}, {"String", 4}}); } TEST(ColumnDynamic, InsertRangeFromOverflow1) @@ -403,16 +476,18 @@ TEST(ColumnDynamic, InsertRangeFromOverflow1) auto column_to = getDynamicWithManyVariants(253); column_to->insertRangeFrom(*column_from, 0, 4); + ASSERT_EQ(column_to->size(), 257); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); auto field = (*column_to)[column_to->size() - 4]; ASSERT_EQ(field, Field(42)); field = (*column_to)[column_to->size() - 3]; ASSERT_EQ(field, Field(43)); field = (*column_to)[column_to->size() - 2]; - ASSERT_EQ(field, Field("42.42")); + ASSERT_EQ(field, Field(42.42)); field = (*column_to)[column_to->size() - 1]; ASSERT_EQ(field, Field("str")); } @@ -428,14 +503,15 @@ TEST(ColumnDynamic, InsertRangeFromOverflow2) column_to->insertRangeFrom(*column_from, 0, 3); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_EQ(column_to->getSharedVariant().size(), 1); auto field = (*column_to)[column_to->size() - 3]; ASSERT_EQ(field, Field(42)); field = (*column_to)[column_to->size() - 2]; ASSERT_EQ(field, Field(43)); field = (*column_to)[column_to->size() - 1]; - ASSERT_EQ(field, Field("42.42")); + ASSERT_EQ(field, Field(42.42)); } TEST(ColumnDynamic, InsertRangeFromOverflow3) @@ -449,15 +525,16 @@ TEST(ColumnDynamic, InsertRangeFromOverflow3) column_to->insert(Field("Str")); column_to->insertRangeFrom(*column_from, 0, 3); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_EQ(column_to->getSharedVariant().size(), 3); auto field = (*column_to)[column_to->size() - 3]; ASSERT_EQ(field, Field(42)); field = (*column_to)[column_to->size() - 2]; ASSERT_EQ(field, Field(43)); field = (*column_to)[column_to->size() - 1]; - ASSERT_EQ(field, Field("42.42")); + ASSERT_EQ(field, Field(42.42)); } TEST(ColumnDynamic, InsertRangeFromOverflow4) @@ -471,12 +548,13 @@ TEST(ColumnDynamic, InsertRangeFromOverflow4) column_to->insertRangeFrom(*column_from, 0, 3); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_EQ(column_to->getSharedVariant().size(), 3); auto field = (*column_to)[column_to->size() - 3]; - ASSERT_EQ(field, Field("42")); + ASSERT_EQ(field, Field(42)); field = (*column_to)[column_to->size() - 2]; - ASSERT_EQ(field, Field("42.42")); + ASSERT_EQ(field, Field(42.42)); field = (*column_to)[column_to->size() - 1]; ASSERT_EQ(field, Field("str")); } @@ -493,15 +571,16 @@ TEST(ColumnDynamic, InsertRangeFromOverflow5) column_to->insert(Field("str")); column_to->insertRangeFrom(*column_from, 0, 4); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_EQ(column_to->getSharedVariant().size(), 3); auto field = (*column_to)[column_to->size() - 4]; ASSERT_EQ(field, Field(42)); field = (*column_to)[column_to->size() - 3]; ASSERT_EQ(field, Field(43)); field = (*column_to)[column_to->size() - 2]; - ASSERT_EQ(field, Field("42.42")); + ASSERT_EQ(field, Field(42.42)); field = (*column_to)[column_to->size() - 1]; ASSERT_EQ(field, Field("str")); } @@ -520,13 +599,14 @@ TEST(ColumnDynamic, InsertRangeFromOverflow6) auto column_to = getDynamicWithManyVariants(253); column_to->insertRangeFrom(*column_from, 2, 5); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); - ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Array(Int8)")); + ASSERT_EQ(column_to->getSharedVariant().size(), 4); auto field = (*column_to)[column_to->size() - 5]; - ASSERT_EQ(field, Field("44")); + ASSERT_EQ(field, Field(44)); field = (*column_to)[column_to->size() - 4]; ASSERT_EQ(field, Field(42.42)); field = (*column_to)[column_to->size() - 3]; @@ -534,7 +614,131 @@ TEST(ColumnDynamic, InsertRangeFromOverflow6) field = (*column_to)[column_to->size() - 2]; ASSERT_EQ(field, Field("str")); field = (*column_to)[column_to->size() - 1]; - ASSERT_EQ(field, Field("[42]")); + ASSERT_EQ(field, Field(Array({Field(42)}))); +} + +TEST(ColumnDynamic, InsertRangeFromOverflow7) +{ + auto column_from = ColumnDynamic::create(3); + column_from->insert(Field(42.42)); + column_from->insert(Field("str1")); + column_from->insert(Field(42)); + column_from->insert(Field(43.43)); + column_from->insert(Field(Array({Field(41)}))); + column_from->insert(Field(43)); + column_from->insert(Field("str2")); + column_from->insert(Field(Array({Field(42)}))); + + auto column_to = ColumnDynamic::create(255); + column_to->insert(Field(42)); + + column_to->insertRangeFrom(*column_from, 0, 8); + ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 4); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Array(Int8)")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); + auto field = (*column_to)[column_to->size() - 8]; + ASSERT_EQ(field, Field(42.42)); + field = (*column_to)[column_to->size() - 7]; + ASSERT_EQ(field, Field("str1")); + field = (*column_to)[column_to->size() - 6]; + ASSERT_EQ(field, Field(42)); + field = (*column_to)[column_to->size() - 5]; + ASSERT_EQ(field, Field(43.43)); + field = (*column_to)[column_to->size() - 4]; + ASSERT_EQ(field, Field(Array({Field(41)}))); + field = (*column_to)[column_to->size() - 3]; + ASSERT_EQ(field, Field(43)); + field = (*column_to)[column_to->size() - 2]; + ASSERT_EQ(field, Field("str2")); + field = (*column_to)[column_to->size() - 1]; + ASSERT_EQ(field, Field(Array({Field(42)}))); +} + +TEST(ColumnDynamic, InsertRangeFromOverflow8) +{ + auto column_from = ColumnDynamic::create(3); + column_from->insert(Field(42.42)); + column_from->insert(Field("str1")); + column_from->insert(Field(42)); + column_from->insert(Field(43.43)); + column_from->insert(Field(Array({Field(41)}))); + column_from->insert(Field(43)); + column_from->insert(Field("str2")); + column_from->insert(Field(Array({Field(42)}))); + + auto column_to = ColumnDynamic::create(3); + column_to->insert(Field(42)); + column_from->insert(Field("str1")); + + column_to->insertRangeFrom(*column_from, 0, 8); + ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 3); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Array(Int8)")); + ASSERT_EQ(column_to->getSharedVariant().size(), 4); + auto field = (*column_to)[column_to->size() - 8]; + ASSERT_EQ(field, Field(42.42)); + field = (*column_to)[column_to->size() - 7]; + ASSERT_EQ(field, Field("str1")); + field = (*column_to)[column_to->size() - 6]; + ASSERT_EQ(field, Field(42)); + field = (*column_to)[column_to->size() - 5]; + ASSERT_EQ(field, Field(43.43)); + field = (*column_to)[column_to->size() - 4]; + ASSERT_EQ(field, Field(Array({Field(41)}))); + field = (*column_to)[column_to->size() - 3]; + ASSERT_EQ(field, Field(43)); + field = (*column_to)[column_to->size() - 2]; + ASSERT_EQ(field, Field("str2")); + field = (*column_to)[column_to->size() - 1]; + ASSERT_EQ(field, Field(Array({Field(42)}))); +} + +TEST(ColumnDynamic, InsertRangeFromOverflow9) +{ + auto column_from = ColumnDynamic::create(3); + column_from->insert(Field("str1")); + column_from->insert(Field(42.42)); + column_from->insert(Field("str2")); + column_from->insert(Field(42)); + column_from->insert(Field(43.43)); + column_from->insert(Field(Array({Field(41)}))); + column_from->insert(Field(43)); + column_from->insert(Field("str2")); + column_from->insert(Field(Array({Field(42)}))); + + auto column_to = ColumnDynamic::create(3); + column_to->insert(Field(42)); + + column_to->insertRangeFrom(*column_from, 0, 9); + ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 3); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Array(Int8)")); + ASSERT_EQ(column_to->getSharedVariant().size(), 4); + auto field = (*column_to)[column_to->size() - 9]; + ASSERT_EQ(field, Field("str1")); + field = (*column_to)[column_to->size() - 8]; + ASSERT_EQ(field, Field(42.42)); + field = (*column_to)[column_to->size() - 7]; + ASSERT_EQ(field, Field("str2")); + field = (*column_to)[column_to->size() - 6]; + ASSERT_EQ(field, Field(42)); + field = (*column_to)[column_to->size() - 5]; + ASSERT_EQ(field, Field(43.43)); + field = (*column_to)[column_to->size() - 4]; + ASSERT_EQ(field, Field(Array({Field(41)}))); + field = (*column_to)[column_to->size() - 3]; + ASSERT_EQ(field, Field(43)); + field = (*column_to)[column_to->size() - 2]; + ASSERT_EQ(field, Field("str2")); + field = (*column_to)[column_to->size() - 1]; + ASSERT_EQ(field, Field(Array({Field(42)}))); } TEST(ColumnDynamic, SerializeDeserializeFromArena1) @@ -583,18 +787,18 @@ TEST(ColumnDynamic, SerializeDeserializeFromArena2) pos = column_to->deserializeAndInsertFromArena(pos); column_to->deserializeAndInsertFromArena(pos); - ASSERT_EQ((*column_from)[column_from->size() - 4], 42); - ASSERT_EQ((*column_from)[column_from->size() - 3], 42.42); - ASSERT_EQ((*column_from)[column_from->size() - 2], "str"); - ASSERT_EQ((*column_from)[column_from->size() - 1], Null()); - ASSERT_EQ(column_to->getVariantInfo().variant_type->getName(), "Variant(Float64, Int8, String)"); - std::vector expected_names = {"Float64", "Int8", "String"}; + ASSERT_EQ((*column_to)[column_to->size() - 4], 42); + ASSERT_EQ((*column_to)[column_to->size() - 3], 42.42); + ASSERT_EQ((*column_to)[column_to->size() - 2], "str"); + ASSERT_EQ((*column_to)[column_to->size() - 1], Null()); + ASSERT_EQ(column_to->getVariantInfo().variant_type->getName(), "Variant(Float64, Int8, SharedVariant, String)"); + std::vector expected_names = {"Float64", "Int8", "SharedVariant", "String"}; ASSERT_EQ(column_to->getVariantInfo().variant_names, expected_names); - std::unordered_map expected_variant_name_to_discriminator = {{"Float64", 0}, {"Int8", 1}, {"String", 2}}; + std::unordered_map expected_variant_name_to_discriminator = {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}; ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator == expected_variant_name_to_discriminator); } -TEST(ColumnDynamic, SerializeDeserializeFromArenaOverflow) +TEST(ColumnDynamic, SerializeDeserializeFromArenaOverflow1) { auto column_from = ColumnDynamic::create(255); column_from->insert(Field(42)); @@ -615,18 +819,56 @@ TEST(ColumnDynamic, SerializeDeserializeFromArenaOverflow) pos = column_to->deserializeAndInsertFromArena(pos); column_to->deserializeAndInsertFromArena(pos); - ASSERT_EQ((*column_from)[column_from->size() - 4], 42); - ASSERT_EQ((*column_from)[column_from->size() - 3], 42.42); - ASSERT_EQ((*column_from)[column_from->size() - 2], "str"); - ASSERT_EQ((*column_from)[column_from->size() - 1], Null()); + ASSERT_EQ((*column_to)[column_to->size() - 4], 42); + ASSERT_EQ((*column_to)[column_to->size() - 3], 42.42); + ASSERT_EQ((*column_to)[column_to->size() - 2], "str"); + ASSERT_EQ((*column_to)[column_to->size() - 1], Null()); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); +} + +TEST(ColumnDynamic, SerializeDeserializeFromArenaOverflow2) +{ + auto column_from = ColumnDynamic::create(2); + column_from->insert(Field(42)); + column_from->insert(Field(42.42)); + column_from->insert(Field("str")); + column_from->insert(Field(Null())); + column_from->insert(Field(Array({Field(42)}))); + + Arena arena; + const char * pos = nullptr; + auto ref1 = column_from->serializeValueIntoArena(0, arena, pos); + column_from->serializeValueIntoArena(1, arena, pos); + column_from->serializeValueIntoArena(2, arena, pos); + column_from->serializeValueIntoArena(3, arena, pos); + column_from->serializeValueIntoArena(4, arena, pos); + + auto column_to = ColumnDynamic::create(3); + column_to->insert(Field(42.42)); + pos = column_to->deserializeAndInsertFromArena(ref1.data); + pos = column_to->deserializeAndInsertFromArena(pos); + pos = column_to->deserializeAndInsertFromArena(pos); + pos = column_to->deserializeAndInsertFromArena(pos); + column_to->deserializeAndInsertFromArena(pos); + + ASSERT_EQ((*column_to)[column_to->size() - 5], 42); + ASSERT_EQ((*column_to)[column_to->size() - 4], 42.42); + ASSERT_EQ((*column_to)[column_to->size() - 3], "str"); + ASSERT_EQ((*column_to)[column_to->size() - 2], Null()); + ASSERT_EQ((*column_to)[column_to->size() - 1], Field(Array({Field(42)}))); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Array(Int8)")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); } TEST(ColumnDynamic, skipSerializedInArena) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(3); column_from->insert(Field(42)); column_from->insert(Field(42.42)); column_from->insert(Field("str")); @@ -647,6 +889,34 @@ TEST(ColumnDynamic, skipSerializedInArena) pos = column_to->skipSerializedInArena(pos); ASSERT_EQ(pos, end); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.empty()); - ASSERT_TRUE(column_to->getVariantInfo().variant_names.empty()); + ASSERT_EQ(column_to->getVariantInfo().variant_name_to_discriminator.at("SharedVariant"), 0); + ASSERT_EQ(column_to->getVariantInfo().variant_names, Names{"SharedVariant"}); +} + +TEST(ColumnDynamic, compare) +{ + auto column_from = ColumnDynamic::create(3); + column_from->insert(Field(42)); + column_from->insert(Field(42.42)); + column_from->insert(Field("str")); + column_from->insert(Field(Null())); + column_from->insert(Field(Array({Field(42)}))); + + ASSERT_EQ(column_from->compareAt(0, 0, *column_from, -1), 0); + ASSERT_EQ(column_from->compareAt(0, 1, *column_from, -1), 1); + ASSERT_EQ(column_from->compareAt(1, 1, *column_from, -1), 0); + ASSERT_EQ(column_from->compareAt(0, 2, *column_from, -1), -1); + ASSERT_EQ(column_from->compareAt(2, 0, *column_from, -1), 1); + ASSERT_EQ(column_from->compareAt(2, 4, *column_from, -1), 1); + ASSERT_EQ(column_from->compareAt(4, 2, *column_from, -1), -1); + ASSERT_EQ(column_from->compareAt(4, 4, *column_from, -1), 0); + ASSERT_EQ(column_from->compareAt(0, 3, *column_from, -1), 1); + ASSERT_EQ(column_from->compareAt(1, 3, *column_from, -1), 1); + ASSERT_EQ(column_from->compareAt(2, 3, *column_from, -1), 1); + ASSERT_EQ(column_from->compareAt(3, 3, *column_from, -1), 0); + ASSERT_EQ(column_from->compareAt(4, 3, *column_from, -1), 1); + ASSERT_EQ(column_from->compareAt(3, 0, *column_from, -1), -1); + ASSERT_EQ(column_from->compareAt(3, 1, *column_from, -1), -1); + ASSERT_EQ(column_from->compareAt(3, 2, *column_from, -1), -1); + ASSERT_EQ(column_from->compareAt(3, 4, *column_from, -1), -1); } diff --git a/src/DataTypes/DataTypeDynamic.cpp b/src/DataTypes/DataTypeDynamic.cpp index a1b1f8325f0..e00638a50ab 100644 --- a/src/DataTypes/DataTypeDynamic.cpp +++ b/src/DataTypes/DataTypeDynamic.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -14,6 +15,7 @@ #include #include #include +#include namespace DB { @@ -71,8 +73,8 @@ static DataTypePtr create(const ASTPtr & arguments) auto * literal = argument->arguments->children[1]->as(); - if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.get() == 0 || literal->value.get() > ColumnVariant::MAX_NESTED_COLUMNS) - throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'max_types' argument for Dynamic type should be a positive integer between 1 and 255"); + if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.get() > ColumnVariant::MAX_NESTED_COLUMNS - 1) + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'max_types' argument for Dynamic type should be a positive integer between 0 and 254"); return std::make_shared(literal->value.get()); } @@ -84,30 +86,72 @@ void registerDataTypeDynamic(DataTypeFactory & factory) std::unique_ptr DataTypeDynamic::getDynamicSubcolumnData(std::string_view subcolumn_name, const DB::IDataType::SubstreamData & data, bool throw_if_null) const { - auto [subcolumn_type_name, subcolumn_nested_name] = Nested::splitName(subcolumn_name); + auto [type_subcolumn_name, subcolumn_nested_name] = Nested::splitName(subcolumn_name); /// Check if requested subcolumn is a valid data type. - auto subcolumn_type = DataTypeFactory::instance().tryGet(String(subcolumn_type_name)); + auto subcolumn_type = DataTypeFactory::instance().tryGet(String(type_subcolumn_name)); if (!subcolumn_type) { if (throw_if_null) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Dynamic type doesn't have subcolumn '{}'", subcolumn_type_name); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Dynamic type doesn't have subcolumn '{}'", type_subcolumn_name); return nullptr; } std::unique_ptr res = std::make_unique(subcolumn_type->getDefaultSerialization()); res->type = subcolumn_type; std::optional discriminator; + ColumnPtr null_map_for_variant_from_shared_variant; if (data.column) { /// If column was provided, we should extract subcolumn from Dynamic column. const auto & dynamic_column = assert_cast(*data.column); const auto & variant_info = dynamic_column.getVariantInfo(); + const auto & variant_column = dynamic_column.getVariantColumn(); + const auto & shared_variant = dynamic_column.getSharedVariant(); /// Check if provided Dynamic column has subcolumn of this type. - auto it = variant_info.variant_name_to_discriminator.find(subcolumn_type->getName()); + String subcolumn_type_name = subcolumn_type->getName(); + auto it = variant_info.variant_name_to_discriminator.find(subcolumn_type_name); if (it != variant_info.variant_name_to_discriminator.end()) { discriminator = it->second; - res->column = dynamic_column.getVariantColumn().getVariantPtrByGlobalDiscriminator(*discriminator); + res->column = variant_column.getVariantPtrByGlobalDiscriminator(*discriminator); + } + /// Otherwise if there is data in shared variant try to find requested type there. + else if (!shared_variant.empty()) + { + /// Create null map for resulting subcolumn to make it Nullable. + auto null_map_column = ColumnUInt8::create(); + NullMap & null_map = assert_cast(*null_map_column).getData(); + null_map.reserve(variant_column.size()); + auto subcolumn = subcolumn_type->createColumn(); + auto shared_variant_local_discr = variant_column.localDiscriminatorByGlobal(dynamic_column.getSharedVariantDiscriminator()); + const auto & local_discriminators = variant_column.getLocalDiscriminators(); + const auto & offsets = variant_column.getOffsets(); + const FormatSettings format_settings; + for (size_t i = 0; i != local_discriminators.size(); ++i) + { + if (local_discriminators[i] == shared_variant_local_discr) + { + auto value = shared_variant.getDataAt(offsets[i]); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + if (type->getName() == subcolumn_type_name) + { + dynamic_column.getVariantSerialization(subcolumn_type, subcolumn_type_name)->deserializeBinary(*subcolumn, buf, format_settings); + null_map.push_back(0); + } + else + { + null_map.push_back(1); + } + } + else + { + null_map.push_back(1); + } + } + + res->column = std::move(subcolumn); + null_map_for_variant_from_shared_variant = std::move(null_map_column); } } @@ -125,7 +169,7 @@ std::unique_ptr DataTypeDynamic::getDynamicSubcolumnDa return nullptr; } - res->serialization = std::make_shared(res->serialization, subcolumn_type->getName(), is_null_map_subcolumn); + res->serialization = std::make_shared(res->serialization, subcolumn_type->getName(), String(subcolumn_nested_name), is_null_map_subcolumn); /// Make resulting subcolumn Nullable only if type subcolumn can be inside Nullable or can be LowCardinality(Nullable()). bool make_subcolumn_nullable = subcolumn_type->canBeInsideNullable() || subcolumn_type->lowCardinality(); if (!is_null_map_subcolumn && make_subcolumn_nullable) @@ -133,10 +177,10 @@ std::unique_ptr DataTypeDynamic::getDynamicSubcolumnDa if (data.column) { + /// Check if provided Dynamic column has subcolumn of this type. In this case we should use VariantSubcolumnCreator/VariantNullMapSubcolumnCreator to + /// create full subcolumn from variant according to discriminators. if (discriminator) { - /// Provided Dynamic column has subcolumn of this type, we should use VariantSubcolumnCreator/VariantNullMapSubcolumnCreator to - /// create full subcolumn from variant according to discriminators. const auto & variant_column = assert_cast(*data.column).getVariantColumn(); std::unique_ptr creator; if (is_null_map_subcolumn) @@ -154,6 +198,21 @@ std::unique_ptr DataTypeDynamic::getDynamicSubcolumnDa make_subcolumn_nullable); res->column = creator->create(res->column); } + /// Check if requested type was extracted from shared variant. In this case we should use + /// VariantSubcolumnCreator to create full subcolumn from variant according to created null map. + else if (null_map_for_variant_from_shared_variant) + { + if (is_null_map_subcolumn) + { + res->column = null_map_for_variant_from_shared_variant; + } + else + { + SerializationVariantElement::VariantSubcolumnCreator creator( + null_map_for_variant_from_shared_variant, "", 0, 0, make_subcolumn_nullable, null_map_for_variant_from_shared_variant); + res->column = creator.create(res->column); + } + } /// Provided Dynamic column doesn't have subcolumn of this type, just create column filled with default values. else if (is_null_map_subcolumn) { diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 6f7dcd65b83..ca2ebdfbdbb 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -150,6 +150,12 @@ DataTypePtr DataTypeFactory::getCustom(DataTypeCustomDescPtr customization) cons return type; } +DataTypePtr DataTypeFactory::getCustom(const String & base_name, DataTypeCustomDescPtr customization) const +{ + auto type = get(base_name); + type->setCustomization(std::move(customization)); + return type; +} void DataTypeFactory::registerDataType(const String & family_name, Value creator, Case case_sensitiveness) { diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index edba9886d1c..a8324341691 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -34,6 +34,7 @@ public: DataTypePtr get(const String & family_name, const ASTPtr & parameters) const; DataTypePtr get(const ASTPtr & ast) const; DataTypePtr getCustom(DataTypeCustomDescPtr customization) const; + DataTypePtr getCustom(const String & base_name, DataTypeCustomDescPtr customization) const; /// Return nullptr in case of error. DataTypePtr tryGet(const String & full_name) const; diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index e96937d522d..67b4a0a5e31 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -192,17 +192,12 @@ MutableColumnPtr DataTypeTuple::createColumn() const MutableColumnPtr DataTypeTuple::createColumn(const ISerialization & serialization) const { - /// If we read Tuple as Variant subcolumn, it may be wrapped to SerializationVariantElement. - /// Here we don't need it, so we drop this wrapper. - const auto * current_serialization = &serialization; - while (const auto * serialization_variant_element = typeid_cast(current_serialization)) - current_serialization = serialization_variant_element->getNested().get(); - - /// If we read subcolumn of nested Tuple, it may be wrapped to SerializationNamed + /// If we read subcolumn of nested Tuple or this Tuple is a subcolumn, it may be wrapped to SerializationWrapper /// several times to allow to reconstruct the substream path name. /// Here we don't need substream path name, so we drop first several wrapper serializations. - while (const auto * serialization_named = typeid_cast(current_serialization)) - current_serialization = serialization_named->getNested().get(); + const auto * current_serialization = &serialization; + while (const auto * serialization_wrapper = dynamic_cast(current_serialization)) + current_serialization = serialization_wrapper->getNested().get(); const auto * serialization_tuple = typeid_cast(current_serialization); if (!serialization_tuple) diff --git a/src/DataTypes/DataTypesBinaryEncoding.cpp b/src/DataTypes/DataTypesBinaryEncoding.cpp index bd994e313ba..610f246265e 100644 --- a/src/DataTypes/DataTypesBinaryEncoding.cpp +++ b/src/DataTypes/DataTypesBinaryEncoding.cpp @@ -444,7 +444,7 @@ void encodeDataType(const DataTypePtr & type, WriteBuffer & buf) case BinaryTypeIndex::Dynamic: { const auto & dynamic_type = assert_cast(*type); - /// Maximum number of dynamic types is 255, we can write it as 1 byte. + /// Maximum number of dynamic types is 254, we can write it as 1 byte. writeBinary(UInt8(dynamic_type.getMaxDynamicTypes()), buf); break; } diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index 7609ffc91ca..67b29750948 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -27,15 +27,21 @@ namespace ErrorCodes struct SerializeBinaryBulkStateDynamic : public ISerialization::SerializeBinaryBulkState { SerializationDynamic::DynamicStructureSerializationVersion structure_version; + size_t max_dynamic_types; DataTypePtr variant_type; Names variant_names; SerializationPtr variant_serialization; ISerialization::SerializeBinaryBulkStatePtr variant_state; - /// Variants statistics. Map (Variant name) -> (Variant size). - ColumnDynamic::Statistics statistics = { .source = ColumnDynamic::Statistics::Source::READ, .data = {} }; + /// Variants statistics. + ColumnDynamic::Statistics statistics; + /// If true, statistics will be recalculated during serialization. + bool recalculate_statistics = false; - explicit SerializeBinaryBulkStateDynamic(UInt64 structure_version_) : structure_version(structure_version_) {} + explicit SerializeBinaryBulkStateDynamic(UInt64 structure_version_) + : structure_version(structure_version_), statistics(ColumnDynamic::Statistics::Source::READ) + { + } }; struct DeserializeBinaryBulkStateDynamic : public ISerialization::DeserializeBinaryBulkState @@ -106,20 +112,41 @@ void SerializationDynamic::serializeBinaryBulkStatePrefix( writeBinaryLittleEndian(structure_version, *stream); auto dynamic_state = std::make_shared(structure_version); + dynamic_state->max_dynamic_types = column_dynamic.getMaxDynamicTypes(); + /// Write max_dynamic_types parameter, because it can differ from the max_dynamic_types + /// that is specified in the Dynamic type (we could decrease it before merge). + writeBinaryLittleEndian(dynamic_state->max_dynamic_types, *stream); + dynamic_state->variant_type = variant_info.variant_type; dynamic_state->variant_names = variant_info.variant_names; const auto & variant_column = column_dynamic.getVariantColumn(); - /// Write internal Variant type name. + /// Write information about variants. + size_t num_variants = dynamic_state->variant_names.size() - 1; /// Don't write shared variant, Dynamic column should always have it. + writeBinaryLittleEndian(num_variants, *stream); if (settings.data_types_binary_encoding) - encodeDataType(dynamic_state->variant_type, *stream); + { + const auto & variants = assert_cast(*dynamic_state->variant_type).getVariants(); + for (const auto & variant: variants) + { + if (variant->getName() != ColumnDynamic::getSharedVariantTypeName()) + encodeDataType(dynamic_state->variant_type, *stream); + } + } else - writeStringBinary(dynamic_state->variant_type->getName(), *stream); + { + for (const auto & name : dynamic_state->variant_names) + { + if (name != ColumnDynamic::getSharedVariantTypeName()) + writeStringBinary(name, *stream); + } + } /// Write statistics in prefix if needed. if (settings.dynamic_write_statistics == SerializeBinaryBulkSettings::DynamicStatisticsMode::PREFIX) { const auto & statistics = column_dynamic.getStatistics(); + /// First, write statistics for usual variants. for (size_t i = 0; i != variant_info.variant_names.size(); ++i) { size_t size = 0; @@ -129,13 +156,55 @@ void SerializationDynamic::serializeBinaryBulkStatePrefix( /// - statistics read from the data part during deserialization of Dynamic column (Statistics::Source::READ). /// We can rely only on statistics calculated during the merge, because column with statistics that was read /// during deserialization from some data part could be filtered/limited/transformed/etc and so the statistics can be outdated. - if (!statistics.data.empty() && statistics.source == ColumnDynamic::Statistics::Source::MERGE) - size = statistics.data.at(variant_info.variant_names[i]); + if (statistics && statistics->source == ColumnDynamic::Statistics::Source::MERGE) + size = statistics->variants_statistics.at(variant_info.variant_names[i]); /// Otherwise we can use only variant sizes from current column. else size = variant_column.getVariantByGlobalDiscriminator(i).size(); writeVarUInt(size, *stream); } + + /// Second, write statistics for variants in shared variant. + /// Check if we have statistics calculated during merge of some data parts (Statistics::Source::MERGE). + if (statistics && statistics->source == ColumnDynamic::Statistics::Source::MERGE) + { + writeVarUInt(statistics->shared_variants_statistics.size(), *stream); + for (const auto & [variant_name, size] : statistics->shared_variants_statistics) + { + writeStringBinary(variant_name, *stream); + writeVarUInt(size, *stream); + } + } + /// If we don't have statistics for shared variants from merge, calculate it from the column. + else + { + std::unordered_map shared_variants_statistics; + const auto & shared_variant = column_dynamic.getSharedVariant(); + for (size_t i = 0; i != shared_variant.size(); ++i) + { + auto value = shared_variant.getDataAt(i); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + auto type_name = type->getName(); + if (auto it = shared_variants_statistics.find(type_name); it != shared_variants_statistics.end()) + ++it->second; + else if (shared_variants_statistics.size() < ColumnDynamic::Statistics::MAX_SHARED_VARIANT_STATISTICS_SIZE) + shared_variants_statistics.emplace(type_name, 1); + } + + writeVarUInt(shared_variants_statistics.size(), *stream); + for (const auto & [variant_name, size] : shared_variants_statistics) + { + writeStringBinary(variant_name, *stream); + writeVarUInt(size, *stream); + } + } + } + /// Otherwise statistics will be written in the suffix, in this case we will recalculate + /// statistics during serialization to make it more precise. + else + { + dynamic_state->recalculate_statistics = true; } dynamic_state->variant_serialization = dynamic_state->variant_type->getDefaultSerialization(); @@ -182,33 +251,58 @@ ISerialization::DeserializeBinaryBulkStatePtr SerializationDynamic::deserializeD UInt64 structure_version; readBinaryLittleEndian(structure_version, *structure_stream); auto structure_state = std::make_shared(structure_version); - /// Read internal Variant type name. + /// Read max_dynamic_types parameter. + readBinaryLittleEndian(structure_state->max_dynamic_types, *structure_stream); + /// Read information about variants. + DataTypes variants; + size_t num_variants; + readBinaryLittleEndian(num_variants, *structure_stream); + variants.reserve(num_variants + 1); /// +1 for shared variant. if (settings.data_types_binary_encoding) { - structure_state->variant_type = decodeDataType(*structure_stream); + for (size_t i = 0; i != num_variants; ++i) + variants.push_back(decodeDataType(*structure_stream)); } else { String data_type_name; - readStringBinary(data_type_name, *structure_stream); - structure_state->variant_type = DataTypeFactory::instance().get(data_type_name); + for (size_t i = 0; i != num_variants; ++i) + { + readStringBinary(data_type_name, *structure_stream); + variants.push_back(DataTypeFactory::instance().get(data_type_name)); + } } - const auto * variant_type = typeid_cast(structure_state->variant_type.get()); - if (!variant_type) - throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect type of Dynamic nested column, expected Variant, got {}", structure_state->variant_type->getName()); + /// Add shared variant, Dynamic column should always have it. + variants.push_back(ColumnDynamic::getSharedVariantDataType()); + auto variant_type = std::make_shared(variants); /// Read statistics. if (settings.dynamic_read_statistics) { - const auto & variants = variant_type->getVariants(); + ColumnDynamic::Statistics statistics(ColumnDynamic::Statistics::Source::READ); + /// First, read statistics for usual variants. size_t variant_size; - for (const auto & variant : variants) + for (const auto & variant : variant_type->getVariants()) { readVarUInt(variant_size, *structure_stream); - structure_state->statistics.data[variant->getName()] = variant_size; + statistics.variants_statistics[variant->getName()] = variant_size; } + + /// Second, rend statistics for shared variants. + size_t statistics_size; + readVarUInt(statistics_size, *structure_stream); + String variant_name; + for (size_t i = 0; i != statistics_size; ++i) + { + readStringBinary(variant_name, *structure_stream); + readVarUInt(variant_size, *structure_stream); + statistics.shared_variants_statistics[variant_name] = variant_size; + } + + structure_state->statistics = std::make_shared(std::move(statistics)); } + structure_state->variant_type = std::move(variant_type); state = structure_state; addToSubstreamsDeserializeStatesCache(cache, settings.path, state); } @@ -231,8 +325,16 @@ void SerializationDynamic::serializeBinaryBulkStateSuffix( /// Write statistics in suffix if needed. if (settings.dynamic_write_statistics == SerializeBinaryBulkSettings::DynamicStatisticsMode::SUFFIX) { + /// First, write statistics for usual variants. for (const auto & variant_name : dynamic_state->variant_names) - writeVarUInt(dynamic_state->statistics.data[variant_name], *stream); + writeVarUInt(dynamic_state->statistics.variants_statistics[variant_name], *stream); + /// Second, write statistics for shared variants. + writeVarUInt(dynamic_state->statistics.shared_variants_statistics.size(), *stream); + for (const auto & [variant_name, size] : dynamic_state->statistics.shared_variants_statistics) + { + writeStringBinary(variant_name, *stream); + writeVarUInt(size, *stream); + } } settings.path.push_back(Substream::DynamicData); @@ -255,9 +357,42 @@ void SerializationDynamic::serializeBinaryBulkWithMultipleStreams( if (!variant_info.variant_type->equals(*dynamic_state->variant_type)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mismatch of internal columns of Dynamic. Expected: {}, Got: {}", dynamic_state->variant_type->getName(), variant_info.variant_type->getName()); + if (column_dynamic.getMaxDynamicTypes() != dynamic_state->max_dynamic_types) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Mismatch of max_dynamic_types parameter of Dynamic. Expected: {}, Got: {}", dynamic_state->max_dynamic_types, column_dynamic.getMaxDynamicTypes()); + settings.path.push_back(Substream::DynamicData); - assert_cast(*dynamic_state->variant_serialization) - .serializeBinaryBulkWithMultipleStreamsAndUpdateVariantStatistics(*variant_column, offset, limit, settings, dynamic_state->variant_state, dynamic_state->statistics.data); + if (dynamic_state->recalculate_statistics) + { + assert_cast(*dynamic_state->variant_serialization) + .serializeBinaryBulkWithMultipleStreamsAndUpdateVariantStatistics(*variant_column, offset, limit, settings, dynamic_state->variant_state, dynamic_state->statistics.variants_statistics); + /// Calculate statistics for shared variants. + const auto & shared_variant = column_dynamic.getSharedVariant(); + if (!shared_variant.empty()) + { + const auto & local_discriminators = variant_column->getLocalDiscriminators(); + const auto & offsets = variant_column->getOffsets(); + const auto shared_variant_discr = variant_column->localDiscriminatorByGlobal(column_dynamic.getSharedVariantDiscriminator()); + size_t end = limit == 0 || offset + limit > local_discriminators.size() ? local_discriminators.size() : offset + limit; + for (size_t i = offset; i != end; ++i) + { + if (local_discriminators[i] == shared_variant_discr) + { + auto value = shared_variant.getDataAt(offsets[i]); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + auto type_name = type->getName(); + if (auto it = dynamic_state->statistics.shared_variants_statistics.find(type_name); it != dynamic_state->statistics.shared_variants_statistics.end()) + ++it->second; + else if (dynamic_state->statistics.shared_variants_statistics.size() < ColumnDynamic::Statistics::MAX_SHARED_VARIANT_STATISTICS_SIZE) + dynamic_state->statistics.shared_variants_statistics.emplace(type_name, 1); + } + } + } + } + else + { + assert_cast(*dynamic_state->variant_serialization).serializeBinaryBulkWithMultipleStreams(*variant_column, offset, limit, settings, dynamic_state->variant_state); + } settings.path.pop_back(); } @@ -272,13 +407,17 @@ void SerializationDynamic::deserializeBinaryBulkWithMultipleStreams( return; auto mutable_column = column->assumeMutable(); + auto & column_dynamic = assert_cast(*mutable_column); auto * dynamic_state = checkAndGetState(state); auto * structure_state = checkAndGetState(dynamic_state->structure_state); if (mutable_column->empty()) - mutable_column = ColumnDynamic::create(structure_state->variant_type->createColumn(), structure_state->variant_type, max_dynamic_types, structure_state->statistics); + { + column_dynamic.setMaxDynamicPaths(structure_state->max_dynamic_types); + column_dynamic.setVariantType(structure_state->variant_type); + column_dynamic.setStatistics(structure_state->statistics); + } - auto & column_dynamic = assert_cast(*mutable_column); const auto & variant_info = column_dynamic.getVariantInfo(); if (!variant_info.variant_type->equals(*structure_state->variant_type)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mismatch of internal columns of Dynamic. Expected: {}, Got: {}", structure_state->variant_type->getName(), variant_info.variant_type->getName()); @@ -329,24 +468,42 @@ void SerializationDynamic::serializeBinary(const IColumn & column, size_t row_nu encodeDataType(std::make_shared(), ostr); return; } + /// Check if this value is in shared variant. In this case it's already + /// in desired binary format. + else if (global_discr == dynamic_column.getSharedVariantDiscriminator()) + { + auto value = dynamic_column.getSharedVariant().getDataAt(variant_column.offsetAt(row_num)); + ostr.write(value.data, value.size); + return; + } const auto & variant_type = assert_cast(*variant_info.variant_type).getVariant(global_discr); + const auto & variant_type_name = variant_info.variant_names[global_discr]; encodeDataType(variant_type, ostr); - variant_type->getDefaultSerialization()->serializeBinary(variant_column.getVariantByGlobalDiscriminator(global_discr), variant_column.offsetAt(row_num), ostr, settings); + dynamic_column.getVariantSerialization(variant_type, variant_type_name)->serializeBinary(variant_column.getVariantByGlobalDiscriminator(global_discr), variant_column.offsetAt(row_num), ostr, settings); } -template -static void deserializeVariant( +template +static ReturnType deserializeVariant( ColumnVariant & variant_column, - const DataTypePtr & variant_type, + const SerializationPtr & variant_serialization, ColumnVariant::Discriminator global_discr, ReadBuffer & istr, DeserializeFunc deserialize) { auto & variant = variant_column.getVariantByGlobalDiscriminator(global_discr); - deserialize(*variant_type->getDefaultSerialization(), variant, istr); + if constexpr (std::is_same_v) + { + if (!deserialize(*variant_serialization, variant, istr)) + return ReturnType(false); + } + else + { + deserialize(*variant_serialization, variant, istr); + } variant_column.getLocalDiscriminators().push_back(variant_column.localDiscriminatorByGlobal(global_discr)); variant_column.getOffsets().push_back(variant.size() - 1); + return ReturnType(true); } void SerializationDynamic::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -360,11 +517,12 @@ void SerializationDynamic::deserializeBinary(IColumn & column, ReadBuffer & istr } auto variant_type_name = variant_type->getName(); + const auto & variant_serialization = dynamic_column.getVariantSerialization(variant_type, variant_type_name); const auto & variant_info = dynamic_column.getVariantInfo(); auto it = variant_info.variant_name_to_discriminator.find(variant_type_name); if (it != variant_info.variant_name_to_discriminator.end()) { - deserializeVariant(dynamic_column.getVariantColumn(), variant_type, it->second, istr, [&settings](const ISerialization & serialization, IColumn & variant, ReadBuffer & buf){ serialization.deserializeBinary(variant, buf, settings); }); + deserializeVariant(dynamic_column.getVariantColumn(), variant_serialization, it->second, istr, [&settings](const ISerialization & serialization, IColumn & variant, ReadBuffer & buf){ serialization.deserializeBinary(variant, buf, settings); }); return; } @@ -372,25 +530,15 @@ void SerializationDynamic::deserializeBinary(IColumn & column, ReadBuffer & istr if (dynamic_column.addNewVariant(variant_type)) { auto discr = variant_info.variant_name_to_discriminator.at(variant_type_name); - deserializeVariant(dynamic_column.getVariantColumn(), variant_type, discr, istr, [&settings](const ISerialization & serialization, IColumn & variant, ReadBuffer & buf){ serialization.deserializeBinary(variant, buf, settings); }); + deserializeVariant(dynamic_column.getVariantColumn(), variant_serialization, discr, istr, [&settings](const ISerialization & serialization, IColumn & variant, ReadBuffer & buf){ serialization.deserializeBinary(variant, buf, settings); }); return; } /// We reached maximum number of variants and couldn't add new variant. - /// This case should be really rare in real use cases. - /// We should always be able to add String variant and insert value as String. - dynamic_column.addStringVariant(); + /// In this case we insert this value into shared variant in binary form. auto tmp_variant_column = variant_type->createColumn(); - variant_type->getDefaultSerialization()->deserializeBinary(*tmp_variant_column, istr, settings); - auto string_column = castColumn(ColumnWithTypeAndName(tmp_variant_column->getPtr(), variant_type, ""), std::make_shared()); - auto & variant_column = dynamic_column.getVariantColumn(); - variant_column.insertIntoVariantFrom(variant_info.variant_name_to_discriminator.at("String"), *string_column, 0); -} - -void SerializationDynamic::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - const auto & dynamic_column = assert_cast(column); - dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeTextCSV(dynamic_column.getVariantColumn(), row_num, ostr, settings); + variant_serialization->deserializeBinary(*tmp_variant_column, istr, settings); + dynamic_column.insertValueIntoSharedVariant(*tmp_variant_column, variant_type, variant_type_name, 0); } template @@ -406,6 +554,7 @@ static void deserializeTextImpl( auto & dynamic_column = assert_cast(column); auto & variant_column = dynamic_column.getVariantColumn(); const auto & variant_info = dynamic_column.getVariantInfo(); + const auto & variant_types = assert_cast(*variant_info.variant_type).getVariants(); String field = read_field(istr); auto field_buf = std::make_unique(field); JSONInferenceInfo json_info; @@ -413,27 +562,81 @@ static void deserializeTextImpl( if (escaping_rule == FormatSettings::EscapingRule::JSON) transformFinalInferredJSONTypeIfNeeded(variant_type, settings, &json_info); - if (checkIfTypeIsComplete(variant_type) && dynamic_column.addNewVariant(variant_type)) + /// If inferred type is not complete, we cannot add it as a new variant. + /// Let's try to deserialize this field into existing variants. + /// If failed, insert this value as String. + if (!checkIfTypeIsComplete(variant_type)) + { + size_t shared_variant_discr = dynamic_column.getSharedVariantDiscriminator(); + for (size_t i = 0; i != variant_types.size(); ++i) + { + field_buf = std::make_unique(field); + if (i != shared_variant_discr + && deserializeVariant( + variant_column, + dynamic_column.getVariantSerialization(variant_types[i], variant_info.variant_names[i]), + i, + *field_buf, + try_deserialize_variant)) + return; + } + + variant_type = std::make_shared(); + /// To be able to deserialize field as String with Quoted escaping rule, it should be quoted. + if (escaping_rule == FormatSettings::EscapingRule::Quoted && (field.size() < 2 || field.front() != '\'' || field.back() != '\'')) + field = "'" + field + "'"; + } + else if (dynamic_column.addNewVariant(variant_type, variant_type->getName())) { auto discr = variant_info.variant_name_to_discriminator.at(variant_type->getName()); - deserializeVariant(dynamic_column.getVariantColumn(), variant_type, discr, *field_buf, deserialize_variant); + deserializeVariant(dynamic_column.getVariantColumn(), dynamic_column.getVariantSerialization(variant_type), discr, *field_buf, deserialize_variant); return; } - /// We couldn't infer type or add new variant. Try to insert field into current variants. + /// We couldn't infer type or add new variant. Insert it into shared variant. + auto tmp_variant_column = variant_type->createColumn(); field_buf = std::make_unique(field); - if (try_deserialize_variant(*variant_info.variant_type->getDefaultSerialization(), variant_column, *field_buf)) - return; + auto variant_type_name = variant_type->getName(); + deserialize_variant(*dynamic_column.getVariantSerialization(variant_type, variant_type_name), *tmp_variant_column, *field_buf); + dynamic_column.insertValueIntoSharedVariant(*tmp_variant_column, variant_type, variant_type_name, 0); +} - /// We couldn't insert field into any existing variant, add String variant and read value as String. - dynamic_column.addStringVariant(); +template +static void serializeTextImpl( + const IColumn & column, + size_t row_num, + WriteBuffer & ostr, + const FormatSettings & settings, + NestedSerialize nested_serialize) +{ + const auto & dynamic_column = assert_cast(column); + const auto & variant_column = dynamic_column.getVariantColumn(); + /// Check if this row has value in shared variant. In this case we should first deserialize it from binary format. + if (variant_column.globalDiscriminatorAt(row_num) == dynamic_column.getSharedVariantDiscriminator()) + { + auto value = dynamic_column.getSharedVariant().getDataAt(variant_column.offsetAt(row_num)); + ReadBufferFromMemory buf(value.data, value.size); + auto variant_type = decodeDataType(buf); + auto tmp_variant_column = variant_type->createColumn(); + auto variant_serialization = dynamic_column.getVariantSerialization(variant_type); + variant_serialization->deserializeBinary(*tmp_variant_column, buf, settings); + nested_serialize(*variant_serialization, *tmp_variant_column, 0, ostr); + } + /// Otherwise just use serialization for Variant. + else + { + nested_serialize(*dynamic_column.getVariantInfo().variant_type->getDefaultSerialization(), variant_column, row_num, ostr); + } +} - if (escaping_rule == FormatSettings::EscapingRule::Quoted && (field.size() < 2 || field.front() != '\'' || field.back() != '\'')) - field = "'" + field + "'"; +void SerializationDynamic::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + auto nested_serialize = [&settings](const ISerialization & serialization, const IColumn & col, size_t row, WriteBuffer & buf) + { + serialization.serializeTextCSV(col, row, buf, settings); + }; - field_buf = std::make_unique(field); - auto string_discr = variant_info.variant_name_to_discriminator.at("String"); - deserializeVariant(dynamic_column.getVariantColumn(), std::make_shared(), string_discr, *field_buf, deserialize_variant); + serializeTextImpl(column, row_num, ostr, settings, nested_serialize); } void SerializationDynamic::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -466,8 +669,12 @@ bool SerializationDynamic::tryDeserializeTextCSV(DB::IColumn & column, DB::ReadB void SerializationDynamic::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & dynamic_column = assert_cast(column); - dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeTextEscaped(dynamic_column.getVariantColumn(), row_num, ostr, settings); + auto nested_serialize = [&settings](const ISerialization & serialization, const IColumn & col, size_t row, WriteBuffer & buf) + { + serialization.serializeTextEscaped(col, row, buf, settings); + }; + + serializeTextImpl(column, row_num, ostr, settings, nested_serialize); } void SerializationDynamic::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -500,8 +707,12 @@ bool SerializationDynamic::tryDeserializeTextEscaped(DB::IColumn & column, DB::R void SerializationDynamic::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & dynamic_column = assert_cast(column); - dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeTextQuoted(dynamic_column.getVariantColumn(), row_num, ostr, settings); + auto nested_serialize = [&settings](const ISerialization & serialization, const IColumn & col, size_t row, WriteBuffer & buf) + { + serialization.serializeTextQuoted(col, row, buf, settings); + }; + + serializeTextImpl(column, row_num, ostr, settings, nested_serialize); } void SerializationDynamic::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -534,8 +745,12 @@ bool SerializationDynamic::tryDeserializeTextQuoted(DB::IColumn & column, DB::Re void SerializationDynamic::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & dynamic_column = assert_cast(column); - dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeTextJSON(dynamic_column.getVariantColumn(), row_num, ostr, settings); + auto nested_serialize = [&settings](const ISerialization & serialization, const IColumn & col, size_t row, WriteBuffer & buf) + { + serialization.serializeTextJSON(col, row, buf, settings); + }; + + serializeTextImpl(column, row_num, ostr, settings, nested_serialize); } void SerializationDynamic::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -568,8 +783,12 @@ bool SerializationDynamic::tryDeserializeTextJSON(DB::IColumn & column, DB::Read void SerializationDynamic::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & dynamic_column = assert_cast(column); - dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeTextRaw(dynamic_column.getVariantColumn(), row_num, ostr, settings); + auto nested_serialize = [&settings](const ISerialization & serialization, const IColumn & col, size_t row, WriteBuffer & buf) + { + serialization.serializeTextRaw(col, row, buf, settings); + }; + + serializeTextImpl(column, row_num, ostr, settings, nested_serialize); } void SerializationDynamic::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -602,8 +821,12 @@ bool SerializationDynamic::tryDeserializeTextRaw(DB::IColumn & column, DB::ReadB void SerializationDynamic::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & dynamic_column = assert_cast(column); - dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeText(dynamic_column.getVariantColumn(), row_num, ostr, settings); + auto nested_serialize = [&settings](const ISerialization & serialization, const IColumn & col, size_t row, WriteBuffer & buf) + { + serialization.serializeText(col, row, buf, settings); + }; + + serializeTextImpl(column, row_num, ostr, settings, nested_serialize); } void SerializationDynamic::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -636,8 +859,12 @@ bool SerializationDynamic::tryDeserializeWholeText(DB::IColumn & column, DB::Rea void SerializationDynamic::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & dynamic_column = assert_cast(column); - dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeTextXML(dynamic_column.getVariantColumn(), row_num, ostr, settings); + auto nested_serialize = [&settings](const ISerialization & serialization, const IColumn & col, size_t row, WriteBuffer & buf) + { + serialization.serializeTextXML(col, row, buf, settings); + }; + + serializeTextImpl(column, row_num, ostr, settings, nested_serialize); } } diff --git a/src/DataTypes/Serializations/SerializationDynamic.h b/src/DataTypes/Serializations/SerializationDynamic.h index 001a3cf87ce..3dbf311fb6c 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.h +++ b/src/DataTypes/Serializations/SerializationDynamic.h @@ -105,9 +105,13 @@ private: { DynamicStructureSerializationVersion structure_version; DataTypePtr variant_type; - ColumnDynamic::Statistics statistics = {.source = ColumnDynamic::Statistics::Source::READ, .data = {}}; + size_t max_dynamic_types; + ColumnDynamic::StatisticsPtr statistics; - explicit DeserializeBinaryBulkStateDynamicStructure(UInt64 structure_version_) : structure_version(structure_version_) {} + explicit DeserializeBinaryBulkStateDynamicStructure(UInt64 structure_version_) + : structure_version(structure_version_) + { + } }; size_t max_dynamic_types; diff --git a/src/DataTypes/Serializations/SerializationDynamicElement.cpp b/src/DataTypes/Serializations/SerializationDynamicElement.cpp index 211f0ac9377..cffca14bca5 100644 --- a/src/DataTypes/Serializations/SerializationDynamicElement.cpp +++ b/src/DataTypes/Serializations/SerializationDynamicElement.cpp @@ -4,7 +4,10 @@ #include #include #include +#include #include +#include +#include #include namespace DB @@ -21,6 +24,8 @@ struct DeserializeBinaryBulkStateDynamicElement : public ISerialization::Deseria ISerialization::DeserializeBinaryBulkStatePtr structure_state; SerializationPtr variant_serialization; ISerialization::DeserializeBinaryBulkStatePtr variant_element_state; + bool read_from_shared_variant; + ColumnPtr shared_variant; }; void SerializationDynamicElement::enumerateStreams( @@ -73,9 +78,10 @@ void SerializationDynamicElement::deserializeBinaryBulkStatePrefix( auto dynamic_element_state = std::make_shared(); dynamic_element_state->structure_state = std::move(structure_state); - const auto & variant_type = checkAndGetState(dynamic_element_state->structure_state)->variant_type; + const auto & variant_type = assert_cast( + *checkAndGetState(dynamic_element_state->structure_state)->variant_type); /// Check if we actually have required element in the Variant. - if (auto global_discr = assert_cast(*variant_type).tryGetVariantDiscriminator(dynamic_element_name)) + if (auto global_discr = variant_type.tryGetVariantDiscriminator(dynamic_element_name)) { settings.path.push_back(Substream::DynamicData); if (is_null_map_subcolumn) @@ -83,6 +89,21 @@ void SerializationDynamicElement::deserializeBinaryBulkStatePrefix( else dynamic_element_state->variant_serialization = std::make_shared(nested_serialization, dynamic_element_name, *global_discr); dynamic_element_state->variant_serialization->deserializeBinaryBulkStatePrefix(settings, dynamic_element_state->variant_element_state, cache); + dynamic_element_state->read_from_shared_variant = false; + settings.path.pop_back(); + } + /// If we don't have this element in the Variant, we will read shared variant and try to find it there. + else + { + auto shared_variant_global_discr = variant_type.tryGetVariantDiscriminator(ColumnDynamic::getSharedVariantTypeName()); + chassert(shared_variant_global_discr.has_value()); + settings.path.push_back(Substream::DynamicData); + dynamic_element_state->variant_serialization = std::make_shared( + ColumnDynamic::getSharedVariantDataType()->getDefaultSerialization(), + ColumnDynamic::getSharedVariantTypeName(), + *shared_variant_global_discr); + dynamic_element_state->variant_serialization->deserializeBinaryBulkStatePrefix(settings, dynamic_element_state->variant_element_state, cache); + dynamic_element_state->read_from_shared_variant = true; settings.path.pop_back(); } @@ -115,23 +136,103 @@ void SerializationDynamicElement::deserializeBinaryBulkWithMultipleStreams( auto * dynamic_element_state = checkAndGetState(state); - if (dynamic_element_state->variant_serialization) + /// Check if this subcolumn should not be read from shared variant. + /// In this case just read data from the corresponding variant. + if (!dynamic_element_state->read_from_shared_variant) { settings.path.push_back(Substream::DynamicData); - dynamic_element_state->variant_serialization->deserializeBinaryBulkWithMultipleStreams(result_column, limit, settings, dynamic_element_state->variant_element_state, cache); + dynamic_element_state->variant_serialization->deserializeBinaryBulkWithMultipleStreams( + result_column, limit, settings, dynamic_element_state->variant_element_state, cache); settings.path.pop_back(); } - else if (is_null_map_subcolumn) - { - auto mutable_column = result_column->assumeMutable(); - auto & data = assert_cast(*mutable_column).getData(); - data.resize_fill(data.size() + limit, 1); - } + /// Otherwise, read the shared variant column and extract requested type from it. else { - auto mutable_column = result_column->assumeMutable(); - mutable_column->insertManyDefaults(limit); - result_column = std::move(mutable_column); + settings.path.push_back(Substream::DynamicData); + /// Initialize shared_variant column if needed. + if (result_column->empty()) + dynamic_element_state->shared_variant = makeNullable(ColumnDynamic::getSharedVariantDataType()->createColumn()); + size_t prev_size = result_column->size(); + dynamic_element_state->variant_serialization->deserializeBinaryBulkWithMultipleStreams( + dynamic_element_state->shared_variant, limit, settings, dynamic_element_state->variant_element_state, cache); + settings.path.pop_back(); + + /// If we need to read a subcolumn from variant column, create an empty variant column, fill it and extract subcolumn. + auto variant_type = DataTypeFactory::instance().get(dynamic_element_name); + auto result_type = makeNullableOrLowCardinalityNullableSafe(variant_type); + MutableColumnPtr variant_column = nested_subcolumn.empty() || is_null_map_subcolumn ? result_column->assumeMutable() : result_type->createColumn(); + variant_column->reserve(variant_column->size() + limit); + MutableColumnPtr non_nullable_variant_column = variant_column->assumeMutable(); + NullMap * null_map = nullptr; + bool is_low_cardinality_nullable = isColumnLowCardinalityNullable(*variant_column); + /// Resulting subolumn can be Nullable, but value is serialized in shared variant as non-Nullable. + /// Extract non-nullable column and remember the null map to fill it during deserialization. + if (isColumnNullable(*variant_column)) + { + auto & nullable_variant_column = assert_cast(*variant_column); + non_nullable_variant_column = nullable_variant_column.getNestedColumnPtr()->assumeMutable(); + null_map = &nullable_variant_column.getNullMapData(); + } + else if (is_null_map_subcolumn) + { + null_map = &assert_cast(*variant_column).getData(); + } + + auto variant_serialization = variant_type->getDefaultSerialization(); + + const auto & nullable_shared_variant = assert_cast(*dynamic_element_state->shared_variant); + const auto & shared_null_map = nullable_shared_variant.getNullMapData(); + const auto & shared_variant = assert_cast(nullable_shared_variant.getNestedColumn()); + const FormatSettings format_settings; + for (size_t i = prev_size; i != shared_variant.size(); ++i) + { + if (!shared_null_map[i]) + { + auto value = shared_variant.getDataAt(i); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + if (type->getName() == dynamic_element_name) + { + /// When requested type is LowCardinality the subcolumn type name will be LowCardinality(Nullable). + /// Value in shared variant is serialized as LowCardinality and we cannot simply deserialize it + /// inside LowCardinality(Nullable) column (it will try to deserialize null bit). In this case we + /// have to create temporary LowCardinality column, deserialize value into it and insert it into + /// resulting LowCardinality(Nullable) (insertion from LowCardinality column to LowCardinality(Nullable) + /// column is allowed). + if (is_low_cardinality_nullable) + { + auto tmp_column = variant_type->createColumn(); + variant_serialization->deserializeBinary(*tmp_column, buf, format_settings); + non_nullable_variant_column->insertFrom(*tmp_column, 0); + } + else if (is_null_map_subcolumn) + { + null_map->push_back(0); + } + else + { + variant_serialization->deserializeBinary(*non_nullable_variant_column, buf, format_settings); + if (null_map) + null_map->push_back(0); + } + } + else + { + variant_column->insertDefault(); + } + } + else + { + variant_column->insertDefault(); + } + } + + /// Extract nested subcolumn if needed. + if (!nested_subcolumn.empty() && !is_null_map_subcolumn) + { + auto subcolumn = result_type->getSubcolumn(nested_subcolumn, variant_column->getPtr()); + result_column->assumeMutable()->insertRangeFrom(*subcolumn, 0, subcolumn->size()); + } } } diff --git a/src/DataTypes/Serializations/SerializationDynamicElement.h b/src/DataTypes/Serializations/SerializationDynamicElement.h index 127d14a55e0..c674cf479ae 100644 --- a/src/DataTypes/Serializations/SerializationDynamicElement.h +++ b/src/DataTypes/Serializations/SerializationDynamicElement.h @@ -13,11 +13,15 @@ private: /// To be able to deserialize Dynamic element as a subcolumn /// we need its type name and global discriminator. String dynamic_element_name; + /// Nested subcolumn of a type dynamic type. For example, for `Tuple(a UInt32)`.a + /// subcolumn dynamic_element_name = 'Tuple(a UInt32)' and nested_subcolumn = 'a'. + /// Needed to extract nested subcolumn from values in shared variant. + String nested_subcolumn; bool is_null_map_subcolumn; public: - SerializationDynamicElement(const SerializationPtr & nested_, const String & dynamic_element_name_, bool is_null_map_subcolumn_ = false) - : SerializationWrapper(nested_), dynamic_element_name(dynamic_element_name_), is_null_map_subcolumn(is_null_map_subcolumn_) + SerializationDynamicElement(const SerializationPtr & nested_, const String & dynamic_element_name_, const String & nested_subcolumn_, bool is_null_map_subcolumn_ = false) + : SerializationWrapper(nested_), dynamic_element_name(dynamic_element_name_), nested_subcolumn(nested_subcolumn_), is_null_map_subcolumn(is_null_map_subcolumn_) { } diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 03b5d9584e0..36dc85f60ee 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -305,8 +305,10 @@ SerializationVariantElement::VariantSubcolumnCreator::VariantSubcolumnCreator( const String & variant_element_name_, ColumnVariant::Discriminator global_variant_discriminator_, ColumnVariant::Discriminator local_variant_discriminator_, - bool make_nullable_) + bool make_nullable_, + const ColumnPtr & null_map_) : local_discriminators(local_discriminators_) + , null_map(null_map_) , variant_element_name(variant_element_name_) , global_variant_discriminator(global_variant_discriminator_) , local_variant_discriminator(local_variant_discriminator_) @@ -314,12 +316,13 @@ SerializationVariantElement::VariantSubcolumnCreator::VariantSubcolumnCreator( { } -DataTypePtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB::DataTypePtr & prev) const + +DataTypePtr SerializationVariantElement::VariantSubcolumnCreator::create(const DataTypePtr & prev) const { return make_nullable ? makeNullableOrLowCardinalityNullableSafe(prev) : prev; } -SerializationPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB::SerializationPtr & prev) const +SerializationPtr SerializationVariantElement::VariantSubcolumnCreator::create(const SerializationPtr & prev) const { return std::make_shared(prev, variant_element_name, global_variant_discriminator); } @@ -339,12 +342,16 @@ ColumnPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB: return res; } - /// In general case we should iterate through discriminators and create null-map for our variant. - NullMap null_map; - null_map.reserve(local_discriminators->size()); - const auto & local_discriminators_data = assert_cast(*local_discriminators).getData(); - for (auto local_discr : local_discriminators_data) - null_map.push_back(local_discr != local_variant_discriminator); + /// In general case we should iterate through discriminators and create null-map for our variant if we don't already have it. + std::optional null_map_from_discriminators; + if (!null_map) + { + null_map_from_discriminators = NullMap(); + null_map_from_discriminators->reserve(local_discriminators->size()); + const auto & local_discriminators_data = assert_cast(*local_discriminators).getData(); + for (auto local_discr : local_discriminators_data) + null_map_from_discriminators->push_back(local_discr != local_variant_discriminator); + } /// Now we can create new column from null-map and variant column using IColumn::expand. auto res_column = IColumn::mutate(prev); @@ -356,13 +363,21 @@ ColumnPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB: if (make_nullable && prev->lowCardinality()) res_column = assert_cast(*res_column).cloneNullable(); - res_column->expand(null_map, /*inverted = */ true); + if (null_map_from_discriminators) + res_column->expand(*null_map_from_discriminators, /*inverted = */ true); + else + res_column->expand(assert_cast(*null_map).getData(), /*inverted = */ true); if (make_nullable && prev->canBeInsideNullable()) { - auto null_map_col = ColumnUInt8::create(); - null_map_col->getData() = std::move(null_map); - return ColumnNullable::create(std::move(res_column), std::move(null_map_col)); + if (null_map_from_discriminators) + { + auto null_map_col = ColumnUInt8::create(); + null_map_col->getData() = std::move(*null_map_from_discriminators); + return ColumnNullable::create(std::move(res_column), std::move(null_map_col)); + } + + return ColumnNullable::create(std::move(res_column), null_map->assumeMutable()); } return res_column; diff --git a/src/DataTypes/Serializations/SerializationVariantElement.h b/src/DataTypes/Serializations/SerializationVariantElement.h index 69101aea0f5..64f86eb2190 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.h +++ b/src/DataTypes/Serializations/SerializationVariantElement.h @@ -63,18 +63,22 @@ public: struct VariantSubcolumnCreator : public ISubcolumnCreator { + private: const ColumnPtr local_discriminators; + const ColumnPtr null_map; /// optional const String variant_element_name; const ColumnVariant::Discriminator global_variant_discriminator; const ColumnVariant::Discriminator local_variant_discriminator; bool make_nullable; + public: VariantSubcolumnCreator( const ColumnPtr & local_discriminators_, const String & variant_element_name_, ColumnVariant::Discriminator global_variant_discriminator_, ColumnVariant::Discriminator local_variant_discriminator_, - bool make_nullable_); + bool make_nullable_, + const ColumnPtr & null_map_ = nullptr); DataTypePtr create(const DataTypePtr & prev) const override; ColumnPtr create(const ColumnPtr & prev) const override; diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 242d2dc9f80..86fde3852b8 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1362,13 +1362,14 @@ public: } auto & variant_column = column_dynamic.getVariantColumn(); - auto variant_info = column_dynamic.getVariantInfo(); + const auto & variant_info = column_dynamic.getVariantInfo(); /// Second, infer ClickHouse type for this element and add it as a new variant. auto element_type = elementToDataType(element, format_settings); - if (column_dynamic.addNewVariant(element_type)) + auto element_type_name = element_type->getName(); + if (column_dynamic.addNewVariant(element_type, element_type_name)) { auto node = buildJSONExtractTree(element_type, "Dynamic inference"); - auto global_discriminator = variant_info.variant_name_to_discriminator[element_type->getName()]; + auto global_discriminator = variant_info.variant_name_to_discriminator.at(element_type_name); auto & variant = variant_column.getVariantByGlobalDiscriminator(global_discriminator); if (!node->insertResultToColumn(variant, element, insert_settings, format_settings, error)) return false; @@ -1377,29 +1378,15 @@ public: return true; } - /// We couldn't add new variant. Try to insert element into current variants. - auto variant_node = buildJSONExtractTree(variant_info.variant_type, "Dynamic inference"); - if (variant_node->insertResultToColumn(variant_column, element, insert_settings, format_settings, error)) - return true; - - /// We couldn't insert element into any existing variant, add String variant and read value as String. - column_dynamic.addStringVariant(); - auto string_global_discriminator = variant_info.variant_name_to_discriminator["String"]; - auto & string_column = variant_column.getVariantByGlobalDiscriminator(string_global_discriminator); - if (!getStringNode()->insertResultToColumn(string_column, element, insert_settings, format_settings, error)) + /// We couldn't add this variant, insert it into shared variant. + auto tmp_variant_column = element_type->createColumn(); + auto node = buildJSONExtractTree(element_type, "Dynamic inference"); + if (!node->insertResultToColumn(*tmp_variant_column, element, insert_settings, format_settings, error)) return false; - variant_column.getLocalDiscriminators().push_back(variant_column.localDiscriminatorByGlobal(string_global_discriminator)); - variant_column.getOffsets().push_back(string_column.size() - 1); + column_dynamic.insertValueIntoSharedVariant(*tmp_variant_column, element_type, element_type_name, 0); return true; } - static const std::unique_ptr> & getStringNode() - { - static const std::unique_ptr> string_node - = buildJSONExtractTree(std::make_shared(), "Dynamic inference"); - return string_node; - } - static DataTypePtr elementToDataType(const typename JSONParser::Element & element, const FormatSettings & format_settings) { JSONInferenceInfo json_inference_info; diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 675283d011e..21b98cf505c 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -43,6 +43,7 @@ #include #include #include +#include #include #include #include @@ -4287,13 +4288,98 @@ private: WrapperType createDynamicToColumnWrapper(const DataTypePtr &) const { return [this] - (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * col_nullable, size_t input_rows_count) -> ColumnPtr + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr { + /// When casting Dynamic to regular column we should cast all variants from current Dynamic column + /// and construct the result based on discriminators. const auto & column_dynamic = assert_cast(*arguments.front().column.get()); + const auto & variant_column = column_dynamic.getVariantColumn(); const auto & variant_info = column_dynamic.getVariantInfo(); - auto variant_wrapper = createVariantToColumnWrapper(assert_cast(*variant_info.variant_type), result_type); - ColumnsWithTypeAndName args = {ColumnWithTypeAndName(column_dynamic.getVariantColumnPtr(), variant_info.variant_type, "")}; - return variant_wrapper(args, result_type, col_nullable, input_rows_count); + + /// First, cast usual variants to result type. + const auto & variant_types = assert_cast(*variant_info.variant_type).getVariants(); + std::vector casted_variant_columns; + casted_variant_columns.reserve(variant_types.size()); + for (size_t i = 0; i != variant_types.size(); ++i) + { + const auto & variant_col = variant_column.getVariantPtrByGlobalDiscriminator(i); + ColumnsWithTypeAndName variant = {{variant_col, variant_types[i], ""}}; + auto variant_wrapper = prepareUnpackDictionaries(variant_types[i], result_type); + casted_variant_columns.push_back(variant_wrapper(variant, result_type, nullptr, variant_col->size())); + } + + /// Second, collect all variants stored in shared variant and cast them to result type. + std::vector variant_columns_from_shared_variant; + DataTypes variant_types_from_shared_variant; + /// We will need to know what variant to use when we see discriminator of a shared variant. + /// To do it, we remember what variant was extracted from each row and what was it's offset. + PaddedPODArray shared_variant_indexes; + PaddedPODArray shared_variant_offsets; + std::unordered_map shared_variant_to_index; + const auto & shared_variant = column_dynamic.getSharedVariant(); + const auto shared_variant_discr = column_dynamic.getSharedVariantDiscriminator(); + const auto & local_discriminators = variant_column.getLocalDiscriminators(); + const auto & offsets = variant_column.getOffsets(); + if (!shared_variant.empty()) + { + shared_variant_indexes.reserve(input_rows_count); + shared_variant_offsets.reserve(input_rows_count); + FormatSettings format_settings; + const auto shared_variant_local_discr = variant_column.localDiscriminatorByGlobal(shared_variant_discr); + for (size_t i = 0; i != input_rows_count; ++i) + { + if (local_discriminators[i] == shared_variant_local_discr) + { + auto value = shared_variant.getDataAt(offsets[i]); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + auto type_name = type->getName(); + auto it = shared_variant_to_index.find(type_name); + /// Check if didn't created column for this variant yet. + if (it == shared_variant_to_index.end()) + { + it = shared_variant_to_index.emplace(type_name, variant_columns_from_shared_variant.size()).first; + variant_columns_from_shared_variant.push_back(type->createColumn()); + variant_types_from_shared_variant.push_back(type); + } + + shared_variant_indexes.push_back(it->second); + shared_variant_offsets.push_back(variant_columns_from_shared_variant[it->second]->size()); + type->getDefaultSerialization()->deserializeBinary(*variant_columns_from_shared_variant[it->second], buf, format_settings); + } + else + { + shared_variant_indexes.emplace_back(); + shared_variant_offsets.emplace_back(); + } + } + } + + /// Cast all extracted variants into result type. + std::vector casted_shared_variant_columns; + casted_shared_variant_columns.reserve(variant_types_from_shared_variant.size()); + for (size_t i = 0; i != variant_types_from_shared_variant.size(); ++i) + { + ColumnsWithTypeAndName variant = {{variant_columns_from_shared_variant[i]->getPtr(), variant_types_from_shared_variant[i], ""}}; + auto variant_wrapper = prepareUnpackDictionaries(variant_types_from_shared_variant[i], result_type); + casted_shared_variant_columns.push_back(variant_wrapper(variant, result_type, nullptr, variant_columns_from_shared_variant[i]->size())); + } + + /// Construct result column from all casted variants. + auto res = result_type->createColumn(); + res->reserve(input_rows_count); + for (size_t i = 0; i != input_rows_count; ++i) + { + auto global_discr = variant_column.globalDiscriminatorByLocal(local_discriminators[i]); + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + res->insertDefault(); + else if (global_discr == shared_variant_discr) + res->insertFrom(*casted_shared_variant_columns[shared_variant_indexes[i]], shared_variant_offsets[i]); + else + res->insertFrom(*casted_variant_columns[global_discr], offsets[i]); + } + + return res; }; } @@ -4320,200 +4406,51 @@ private: }; } - std::pair getReducedVariant( - const ColumnVariant & variant_column, - const DataTypePtr & variant_type, - const std::unordered_map & variant_name_to_discriminator, - size_t max_result_num_variants, - const ColumnDynamic::Statistics & statistics = {}) const + WrapperType createVariantToDynamicWrapper(const DataTypeVariant & from_variant_type, const DataTypeDynamic & dynamic_type) const { - const auto & variant_types = assert_cast(*variant_type).getVariants(); - /// First check if we don't exceed the limit in current Variant column. - if (variant_types.size() < max_result_num_variants || (variant_types.size() == max_result_num_variants && variant_name_to_discriminator.contains("String"))) - return {variant_column.getPtr(), variant_type}; - - /// We want to keep the most frequent variants and convert to string the rarest. - std::vector> variant_sizes; - variant_sizes.reserve(variant_types.size()); - std::optional old_string_discriminator; - /// List of variants that should be converted to a single String variant. - std::vector variants_to_convert_to_string; - for (size_t i = 0; i != variant_types.size(); ++i) + /// First create extended Variant with shared variant type and cast this Variant to it. + auto variants_for_dynamic = from_variant_type.getVariants(); + size_t number_of_variants = variants_for_dynamic.size(); + variants_for_dynamic.push_back(ColumnDynamic::getSharedVariantDataType()); + const auto & variant_type_for_dynamic = std::make_shared(variants_for_dynamic); + auto old_to_new_variant_wrapper = createVariantToVariantWrapper(from_variant_type, *variant_type_for_dynamic); + auto max_dynamic_types = dynamic_type.getMaxDynamicTypes(); + return [old_to_new_variant_wrapper, variant_type_for_dynamic, number_of_variants, max_dynamic_types] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * col_nullable, size_t input_rows_count) -> ColumnPtr { - /// String variant won't be removed. - String variant_name = variant_types[i]->getName(); + auto variant_column_for_dynamic = old_to_new_variant_wrapper(arguments, result_type, col_nullable, input_rows_count); + /// If resulting Dynamic column can contain all variants from this Variant column, just create Dynamic column from it. + if (max_dynamic_types >= number_of_variants) + return ColumnDynamic::create(variant_column_for_dynamic, variant_type_for_dynamic, max_dynamic_types, max_dynamic_types); - if (variant_name == "String") - { - old_string_discriminator = i; - /// For simplicity, add this variant to the list that will be converted to string, - /// so we will process it with other variants when constructing the new String variant. - variants_to_convert_to_string.push_back(i); - } - else - { - size_t size = 0; - if (statistics.data.empty()) - size = variant_column.getVariantByGlobalDiscriminator(i).size(); - else - size = statistics.data.at(variant_name); - variant_sizes.emplace_back(size, i); - } - } - - /// Sort variants by sizes, so we will keep the most frequent. - std::sort(variant_sizes.begin(), variant_sizes.end(), std::greater()); - - DataTypes remaining_variants; - remaining_variants.reserve(max_result_num_variants); - /// Add String variant in advance. - remaining_variants.push_back(std::make_shared()); - for (auto [_, discr] : variant_sizes) - { - if (remaining_variants.size() != max_result_num_variants) - remaining_variants.push_back(variant_types[discr]); - else - variants_to_convert_to_string.push_back(discr); - } - - auto reduced_variant = std::make_shared(remaining_variants); - const auto & new_variants = reduced_variant->getVariants(); - /// To construct reduced variant column we will need mapping from old to new discriminators. - std::vector old_to_new_discriminators_mapping; - old_to_new_discriminators_mapping.resize(variant_types.size()); - ColumnVariant::Discriminator string_variant_discriminator = 0; - for (size_t i = 0; i != new_variants.size(); ++i) - { - String variant_name = new_variants[i]->getName(); - if (variant_name == "String") - { - string_variant_discriminator = i; - for (auto discr : variants_to_convert_to_string) - old_to_new_discriminators_mapping[discr] = i; - } - else - { - auto old_discr = variant_name_to_discriminator.at(variant_name); - old_to_new_discriminators_mapping[old_discr] = i; - } - } - - /// Convert all reduced variants to String. - std::unordered_map variants_converted_to_string; - variants_converted_to_string.reserve(variants_to_convert_to_string.size()); - size_t string_variant_size = 0; - for (auto discr : variants_to_convert_to_string) - { - auto string_type = std::make_shared(); - auto string_wrapper = prepareUnpackDictionaries(variant_types[discr], string_type); - auto column_to_convert = ColumnWithTypeAndName(variant_column.getVariantPtrByGlobalDiscriminator(discr), variant_types[discr], ""); - ColumnsWithTypeAndName args = {column_to_convert}; - auto variant_string_column = string_wrapper(args, string_type, nullptr, column_to_convert.column->size()); - string_variant_size += variant_string_column->size(); - variants_converted_to_string[discr] = variant_string_column; - } - - /// Create new discriminators and offsets and fill new String variant according to old discriminators. - auto string_variant = ColumnString::create(); - string_variant->reserve(string_variant_size); - auto new_discriminators_column = variant_column.getLocalDiscriminatorsPtr()->cloneEmpty(); - auto & new_discriminators_data = assert_cast(*new_discriminators_column).getData(); - new_discriminators_data.reserve(variant_column.size()); - auto new_offsets = variant_column.getOffsetsPtr()->cloneEmpty(); - auto & new_offsets_data = assert_cast(*new_offsets).getData(); - new_offsets_data.reserve(variant_column.size()); - const auto & old_local_discriminators = variant_column.getLocalDiscriminators(); - const auto & old_offsets = variant_column.getOffsets(); - for (size_t i = 0; i != old_local_discriminators.size(); ++i) - { - auto old_discr = variant_column.globalDiscriminatorByLocal(old_local_discriminators[i]); - - if (old_discr == ColumnVariant::NULL_DISCRIMINATOR) - { - new_discriminators_data.push_back(ColumnVariant::NULL_DISCRIMINATOR); - new_offsets_data.push_back(0); - continue; - } - - auto new_discr = old_to_new_discriminators_mapping[old_discr]; - new_discriminators_data.push_back(new_discr); - if (new_discr != string_variant_discriminator) - { - new_offsets_data.push_back(old_offsets[i]); - } - else - { - new_offsets_data.push_back(string_variant->size()); - string_variant->insertFrom(*variants_converted_to_string[old_discr], old_offsets[i]); - } - } - - /// Create new list of variant columns. - Columns new_variant_columns; - new_variant_columns.resize(new_variants.size()); - for (size_t i = 0; i != variant_types.size(); ++i) - { - auto new_discr = old_to_new_discriminators_mapping[i]; - if (new_discr != string_variant_discriminator) - new_variant_columns[new_discr] = variant_column.getVariantPtrByGlobalDiscriminator(i); - } - new_variant_columns[string_variant_discriminator] = std::move(string_variant); - return {ColumnVariant::create(std::move(new_discriminators_column), std::move(new_offsets), new_variant_columns), reduced_variant}; - } - - WrapperType createVariantToDynamicWrapper(const DataTypePtr & from_type, const DataTypeDynamic & dynamic_type) const - { - const auto & from_variant_type = assert_cast(*from_type); - size_t max_dynamic_types = dynamic_type.getMaxDynamicTypes(); - const auto & variants = from_variant_type.getVariants(); - std::unordered_map variant_name_to_discriminator; - variant_name_to_discriminator.reserve(variants.size()); - for (size_t i = 0; i != variants.size(); ++i) - variant_name_to_discriminator[variants[i]->getName()] = i; - - return [from_type, max_dynamic_types, variant_name_to_discriminator, this] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t) -> ColumnPtr - { - const auto & variant_column = assert_cast(*arguments.front().column); - auto [reduced_variant_column, reduced_variant_type] = getReducedVariant(variant_column, from_type, variant_name_to_discriminator, max_dynamic_types); - return ColumnDynamic::create(reduced_variant_column, reduced_variant_type, max_dynamic_types); + /// Otherwise some variants should go to the shared variant. Create temporary Dynamic column from this Variant and insert + /// all data to the resulting Dynamic column, this insertion will do all the logic with shared variant. + auto tmp_dynamic_column = ColumnDynamic::create(variant_column_for_dynamic, variant_type_for_dynamic, number_of_variants, number_of_variants); + auto result_dynamic_column = ColumnDynamic::create(max_dynamic_types); + result_dynamic_column->insertRangeFrom(*tmp_dynamic_column, 0, tmp_dynamic_column->size()); + return result_dynamic_column; }; } WrapperType createColumnToDynamicWrapper(const DataTypePtr & from_type, const DataTypeDynamic & dynamic_type) const { if (const auto * variant_type = typeid_cast(from_type.get())) - return createVariantToDynamicWrapper(from_type, dynamic_type); - - if (dynamic_type.getMaxDynamicTypes() == 1) - { - DataTypePtr string_type = std::make_shared(); - if (from_type->isNullable()) - string_type = makeNullable(string_type); - auto string_wrapper = prepareUnpackDictionaries(from_type, string_type); - auto variant_type = std::make_shared(DataTypes{removeNullable(string_type)}); - auto variant_wrapper = createColumnToVariantWrapper(string_type, *variant_type); - return [string_wrapper, variant_wrapper, string_type, variant_type, max_dynamic_types=dynamic_type.getMaxDynamicTypes()] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * col_nullable, size_t input_rows_count) -> ColumnPtr - { - auto string_column = string_wrapper(arguments, string_type, col_nullable, input_rows_count); - auto column = ColumnWithTypeAndName(string_column, string_type, ""); - ColumnsWithTypeAndName args = {column}; - auto variant_column = variant_wrapper(args, variant_type, nullptr, string_column->size()); - return ColumnDynamic::create(variant_column, variant_type, max_dynamic_types); - }; - } + return createVariantToDynamicWrapper(*variant_type, dynamic_type); if (context && context->getSettingsRef().cast_string_to_dynamic_use_inference && isStringOrFixedString(removeNullable(removeLowCardinality(from_type)))) return createStringToDynamicThroughParsingWrapper(); + /// First, cast column to Variant with 2 variants - the type of the column we cast and shared variant type. auto variant_type = std::make_shared(DataTypes{removeNullableOrLowCardinalityNullable(from_type)}); - auto variant_wrapper = createColumnToVariantWrapper(from_type, *variant_type); - return [variant_wrapper, variant_type, max_dynamic_types=dynamic_type.getMaxDynamicTypes()] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * col_nullable, size_t input_rows_count) -> ColumnPtr + auto column_to_variant_wrapper = createColumnToVariantWrapper(from_type, *variant_type); + /// Second, cast this Variant to Dynamic. + auto variant_to_dynamic_wrapper = createVariantToDynamicWrapper(*variant_type, dynamic_type); + return [column_to_variant_wrapper, variant_to_dynamic_wrapper, variant_type] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * col_nullable, size_t input_rows_count) -> ColumnPtr { - auto variant_res = variant_wrapper(arguments, variant_type, col_nullable, input_rows_count); - return ColumnDynamic::create(variant_res, variant_type, max_dynamic_types); + auto variant_res = column_to_variant_wrapper(arguments, variant_type, col_nullable, input_rows_count); + ColumnsWithTypeAndName args = {{variant_res, variant_type, ""}}; + return variant_to_dynamic_wrapper(args, result_type, nullptr, input_rows_count); }; } @@ -4530,21 +4467,26 @@ private: (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t) -> ColumnPtr { const auto & column_dynamic = assert_cast(*arguments[0].column); - return ColumnDynamic::create(column_dynamic.getVariantColumnPtr(), column_dynamic.getVariantInfo(), to_max_types); + /// We should use the same limit as already used in column and change only global limit. + /// It's needed because shared variant should contain values only when limit is exceeded, + /// so if there are already some data, we cannot increase the limit. + return ColumnDynamic::create(column_dynamic.getVariantColumnPtr(), column_dynamic.getVariantInfo(), column_dynamic.getMaxDynamicTypes(), to_max_types); }; } - return [to_max_types, this] + return [to_max_types] (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t) -> ColumnPtr { const auto & column_dynamic = assert_cast(*arguments[0].column); - auto [reduced_variant_column, reduced_variant_type] = getReducedVariant( - column_dynamic.getVariantColumn(), - column_dynamic.getVariantInfo().variant_type, - column_dynamic.getVariantInfo().variant_name_to_discriminator, - to_max_types, - column_dynamic.getStatistics()); - return ColumnDynamic::create(reduced_variant_column, reduced_variant_type, to_max_types); + /// If real limit in the column is not greater than desired, just use the same variant column. + if (column_dynamic.getMaxDynamicTypes() <= to_max_types) + return ColumnDynamic::create(column_dynamic.getVariantColumnPtr(), column_dynamic.getVariantInfo(), column_dynamic.getMaxDynamicTypes(), to_max_types); + + /// Otherwise some variants should go to the shared variant. In this case we can just insert all + /// the data into resulting column and it will do all the logic with shared variant. + auto result_dynamic_column = ColumnDynamic::create(to_max_types); + result_dynamic_column->insertRangeFrom(column_dynamic, 0, column_dynamic.size()); + return result_dynamic_column; }; } diff --git a/src/Functions/dynamicType.cpp b/src/Functions/dynamicType.cpp index e8ca73597d6..327cdfe1616 100644 --- a/src/Functions/dynamicType.cpp +++ b/src/Functions/dynamicType.cpp @@ -2,10 +2,14 @@ #include #include #include +#include +#include #include #include #include #include +#include +#include #include @@ -65,11 +69,15 @@ public: const auto & variant_column = dynamic_column->getVariantColumn(); auto res = result_type->createColumn(); String element_type; + auto shared_variant_discr = dynamic_column->getSharedVariantDiscriminator(); + const auto & shared_variant = dynamic_column->getSharedVariant(); for (size_t i = 0; i != input_rows_count; ++i) { auto global_discr = variant_column.globalDiscriminatorAt(i); if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) element_type = name_for_null; + else if (global_discr == shared_variant_discr) + element_type = getTypeNameFromSharedVariantValue(shared_variant.getDataAt(variant_column.offsetAt(i))); else element_type = variant_info.variant_names[global_discr]; @@ -78,6 +86,63 @@ public: return res; } + + String getTypeNameFromSharedVariantValue(StringRef value) const + { + ReadBufferFromMemory buf(value.data, value.size); + return decodeDataType(buf)->getName(); + } +}; + +class FunctionIsDynamicElementInSharedData : public IFunction +{ +public: + static constexpr auto name = "isDynamicElementInSharedData"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.empty() || arguments.size() > 1) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 1", + getName(), arguments.empty()); + + if (!isDynamic(arguments[0].type.get())) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for function {} must be Dynamic, got {} instead", + getName(), arguments[0].type->getName()); + + return DataTypeFactory::instance().get("Bool"); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + const ColumnDynamic * dynamic_column = checkAndGetColumn(arguments[0].column.get()); + if (!dynamic_column) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for function {} must be Dynamic, got {} instead", + getName(), arguments[0].type->getName()); + + const auto & variant_column = dynamic_column->getVariantColumn(); + const auto & local_discriminators = variant_column.getLocalDiscriminators(); + auto res = result_type->createColumn(); + auto & res_data = assert_cast(*res).getData(); + res_data.reserve(dynamic_column->size()); + auto shared_variant_local_discr = variant_column.localDiscriminatorByGlobal(dynamic_column->getSharedVariantDiscriminator()); + for (size_t i = 0; i != input_rows_count; ++i) + res_data.push_back(local_discriminators[i] == shared_variant_local_discr); + + return res; + } }; } @@ -88,7 +153,7 @@ REGISTER_FUNCTION(DynamicType) .description = R"( Returns the variant type name for each row of `Dynamic` column. If row contains NULL, it returns 'None' for it. )", - .syntax = {"dynamicType(variant)"}, + .syntax = {"dynamicType(dynamic)"}, .arguments = {{"dynamic", "Dynamic column"}}, .examples = {{{ "Example", @@ -104,6 +169,30 @@ SELECT d, dynamicType(d) FROM test; │ Hello, World! │ String │ │ [1,2,3] │ Array(Int64) │ └───────────────┴────────────────┘ +)"}}}, + .categories{"Variant"}, + }); + + factory.registerFunction(FunctionDocumentation{ + .description = R"( +Returns true for rows in Dynamic column that are not separated into subcolumns and stored inside shared variant in binary form. +)", + .syntax = {"isDynamicElementInSharedData(dynamic)"}, + .arguments = {{"dynamic", "Dynamic column"}}, + .examples = {{{ + "Example", + R"( +CREATE TABLE test (d Dynamic(max_types=2)) ENGINE = Memory; +INSERT INTO test VALUES (NULL), (42), ('Hello, World!'), ([1, 2, 3]); +SELECT d, isDynamicElementInSharedData(d) FROM test; +)", + R"( +┌─d─────────────┬─isDynamicElementInSharedData(d)─┐ +│ ᴺᵁᴸᴸ │ false │ +│ 42 │ false │ +│ Hello, World! │ true │ +│ [1,2,3] │ true │ +└───────────────┴────────────────────┘ )"}}}, .categories{"Variant"}, }); diff --git a/tests/queries/0_stateless/00000_test.sql b/tests/queries/0_stateless/00000_test.sql new file mode 100644 index 00000000000..db9dd774484 --- /dev/null +++ b/tests/queries/0_stateless/00000_test.sql @@ -0,0 +1,43 @@ +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; +set enable_named_columns_in_function_tuple = 0; +drop table if exists test; +create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1; + + +system stop merges test; +insert into test select number, number from numbers(10); +insert into test select number, tuple(if(number % 3 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(10); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(5); +insert into test select number, multiIf(number % 5 == 0, tuple(if(number % 3 == 0, toDateTime(number), toIPv4(number)))::Tuple(a Dynamic(max_types=3)), number % 5 == 1 or number % 5 == 2, number, 'str_' || number) from numbers(10); + +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; + +system stop merges test; +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(5); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(20); + +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; + +system stop merges test; +insert into test select number, tuple(toDateTime(number))::Tuple(a Dynamic(max_types=3)) from numbers(4); + +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; + +drop table test; + diff --git a/tests/queries/0_stateless/03033_dynamic_text_serialization.reference b/tests/queries/0_stateless/03033_dynamic_text_serialization.reference index d965245266c..9fc356cc5e6 100644 --- a/tests/queries/0_stateless/03033_dynamic_text_serialization.reference +++ b/tests/queries/0_stateless/03033_dynamic_text_serialization.reference @@ -11,11 +11,11 @@ JSON {"d":["1","str",["1","2","3"]],"dynamicType(d)":"Tuple(Int64, String, Array(Int64))"} {"d":null,"dynamicType(d)":"None"} {"d":true,"dynamicType(d)":"Bool"} -{"d":"42","dynamicType(d)":"Int64"} -{"d":"42.42","dynamicType(d)":"String"} -{"d":"str","dynamicType(d)":"String"} -{"d":null,"dynamicType(d)":"None"} -{"d":"1","dynamicType(d)":"Int64"} +{"d":"42","dynamicType(d)":"Int64","isDynamicElementInSharedData(d)":false} +{"d":42.42,"dynamicType(d)":"Float64","isDynamicElementInSharedData(d)":false} +{"d":"str","dynamicType(d)":"String","isDynamicElementInSharedData(d)":true} +{"d":null,"dynamicType(d)":"None","isDynamicElementInSharedData(d)":false} +{"d":true,"dynamicType(d)":"Bool","isDynamicElementInSharedData(d)":true} CSV 42,"Int64" 42.42,"Float64" @@ -44,12 +44,12 @@ Cast using parsing [1,2,3] Array(Int64) 2020-01-01 Date 2020-01-01 10:00:00.000000000 DateTime64(9) -\N None +NULL String true Bool -42 Int64 -42.42 Float64 -[1, 2, 3] String -2020-01-01 String -2020-01-01 10:00:00 String -\N None -true String +42 Int64 false +42.42 Float64 false +[1,2,3] Array(Int64) false +2020-01-01 Date true +2020-01-01 10:00:00.000000000 DateTime64(9) true +NULL String true +true Bool true diff --git a/tests/queries/0_stateless/03033_dynamic_text_serialization.sql b/tests/queries/0_stateless/03033_dynamic_text_serialization.sql index d12d110fe28..45539cb13eb 100644 --- a/tests/queries/0_stateless/03033_dynamic_text_serialization.sql +++ b/tests/queries/0_stateless/03033_dynamic_text_serialization.sql @@ -16,7 +16,7 @@ select d, dynamicType(d) from format(JSONEachRow, 'd Dynamic', $$ {"d" : true} $$) format JSONEachRow; -select d, dynamicType(d) from format(JSONEachRow, 'd Dynamic(max_types=2)', $$ +select d, dynamicType(d), isDynamicElementInSharedData(d) from format(JSONEachRow, 'd Dynamic(max_types=2)', $$ {"d" : 42} {"d" : 42.42} {"d" : "str"} @@ -69,6 +69,6 @@ create table test (s String) engine=Memory; insert into test values ('42'), ('42.42'), ('[1, 2, 3]'), ('2020-01-01'), ('2020-01-01 10:00:00'), ('NULL'), ('true'); set cast_string_to_dynamic_use_inference=1; select s::Dynamic as d, dynamicType(d) from test; -select s::Dynamic(max_types=3) as d, dynamicType(d) from test; +select s::Dynamic(max_types=3) as d, dynamicType(d), isDynamicElementInSharedData(d) from test; drop table test; diff --git a/tests/queries/0_stateless/03034_dynamic_conversions.reference b/tests/queries/0_stateless/03034_dynamic_conversions.reference index 45f94f7ecc4..e22b64701a3 100644 --- a/tests/queries/0_stateless/03034_dynamic_conversions.reference +++ b/tests/queries/0_stateless/03034_dynamic_conversions.reference @@ -1,9 +1,9 @@ 0 UInt64 1 UInt64 2 UInt64 -0 String -1 String -2 String +0 UInt64 +1 UInt64 +2 UInt64 0 1 2 @@ -25,15 +25,15 @@ str_1 String \N None 4 UInt64 str_5 String -0 String +0 UInt64 str_1 String -[0,1] String +[0,1] Array(UInt64) \N None -4 String +4 UInt64 str_5 String 0 UInt64 str_1 String -[0,1] String +[0,1] Array(UInt64) \N None 4 UInt64 str_5 String @@ -51,13 +51,13 @@ str_5 String 2 0 UInt64 str_1 String -[0,1] String +[0,1] Array(UInt64) \N None 4 UInt64 str_5 String 0 UInt64 1970-01-02 Date -[0,1] String +[0,1] Array(UInt64) \N None 4 UInt64 1970-01-06 Date diff --git a/tests/queries/0_stateless/03034_dynamic_conversions.sql b/tests/queries/0_stateless/03034_dynamic_conversions.sql index ed75fbf2377..c0b470f29c5 100644 --- a/tests/queries/0_stateless/03034_dynamic_conversions.sql +++ b/tests/queries/0_stateless/03034_dynamic_conversions.sql @@ -3,7 +3,7 @@ set allow_experimental_variant_type=1; set use_variant_as_common_type=1; select number::Dynamic as d, dynamicType(d) from numbers(3); -select number::Dynamic(max_types=1) as d, dynamicType(d) from numbers(3); +select number::Dynamic(max_types=0) as d, dynamicType(d) from numbers(3); select number::Dynamic::UInt64 as v from numbers(3); select number::Dynamic::String as v from numbers(3); select number::Dynamic::Date as v from numbers(3); @@ -12,13 +12,13 @@ select number::Dynamic::Variant(UInt64, String) as v, variantType(v) from number select (number % 2 ? NULL : number)::Dynamic as d, dynamicType(d) from numbers(3); select multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, range(number), NULL)::Dynamic as d, dynamicType(d) from numbers(6); +select multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, range(number), NULL)::Dynamic(max_types=0) as d, dynamicType(d) from numbers(6); select multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, range(number), NULL)::Dynamic(max_types=1) as d, dynamicType(d) from numbers(6); select multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, range(number), NULL)::Dynamic(max_types=2) as d, dynamicType(d) from numbers(6); -select multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, range(number), NULL)::Dynamic(max_types=3) as d, dynamicType(d) from numbers(6); select number::Dynamic(max_types=2)::Dynamic(max_types=3) as d from numbers(3); select number::Dynamic(max_types=2)::Dynamic(max_types=1) as d from numbers(3); -select multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, range(number), NULL)::Dynamic(max_types=3)::Dynamic(max_types=2) as d, dynamicType(d) from numbers(6); +select multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, range(number), NULL)::Dynamic(max_types=2)::Dynamic(max_types=1) as d, dynamicType(d) from numbers(6); select multiIf(number % 4 == 0, number, number % 4 == 1, toDate(number), number % 4 == 2, range(number), NULL)::Dynamic(max_types=4)::Dynamic(max_types=3) as d, dynamicType(d) from numbers(6); diff --git a/tests/queries/0_stateless/03035_dynamic_sorting.reference b/tests/queries/0_stateless/03035_dynamic_sorting.reference index 9b8df11c7a9..f253c34ce8a 100644 --- a/tests/queries/0_stateless/03035_dynamic_sorting.reference +++ b/tests/queries/0_stateless/03035_dynamic_sorting.reference @@ -1,299 +1,442 @@ order by d1 nulls first -\N None -\N None -\N None -\N None -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,4] Array(Int64) -42 Int64 -42 Int64 -42 Int64 -42 Int64 -42 Int64 -43 Int64 -abc String -abc String -abc String -abc String -abc String -abd String +\N None false +\N None false +\N None false +\N None false +\N None false +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,4] Array(Int64) true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-02 Date true +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +43 Int64 false +abc String false +abc String false +abc String false +abc String false +abc String false +abc String false +abd String false order by d1 nulls last -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,4] Array(Int64) -42 Int64 -42 Int64 -42 Int64 -42 Int64 -42 Int64 -43 Int64 -abc String -abc String -abc String -abc String -abc String -abd String -\N None -\N None -\N None -\N None +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,4] Array(Int64) true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-02 Date true +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +43 Int64 false +abc String false +abc String false +abc String false +abc String false +abc String false +abc String false +abd String false +\N None false +\N None false +\N None false +\N None false +\N None false order by d2 nulls first -\N None -\N None -\N None -\N None -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,4] Array(Int64) -42 Int64 -42 Int64 -42 Int64 -42 Int64 -42 Int64 -43 Int64 -abc String -abc String -abc String -abc String -abc String -abd String +\N None false +\N None false +\N None false +\N None false +\N None false +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,4] Array(Int64) true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-02 Date true +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +43 Int64 false +abc String false +abc String false +abc String false +abc String false +abc String false +abc String false +abd String false order by d2 nulls last -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,4] Array(Int64) -42 Int64 -42 Int64 -42 Int64 -42 Int64 -42 Int64 -43 Int64 -abc String -abc String -abc String -abc String -abc String -abd String -\N None -\N None -\N None -\N None +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,4] Array(Int64) true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-02 Date true +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +43 Int64 false +abc String false +abc String false +abc String false +abc String false +abc String false +abc String false +abd String false +\N None false +\N None false +\N None false +\N None false +\N None false order by d1, d2 nulls first -[1,2,3] \N Array(Int64) None -[1,2,3] [1,2,3] Array(Int64) Array(Int64) -[1,2,3] [1,2,4] Array(Int64) Array(Int64) -[1,2,3] 42 Array(Int64) Int64 -[1,2,3] abc Array(Int64) String -[1,2,4] [1,2,3] Array(Int64) Array(Int64) -42 \N Int64 None -42 [1,2,3] Int64 Array(Int64) -42 42 Int64 Int64 -42 43 Int64 Int64 -42 abc Int64 String -43 42 Int64 Int64 -abc \N String None -abc [1,2,3] String Array(Int64) -abc 42 String Int64 -abc abc String String -abc abd String String -abd abc String String -\N \N None None -\N [1,2,3] None Array(Int64) -\N 42 None Int64 -\N abc None String +[1,2,3] \N Array(Int64) true None false +[1,2,3] [1,2,3] Array(Int64) true Array(Int64) true +[1,2,3] [1,2,4] Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 Array(Int64) true Date true +[1,2,3] 42 Array(Int64) true Int64 false +[1,2,3] abc Array(Int64) true String false +[1,2,4] [1,2,3] Array(Int64) true Array(Int64) true +2020-01-01 \N Date true None false +2020-01-01 [1,2,3] Date true Array(Int64) true +2020-01-01 2020-01-01 Date true Date true +2020-01-01 2020-01-02 Date true Date true +2020-01-01 42 Date true Int64 false +2020-01-01 abc Date true String false +2020-01-02 2020-01-01 Date true Date true +42 \N Int64 false None false +42 [1,2,3] Int64 false Array(Int64) true +42 2020-01-01 Int64 false Date true +42 42 Int64 false Int64 false +42 43 Int64 false Int64 false +42 abc Int64 false String false +43 42 Int64 false Int64 false +abc \N String false None false +abc [1,2,3] String false Array(Int64) true +abc 2020-01-01 String false Date true +abc 42 String false Int64 false +abc abc String false String false +abc abd String false String false +abd abc String false String false +\N \N None false None false +\N [1,2,3] None false Array(Int64) true +\N 2020-01-01 None false Date true +\N 42 None false Int64 false +\N abc None false String false order by d1, d2 nulls last -[1,2,3] [1,2,3] Array(Int64) Array(Int64) -[1,2,3] [1,2,4] Array(Int64) Array(Int64) -[1,2,3] 42 Array(Int64) Int64 -[1,2,3] abc Array(Int64) String -[1,2,3] \N Array(Int64) None -[1,2,4] [1,2,3] Array(Int64) Array(Int64) -42 [1,2,3] Int64 Array(Int64) -42 42 Int64 Int64 -42 43 Int64 Int64 -42 abc Int64 String -42 \N Int64 None -43 42 Int64 Int64 -abc [1,2,3] String Array(Int64) -abc 42 String Int64 -abc abc String String -abc abd String String -abc \N String None -abd abc String String -\N [1,2,3] None Array(Int64) -\N 42 None Int64 -\N abc None String -\N \N None None +[1,2,3] [1,2,3] Array(Int64) true Array(Int64) true +[1,2,3] [1,2,4] Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 Array(Int64) true Date true +[1,2,3] 42 Array(Int64) true Int64 false +[1,2,3] abc Array(Int64) true String false +[1,2,3] \N Array(Int64) true None false +[1,2,4] [1,2,3] Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] Date true Array(Int64) true +2020-01-01 2020-01-01 Date true Date true +2020-01-01 2020-01-02 Date true Date true +2020-01-01 42 Date true Int64 false +2020-01-01 abc Date true String false +2020-01-01 \N Date true None false +2020-01-02 2020-01-01 Date true Date true +42 [1,2,3] Int64 false Array(Int64) true +42 2020-01-01 Int64 false Date true +42 42 Int64 false Int64 false +42 43 Int64 false Int64 false +42 abc Int64 false String false +42 \N Int64 false None false +43 42 Int64 false Int64 false +abc [1,2,3] String false Array(Int64) true +abc 2020-01-01 String false Date true +abc 42 String false Int64 false +abc abc String false String false +abc abd String false String false +abc \N String false None false +abd abc String false String false +\N [1,2,3] None false Array(Int64) true +\N 2020-01-01 None false Date true +\N 42 None false Int64 false +\N abc None false String false +\N \N None false None false order by d2, d1 nulls first -\N [1,2,3] None Array(Int64) -[1,2,3] [1,2,3] Array(Int64) Array(Int64) -[1,2,4] [1,2,3] Array(Int64) Array(Int64) -42 [1,2,3] Int64 Array(Int64) -abc [1,2,3] String Array(Int64) -[1,2,3] [1,2,4] Array(Int64) Array(Int64) -\N 42 None Int64 -[1,2,3] 42 Array(Int64) Int64 -42 42 Int64 Int64 -43 42 Int64 Int64 -abc 42 String Int64 -42 43 Int64 Int64 -\N abc None String -[1,2,3] abc Array(Int64) String -42 abc Int64 String -abc abc String String -abd abc String String -abc abd String String -\N \N None None -[1,2,3] \N Array(Int64) None -42 \N Int64 None -abc \N String None +\N [1,2,3] None false Array(Int64) true +[1,2,3] [1,2,3] Array(Int64) true Array(Int64) true +[1,2,4] [1,2,3] Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] Date true Array(Int64) true +42 [1,2,3] Int64 false Array(Int64) true +abc [1,2,3] String false Array(Int64) true +[1,2,3] [1,2,4] Array(Int64) true Array(Int64) true +\N 2020-01-01 None false Date true +[1,2,3] 2020-01-01 Array(Int64) true Date true +2020-01-01 2020-01-01 Date true Date true +2020-01-02 2020-01-01 Date true Date true +42 2020-01-01 Int64 false Date true +abc 2020-01-01 String false Date true +2020-01-01 2020-01-02 Date true Date true +\N 42 None false Int64 false +[1,2,3] 42 Array(Int64) true Int64 false +2020-01-01 42 Date true Int64 false +42 42 Int64 false Int64 false +43 42 Int64 false Int64 false +abc 42 String false Int64 false +42 43 Int64 false Int64 false +\N abc None false String false +[1,2,3] abc Array(Int64) true String false +2020-01-01 abc Date true String false +42 abc Int64 false String false +abc abc String false String false +abd abc String false String false +abc abd String false String false +\N \N None false None false +[1,2,3] \N Array(Int64) true None false +2020-01-01 \N Date true None false +42 \N Int64 false None false +abc \N String false None false order by d2, d1 nulls last -[1,2,3] [1,2,3] Array(Int64) Array(Int64) -[1,2,4] [1,2,3] Array(Int64) Array(Int64) -42 [1,2,3] Int64 Array(Int64) -abc [1,2,3] String Array(Int64) -\N [1,2,3] None Array(Int64) -[1,2,3] [1,2,4] Array(Int64) Array(Int64) -[1,2,3] 42 Array(Int64) Int64 -42 42 Int64 Int64 -43 42 Int64 Int64 -abc 42 String Int64 -\N 42 None Int64 -42 43 Int64 Int64 -[1,2,3] abc Array(Int64) String -42 abc Int64 String -abc abc String String -abd abc String String -\N abc None String -abc abd String String -[1,2,3] \N Array(Int64) None -42 \N Int64 None -abc \N String None -\N \N None None +[1,2,3] [1,2,3] Array(Int64) true Array(Int64) true +[1,2,4] [1,2,3] Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] Date true Array(Int64) true +42 [1,2,3] Int64 false Array(Int64) true +abc [1,2,3] String false Array(Int64) true +\N [1,2,3] None false Array(Int64) true +[1,2,3] [1,2,4] Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 Array(Int64) true Date true +2020-01-01 2020-01-01 Date true Date true +2020-01-02 2020-01-01 Date true Date true +42 2020-01-01 Int64 false Date true +abc 2020-01-01 String false Date true +\N 2020-01-01 None false Date true +2020-01-01 2020-01-02 Date true Date true +[1,2,3] 42 Array(Int64) true Int64 false +2020-01-01 42 Date true Int64 false +42 42 Int64 false Int64 false +43 42 Int64 false Int64 false +abc 42 String false Int64 false +\N 42 None false Int64 false +42 43 Int64 false Int64 false +[1,2,3] abc Array(Int64) true String false +2020-01-01 abc Date true String false +42 abc Int64 false String false +abc abc String false String false +abd abc String false String false +\N abc None false String false +abc abd String false String false +[1,2,3] \N Array(Int64) true None false +2020-01-01 \N Date true None false +42 \N Int64 false None false +abc \N String false None false +\N \N None false None false d1 = d2 -[1,2,3] [1,2,3] 1 Array(Int64) Array(Int64) -[1,2,3] [1,2,4] 0 Array(Int64) Array(Int64) -[1,2,3] 42 0 Array(Int64) Int64 -[1,2,3] abc 0 Array(Int64) String -[1,2,3] \N 0 Array(Int64) None -[1,2,4] [1,2,3] 0 Array(Int64) Array(Int64) -42 [1,2,3] 0 Int64 Array(Int64) -42 42 1 Int64 Int64 -42 43 0 Int64 Int64 -42 abc 0 Int64 String -42 \N 0 Int64 None -43 42 0 Int64 Int64 -abc [1,2,3] 0 String Array(Int64) -abc 42 0 String Int64 -abc abc 1 String String -abc abd 0 String String -abc \N 0 String None -abd abc 0 String String -\N [1,2,3] 0 None Array(Int64) -\N 42 0 None Int64 -\N abc 0 None String -\N \N 1 None None +[1,2,3] [1,2,3] 1 Array(Int64) true Array(Int64) true +[1,2,3] [1,2,4] 0 Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 0 Array(Int64) true Date true +[1,2,3] 42 0 Array(Int64) true Int64 false +[1,2,3] abc 0 Array(Int64) true String false +[1,2,3] \N 0 Array(Int64) true None false +[1,2,4] [1,2,3] 0 Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] 0 Date true Array(Int64) true +2020-01-01 2020-01-01 1 Date true Date true +2020-01-01 2020-01-02 0 Date true Date true +2020-01-01 42 0 Date true Int64 false +2020-01-01 abc 0 Date true String false +2020-01-01 \N 0 Date true None false +2020-01-02 2020-01-01 0 Date true Date true +42 [1,2,3] 0 Int64 false Array(Int64) true +42 2020-01-01 0 Int64 false Date true +42 42 1 Int64 false Int64 false +42 43 0 Int64 false Int64 false +42 abc 0 Int64 false String false +42 \N 0 Int64 false None false +43 42 0 Int64 false Int64 false +abc [1,2,3] 0 String false Array(Int64) true +abc 2020-01-01 0 String false Date true +abc 42 0 String false Int64 false +abc abc 1 String false String false +abc abd 0 String false String false +abc \N 0 String false None false +abd abc 0 String false String false +\N [1,2,3] 0 None false Array(Int64) true +\N 2020-01-01 0 None false Date true +\N 42 0 None false Int64 false +\N abc 0 None false String false +\N \N 1 None false None false d1 < d2 -[1,2,3] [1,2,3] 0 Array(Int64) Array(Int64) -[1,2,3] [1,2,4] 1 Array(Int64) Array(Int64) -[1,2,3] 42 1 Array(Int64) Int64 -[1,2,3] abc 1 Array(Int64) String -[1,2,3] \N 1 Array(Int64) None -[1,2,4] [1,2,3] 0 Array(Int64) Array(Int64) -42 [1,2,3] 0 Int64 Array(Int64) -42 42 0 Int64 Int64 -42 43 1 Int64 Int64 -42 abc 1 Int64 String -42 \N 1 Int64 None -43 42 0 Int64 Int64 -abc [1,2,3] 0 String Array(Int64) -abc 42 0 String Int64 -abc abc 0 String String -abc abd 1 String String -abc \N 1 String None -abd abc 0 String String -\N [1,2,3] 0 None Array(Int64) -\N 42 0 None Int64 -\N abc 0 None String -\N \N 0 None None +[1,2,3] [1,2,3] 0 Array(Int64) true Array(Int64) true +[1,2,3] [1,2,4] 1 Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 1 Array(Int64) true Date true +[1,2,3] 42 1 Array(Int64) true Int64 false +[1,2,3] abc 1 Array(Int64) true String false +[1,2,3] \N 1 Array(Int64) true None false +[1,2,4] [1,2,3] 0 Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] 0 Date true Array(Int64) true +2020-01-01 2020-01-01 0 Date true Date true +2020-01-01 2020-01-02 1 Date true Date true +2020-01-01 42 1 Date true Int64 false +2020-01-01 abc 1 Date true String false +2020-01-01 \N 1 Date true None false +2020-01-02 2020-01-01 0 Date true Date true +42 [1,2,3] 0 Int64 false Array(Int64) true +42 2020-01-01 0 Int64 false Date true +42 42 0 Int64 false Int64 false +42 43 1 Int64 false Int64 false +42 abc 1 Int64 false String false +42 \N 1 Int64 false None false +43 42 0 Int64 false Int64 false +abc [1,2,3] 0 String false Array(Int64) true +abc 2020-01-01 0 String false Date true +abc 42 0 String false Int64 false +abc abc 0 String false String false +abc abd 1 String false String false +abc \N 1 String false None false +abd abc 0 String false String false +\N [1,2,3] 0 None false Array(Int64) true +\N 2020-01-01 0 None false Date true +\N 42 0 None false Int64 false +\N abc 0 None false String false +\N \N 0 None false None false d1 <= d2 -[1,2,3] [1,2,3] 1 Array(Int64) Array(Int64) -[1,2,3] [1,2,4] 1 Array(Int64) Array(Int64) -[1,2,3] 42 1 Array(Int64) Int64 -[1,2,3] abc 1 Array(Int64) String -[1,2,3] \N 1 Array(Int64) None -[1,2,4] [1,2,3] 0 Array(Int64) Array(Int64) -42 [1,2,3] 0 Int64 Array(Int64) -42 42 1 Int64 Int64 -42 43 1 Int64 Int64 -42 abc 1 Int64 String -42 \N 1 Int64 None -43 42 0 Int64 Int64 -abc [1,2,3] 0 String Array(Int64) -abc 42 0 String Int64 -abc abc 1 String String -abc abd 1 String String -abc \N 1 String None -abd abc 0 String String -\N [1,2,3] 0 None Array(Int64) -\N 42 0 None Int64 -\N abc 0 None String -\N \N 1 None None +[1,2,3] [1,2,3] 1 Array(Int64) true Array(Int64) true +[1,2,3] [1,2,4] 1 Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 1 Array(Int64) true Date true +[1,2,3] 42 1 Array(Int64) true Int64 false +[1,2,3] abc 1 Array(Int64) true String false +[1,2,3] \N 1 Array(Int64) true None false +[1,2,4] [1,2,3] 0 Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] 0 Date true Array(Int64) true +2020-01-01 2020-01-01 1 Date true Date true +2020-01-01 2020-01-02 1 Date true Date true +2020-01-01 42 1 Date true Int64 false +2020-01-01 abc 1 Date true String false +2020-01-01 \N 1 Date true None false +2020-01-02 2020-01-01 0 Date true Date true +42 [1,2,3] 0 Int64 false Array(Int64) true +42 2020-01-01 0 Int64 false Date true +42 42 1 Int64 false Int64 false +42 43 1 Int64 false Int64 false +42 abc 1 Int64 false String false +42 \N 1 Int64 false None false +43 42 0 Int64 false Int64 false +abc [1,2,3] 0 String false Array(Int64) true +abc 2020-01-01 0 String false Date true +abc 42 0 String false Int64 false +abc abc 1 String false String false +abc abd 1 String false String false +abc \N 1 String false None false +abd abc 0 String false String false +\N [1,2,3] 0 None false Array(Int64) true +\N 2020-01-01 0 None false Date true +\N 42 0 None false Int64 false +\N abc 0 None false String false +\N \N 1 None false None false d1 > d2 -[1,2,3] [1,2,3] 0 Array(Int64) Array(Int64) -[1,2,3] [1,2,4] 0 Array(Int64) Array(Int64) -[1,2,3] 42 0 Array(Int64) Int64 -[1,2,3] abc 0 Array(Int64) String -[1,2,3] \N 0 Array(Int64) None -[1,2,4] [1,2,3] 1 Array(Int64) Array(Int64) -42 [1,2,3] 1 Int64 Array(Int64) -42 42 0 Int64 Int64 -42 43 0 Int64 Int64 -42 abc 0 Int64 String -42 \N 0 Int64 None -43 42 1 Int64 Int64 -abc [1,2,3] 1 String Array(Int64) -abc 42 1 String Int64 -abc abc 0 String String -abc abd 0 String String -abc \N 0 String None -abd abc 1 String String -\N [1,2,3] 1 None Array(Int64) -\N 42 1 None Int64 -\N abc 1 None String -\N \N 0 None None +[1,2,3] [1,2,3] 0 Array(Int64) true Array(Int64) true +[1,2,3] [1,2,4] 0 Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 0 Array(Int64) true Date true +[1,2,3] 42 0 Array(Int64) true Int64 false +[1,2,3] abc 0 Array(Int64) true String false +[1,2,3] \N 0 Array(Int64) true None false +[1,2,4] [1,2,3] 1 Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] 1 Date true Array(Int64) true +2020-01-01 2020-01-01 0 Date true Date true +2020-01-01 2020-01-02 0 Date true Date true +2020-01-01 42 0 Date true Int64 false +2020-01-01 abc 0 Date true String false +2020-01-01 \N 0 Date true None false +2020-01-02 2020-01-01 1 Date true Date true +42 [1,2,3] 1 Int64 false Array(Int64) true +42 2020-01-01 1 Int64 false Date true +42 42 0 Int64 false Int64 false +42 43 0 Int64 false Int64 false +42 abc 0 Int64 false String false +42 \N 0 Int64 false None false +43 42 1 Int64 false Int64 false +abc [1,2,3] 1 String false Array(Int64) true +abc 2020-01-01 1 String false Date true +abc 42 1 String false Int64 false +abc abc 0 String false String false +abc abd 0 String false String false +abc \N 0 String false None false +abd abc 1 String false String false +\N [1,2,3] 1 None false Array(Int64) true +\N 2020-01-01 1 None false Date true +\N 42 1 None false Int64 false +\N abc 1 None false String false +\N \N 0 None false None false d1 >= d2 -[1,2,3] [1,2,3] 1 Array(Int64) Array(Int64) -[1,2,3] [1,2,4] 1 Array(Int64) Array(Int64) -[1,2,3] 42 1 Array(Int64) Int64 -[1,2,3] abc 1 Array(Int64) String -[1,2,3] \N 1 Array(Int64) None -[1,2,4] [1,2,3] 1 Array(Int64) Array(Int64) -42 [1,2,3] 1 Int64 Array(Int64) -42 42 1 Int64 Int64 -42 43 1 Int64 Int64 -42 abc 1 Int64 String -42 \N 1 Int64 None -43 42 1 Int64 Int64 -abc [1,2,3] 1 String Array(Int64) -abc 42 1 String Int64 -abc abc 1 String String -abc abd 1 String String -abc \N 1 String None -abd abc 1 String String -\N [1,2,3] 1 None Array(Int64) -\N 42 1 None Int64 -\N abc 1 None String -\N \N 1 None None +[1,2,3] [1,2,3] 1 Array(Int64) true Array(Int64) true +[1,2,3] [1,2,4] 1 Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 1 Array(Int64) true Date true +[1,2,3] 42 1 Array(Int64) true Int64 false +[1,2,3] abc 1 Array(Int64) true String false +[1,2,3] \N 1 Array(Int64) true None false +[1,2,4] [1,2,3] 1 Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] 1 Date true Array(Int64) true +2020-01-01 2020-01-01 1 Date true Date true +2020-01-01 2020-01-02 1 Date true Date true +2020-01-01 42 1 Date true Int64 false +2020-01-01 abc 1 Date true String false +2020-01-01 \N 1 Date true None false +2020-01-02 2020-01-01 1 Date true Date true +42 [1,2,3] 1 Int64 false Array(Int64) true +42 2020-01-01 1 Int64 false Date true +42 42 1 Int64 false Int64 false +42 43 1 Int64 false Int64 false +42 abc 1 Int64 false String false +42 \N 1 Int64 false None false +43 42 1 Int64 false Int64 false +abc [1,2,3] 1 String false Array(Int64) true +abc 2020-01-01 1 String false Date true +abc 42 1 String false Int64 false +abc abc 1 String false String false +abc abd 1 String false String false +abc \N 1 String false None false +abd abc 1 String false String false +\N [1,2,3] 1 None false Array(Int64) true +\N 2020-01-01 1 None false Date true +\N 42 1 None false Int64 false +\N abc 1 None false String false +\N \N 1 None false None false diff --git a/tests/queries/0_stateless/03035_dynamic_sorting.sql b/tests/queries/0_stateless/03035_dynamic_sorting.sql index 0487fafc955..e0039a348c6 100644 --- a/tests/queries/0_stateless/03035_dynamic_sorting.sql +++ b/tests/queries/0_stateless/03035_dynamic_sorting.sql @@ -1,80 +1,55 @@ set allow_experimental_dynamic_type = 1; drop table if exists test; -create table test (d1 Dynamic, d2 Dynamic) engine=Memory; - -insert into test values (42, 42); -insert into test values (42, 43); -insert into test values (43, 42); - -insert into test values ('abc', 'abc'); -insert into test values ('abc', 'abd'); -insert into test values ('abd', 'abc'); - -insert into test values ([1,2,3], [1,2,3]); -insert into test values ([1,2,3], [1,2,4]); -insert into test values ([1,2,4], [1,2,3]); - -insert into test values (NULL, NULL); - -insert into test values (42, 'abc'); -insert into test values ('abc', 42); - -insert into test values (42, [1,2,3]); -insert into test values ([1,2,3], 42); - -insert into test values (42, NULL); -insert into test values (NULL, 42); - -insert into test values ('abc', [1,2,3]); -insert into test values ([1,2,3], 'abc'); - -insert into test values ('abc', NULL); -insert into test values (NULL, 'abc'); - -insert into test values ([1,2,3], NULL); -insert into test values (NULL, [1,2,3]); +create table test (d1 Dynamic(max_types=2), d2 Dynamic(max_types=2)) engine=Memory; +insert into test values (42, 42), (42, 43), (43, 42), ('abc', 'abc'), ('abc', 'abd'), ('abd', 'abc'), +([1,2,3], [1,2,3]), ([1,2,3], [1,2,4]), ([1,2,4], [1,2,3]), +('2020-01-01', '2020-01-01'), ('2020-01-01', '2020-01-02'), ('2020-01-02', '2020-01-01'), +(NULL, NULL), (42, 'abc'), ('abc', 42), (42, [1,2,3]), ([1,2,3], 42), (42, NULL), (NULL, 42), +('abc', [1,2,3]), ([1,2,3], 'abc'), ('abc', NULL), (NULL, 'abc'), ([1,2,3], NULL), (NULL, [1,2,3]), +(42, '2020-01-01'), ('2020-01-01', 42), ('2020-01-01', 'abc'), ('abc', '2020-01-01'), +('2020-01-01', [1,2,3]), ([1,2,3], '2020-01-01'), ('2020-01-01', NULL), (NULL, '2020-01-01'); select 'order by d1 nulls first'; -select d1, dynamicType(d1) from test order by d1 nulls first; +select d1, dynamicType(d1), isDynamicElementInSharedData(d1) from test order by d1 nulls first; select 'order by d1 nulls last'; -select d1, dynamicType(d1) from test order by d1 nulls last; +select d1, dynamicType(d1), isDynamicElementInSharedData(d1) from test order by d1 nulls last; select 'order by d2 nulls first'; -select d2, dynamicType(d2) from test order by d2 nulls first; +select d2, dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d2 nulls first; select 'order by d2 nulls last'; -select d2, dynamicType(d2) from test order by d2 nulls last; +select d2, dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d2 nulls last; select 'order by d1, d2 nulls first'; -select d1, d2, dynamicType(d1), dynamicType(d2) from test order by d1, d2 nulls first; +select d1, d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d1, d2 nulls first; select 'order by d1, d2 nulls last'; -select d1, d2, dynamicType(d1), dynamicType(d2) from test order by d1, d2 nulls last; +select d1, d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d1, d2 nulls last; select 'order by d2, d1 nulls first'; -select d1, d2, dynamicType(d1), dynamicType(d2) from test order by d2, d1 nulls first; +select d1, d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d2, d1 nulls first; select 'order by d2, d1 nulls last'; -select d1, d2, dynamicType(d1), dynamicType(d2) from test order by d2, d1 nulls last; +select d1, d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d2, d1 nulls last; select 'd1 = d2'; -select d1, d2, d1 = d2, dynamicType(d1), dynamicType(d2) from test order by d1, d2; +select d1, d2, d1 = d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d1, d2; select 'd1 < d2'; -select d1, d2, d1 < d2, dynamicType(d1), dynamicType(d2) from test order by d1, d2; +select d1, d2, d1 < d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d1, d2; select 'd1 <= d2'; -select d1, d2, d1 <= d2, dynamicType(d1), dynamicType(d2) from test order by d1, d2; +select d1, d2, d1 <= d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d1, d2; select 'd1 > d2'; -select d1, d2, d1 > d2, dynamicType(d1), dynamicType(d2) from test order by d1, d2; +select d1, d2, d1 > d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d1, d2; select 'd1 >= d2'; -select d1, d2, d2 >= d2, dynamicType(d1), dynamicType(d2) from test order by d1, d2; +select d1, d2, d2 >= d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d1, d2; drop table test; diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.reference b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.reference new file mode 100644 index 00000000000..ca6c5dbba82 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.reference @@ -0,0 +1,20 @@ +Array(Array(Dynamic)) +Array(Variant(String, UInt64)) +LowCardinality(String) +None +String +UInt64 +360000 +360000 +200000 +200000 +0 +0 +20000 +20000 +200000 +200000 +20000 +20000 +200000 +0 diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.sql b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.sql new file mode 100644 index 00000000000..bff28fb5c90 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.sql @@ -0,0 +1,43 @@ +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan + +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; + +drop table if exists test; +create table test (id UInt64, d Dynamic(max_types=2)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000; + +insert into test select number, number from numbers(100000) settings min_insert_block_size_rows=50000; +insert into test select number, 'str_' || toString(number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1)) from numbers(200000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, NULL from numbers(300000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, multiIf(number % 4 == 3, 'str_' || toString(number), number % 4 == 2, NULL, number % 4 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1))) from numbers(400000, 400000) settings min_insert_block_size_rows=50000; +insert into test select number, if (number % 5 == 1, [range((number % 10 + 1)::UInt64)]::Array(Array(Dynamic)), number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, if (number % 5 == 1, ('str_' || number)::LowCardinality(String)::Dynamic, number::Dynamic) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; + +select distinct dynamicType(d) as type from test order by type; +select count() from test where dynamicType(d) == 'UInt64'; +select count() from test where d.UInt64 is not NULL; +select count() from test where dynamicType(d) == 'String'; +select count() from test where d.String is not NULL; +select count() from test where dynamicType(d) == 'Date'; +select count() from test where d.Date is not NULL; +select count() from test where dynamicType(d) == 'LowCardinality(String)'; +select count() from test where d.`LowCardinality(String)` is not NULL; +select count() from test where dynamicType(d) == 'Array(Variant(String, UInt64))'; +select count() from test where not empty(d.`Array(Variant(String, UInt64))`); +select count() from test where dynamicType(d) == 'Array(Array(Dynamic))'; +select count() from test where not empty(d.`Array(Array(Dynamic))`); +select count() from test where d is NULL; +select count() from test where not empty(d.`Tuple(a Array(Dynamic))`.a.String); + +select d, d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.Int8, d.Date, d.`LowCardinality(String)`, d.`Array(String)` from test format Null; +select d, d.UInt64, d.Date, d.`LowCardinality(String)`, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.UInt64, d.Date, d.`LowCardinality(String)`, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64, d.`Array(Variant(String, UInt64))`.String from test format Null; +select d, d.`Tuple(a UInt64, b String)`.a, d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Dynamic)`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.`Array(Array(Dynamic))`.size1, d.`Array(Array(Dynamic))`.UInt64, d.`Array(Array(Dynamic))`.`Map(String, Tuple(a UInt64))`.values.a from test format Null; + +drop table test; diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.reference b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.reference new file mode 100644 index 00000000000..ca6c5dbba82 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.reference @@ -0,0 +1,20 @@ +Array(Array(Dynamic)) +Array(Variant(String, UInt64)) +LowCardinality(String) +None +String +UInt64 +360000 +360000 +200000 +200000 +0 +0 +20000 +20000 +200000 +200000 +20000 +20000 +200000 +0 diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.sql b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.sql new file mode 100644 index 00000000000..4eed3d15529 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.sql @@ -0,0 +1,43 @@ +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan + +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; + +drop table if exists test; +create table test (id UInt64, d Dynamic(max_types=2)) engine=Memory; + +insert into test select number, number from numbers(100000) settings min_insert_block_size_rows=50000; +insert into test select number, 'str_' || toString(number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1)) from numbers(200000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, NULL from numbers(300000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, multiIf(number % 4 == 3, 'str_' || toString(number), number % 4 == 2, NULL, number % 4 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1))) from numbers(400000, 400000) settings min_insert_block_size_rows=50000; +insert into test select number, if (number % 5 == 1, [range((number % 10 + 1)::UInt64)]::Array(Array(Dynamic)), number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, if (number % 5 == 1, ('str_' || number)::LowCardinality(String)::Dynamic, number::Dynamic) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; + +select distinct dynamicType(d) as type from test order by type; +select count() from test where dynamicType(d) == 'UInt64'; +select count() from test where d.UInt64 is not NULL; +select count() from test where dynamicType(d) == 'String'; +select count() from test where d.String is not NULL; +select count() from test where dynamicType(d) == 'Date'; +select count() from test where d.Date is not NULL; +select count() from test where dynamicType(d) == 'LowCardinality(String)'; +select count() from test where d.`LowCardinality(String)` is not NULL; +select count() from test where dynamicType(d) == 'Array(Variant(String, UInt64))'; +select count() from test where not empty(d.`Array(Variant(String, UInt64))`); +select count() from test where dynamicType(d) == 'Array(Array(Dynamic))'; +select count() from test where not empty(d.`Array(Array(Dynamic))`); +select count() from test where d is NULL; +select count() from test where not empty(d.`Tuple(a Array(Dynamic))`.a.String); + +select d, d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.Int8, d.Date, d.`Array(String)` from test format Null; +select d, d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64, d.`Array(Variant(String, UInt64))`.String from test format Null; +select d, d.`Tuple(a UInt64, b String)`.a, d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Dynamic)`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.`Array(Array(Dynamic))`.size1, d.`Array(Array(Dynamic))`.UInt64, d.`Array(Array(Dynamic))`.`Map(String, Tuple(a UInt64))`.values.a from test format Null; + +drop table test; diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 new file mode 100644 index 00000000000..9c1f8fa45e8 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 @@ -0,0 +1,2460 @@ +Array(Array(Dynamic)) +Array(Variant(String, UInt64)) +LowCardinality(String) +None +String +UInt64 +36 +36 +20 +20 +0 +0 +2 +2 +20 +20 +2 +2 +20 +0 +0 0 \N [] +1 1 \N [] +2 2 \N [] +3 3 \N [] +4 4 \N [] +5 5 \N [] +6 6 \N [] +7 7 \N [] +8 8 \N [] +9 9 \N [] +str_10 \N str_10 [] +10 10 \N [] +10 10 \N [] +[[0,1]] \N \N [] +str_11 \N \N [] +str_11 \N str_11 [] +str_12 \N str_12 [] +12 12 \N [] +12 12 \N [] +str_13 \N str_13 [] +13 13 \N [] +13 13 \N [] +str_14 \N str_14 [] +14 14 \N [] +14 14 \N [] +str_15 \N str_15 [] +15 15 \N [] +15 15 \N [] +[[0,1,2,3,4,5,6]] \N \N [] +str_16 \N \N [] +str_16 \N str_16 [] +str_17 \N str_17 [] +17 17 \N [] +17 17 \N [] +str_18 \N str_18 [] +18 18 \N [] +18 18 \N [] +str_19 \N str_19 [] +19 19 \N [] +19 19 \N [] +[20] \N \N [20] +['str_21','str_21'] \N \N ['str_21','str_21'] +[22,22,22] \N \N [22,22,22] +[23,23,23,23] \N \N [23,23,23,23] +[24,24,24,24,24] \N \N [24,24,24,24,24] +[25,25,25,25,25,25] \N \N [25,25,25,25,25,25] +[26,26,26,26,26,26,26] \N \N [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] \N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] \N \N [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] \N \N [29,29,29,29,29,29,29,29,29,29] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +[40] \N \N [40] +41 41 \N [] +\N \N \N [] +str_43 \N str_43 [] +[44,44,44,44,44] \N \N [44,44,44,44,44] +45 45 \N [] +\N \N \N [] +str_47 \N str_47 [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] \N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 49 \N [] +\N \N \N [] +str_51 \N str_51 [] +[52,52,52] \N \N [52,52,52] +53 53 \N [] +\N \N \N [] +str_55 \N str_55 [] +[56,56,56,56,56,56,56] \N \N [56,56,56,56,56,56,56] +57 57 \N [] +\N \N \N [] +str_59 \N str_59 [] +[60] \N \N [60] +61 61 \N [] +\N \N \N [] +str_63 \N str_63 [] +[64,64,64,64,64] \N \N [64,64,64,64,64] +65 65 \N [] +\N \N \N [] +str_67 \N str_67 [] +[68,68,68,68,68,68,68,68,68] \N \N [68,68,68,68,68,68,68,68,68] +69 69 \N [] +\N \N \N [] +str_71 \N str_71 [] +[NULL,NULL,NULL] \N \N [NULL,NULL,NULL] +73 73 \N [] +\N \N \N [] +str_75 \N str_75 [] +[76,76,76,76,76,76,76] \N \N [76,76,76,76,76,76,76] +77 77 \N [] +\N \N \N [] +str_79 \N str_79 [] +0 \N [] +1 \N [] +2 \N [] +3 \N [] +4 \N [] +5 \N [] +6 \N [] +7 \N [] +8 \N [] +9 \N [] +\N str_10 [] +10 \N [] +10 \N [] +\N \N [] +\N \N [] +\N str_11 [] +\N str_12 [] +12 \N [] +12 \N [] +\N str_13 [] +13 \N [] +13 \N [] +\N str_14 [] +14 \N [] +14 \N [] +\N str_15 [] +15 \N [] +15 \N [] +\N \N [] +\N \N [] +\N str_16 [] +\N str_17 [] +17 \N [] +17 \N [] +\N str_18 [] +18 \N [] +18 \N [] +\N str_19 [] +19 \N [] +19 \N [] +\N \N [20] +\N \N ['str_21','str_21'] +\N \N [22,22,22] +\N \N [23,23,23,23] +\N \N [24,24,24,24,24] +\N \N [25,25,25,25,25,25] +\N \N [26,26,26,26,26,26,26] +\N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [28,28,28,28,28,28,28,28,28] +\N \N [29,29,29,29,29,29,29,29,29,29] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [40] +41 \N [] +\N \N [] +\N str_43 [] +\N \N [44,44,44,44,44] +45 \N [] +\N \N [] +\N str_47 [] +\N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 \N [] +\N \N [] +\N str_51 [] +\N \N [52,52,52] +53 \N [] +\N \N [] +\N str_55 [] +\N \N [56,56,56,56,56,56,56] +57 \N [] +\N \N [] +\N str_59 [] +\N \N [60] +61 \N [] +\N \N [] +\N str_63 [] +\N \N [64,64,64,64,64] +65 \N [] +\N \N [] +\N str_67 [] +\N \N [68,68,68,68,68,68,68,68,68] +69 \N [] +\N \N [] +\N str_71 [] +\N \N [NULL,NULL,NULL] +73 \N [] +\N \N [] +\N str_75 [] +\N \N [76,76,76,76,76,76,76] +77 \N [] +\N \N [] +\N str_79 [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +0 0 \N [] 0 [] +1 1 \N [] 0 [] +2 2 \N [] 0 [] +3 3 \N [] 0 [] +4 4 \N [] 0 [] +5 5 \N [] 0 [] +6 6 \N [] 0 [] +7 7 \N [] 0 [] +8 8 \N [] 0 [] +9 9 \N [] 0 [] +str_10 \N \N [] 0 [] +10 10 \N [] 0 [] +10 10 \N [] 0 [] +[[0,1]] \N \N [] 0 [] +str_11 \N \N [] 0 [] +str_11 \N \N [] 0 [] +str_12 \N \N [] 0 [] +12 12 \N [] 0 [] +12 12 \N [] 0 [] +str_13 \N \N [] 0 [] +13 13 \N [] 0 [] +13 13 \N [] 0 [] +str_14 \N \N [] 0 [] +14 14 \N [] 0 [] +14 14 \N [] 0 [] +str_15 \N \N [] 0 [] +15 15 \N [] 0 [] +15 15 \N [] 0 [] +[[0,1,2,3,4,5,6]] \N \N [] 0 [] +str_16 \N \N [] 0 [] +str_16 \N \N [] 0 [] +str_17 \N \N [] 0 [] +17 17 \N [] 0 [] +17 17 \N [] 0 [] +str_18 \N \N [] 0 [] +18 18 \N [] 0 [] +18 18 \N [] 0 [] +str_19 \N \N [] 0 [] +19 19 \N [] 0 [] +19 19 \N [] 0 [] +[20] \N \N [20] 1 [20] +['str_21','str_21'] \N \N ['str_21','str_21'] 2 [NULL,NULL] +[22,22,22] \N \N [22,22,22] 3 [22,22,22] +[23,23,23,23] \N \N [23,23,23,23] 4 [23,23,23,23] +[24,24,24,24,24] \N \N [24,24,24,24,24] 5 [24,24,24,24,24] +[25,25,25,25,25,25] \N \N [25,25,25,25,25,25] 6 [25,25,25,25,25,25] +[26,26,26,26,26,26,26] \N \N [26,26,26,26,26,26,26] 7 [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] \N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 8 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] \N \N [28,28,28,28,28,28,28,28,28] 9 [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] \N \N [29,29,29,29,29,29,29,29,29,29] 10 [29,29,29,29,29,29,29,29,29,29] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +[40] \N \N [40] 1 [40] +41 41 \N [] 0 [] +\N \N \N [] 0 [] +str_43 \N \N [] 0 [] +[44,44,44,44,44] \N \N [44,44,44,44,44] 5 [44,44,44,44,44] +45 45 \N [] 0 [] +\N \N \N [] 0 [] +str_47 \N \N [] 0 [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] \N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 9 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +49 49 \N [] 0 [] +\N \N \N [] 0 [] +str_51 \N \N [] 0 [] +[52,52,52] \N \N [52,52,52] 3 [52,52,52] +53 53 \N [] 0 [] +\N \N \N [] 0 [] +str_55 \N \N [] 0 [] +[56,56,56,56,56,56,56] \N \N [56,56,56,56,56,56,56] 7 [56,56,56,56,56,56,56] +57 57 \N [] 0 [] +\N \N \N [] 0 [] +str_59 \N \N [] 0 [] +[60] \N \N [60] 1 [60] +61 61 \N [] 0 [] +\N \N \N [] 0 [] +str_63 \N \N [] 0 [] +[64,64,64,64,64] \N \N [64,64,64,64,64] 5 [64,64,64,64,64] +65 65 \N [] 0 [] +\N \N \N [] 0 [] +str_67 \N \N [] 0 [] +[68,68,68,68,68,68,68,68,68] \N \N [68,68,68,68,68,68,68,68,68] 9 [68,68,68,68,68,68,68,68,68] +69 69 \N [] 0 [] +\N \N \N [] 0 [] +str_71 \N \N [] 0 [] +[NULL,NULL,NULL] \N \N [NULL,NULL,NULL] 3 [NULL,NULL,NULL] +73 73 \N [] 0 [] +\N \N \N [] 0 [] +str_75 \N \N [] 0 [] +[76,76,76,76,76,76,76] \N \N [76,76,76,76,76,76,76] 7 [76,76,76,76,76,76,76] +77 77 \N [] 0 [] +\N \N \N [] 0 [] +str_79 \N \N [] 0 [] +0 \N [] 0 [] [] +1 \N [] 0 [] [] +2 \N [] 0 [] [] +3 \N [] 0 [] [] +4 \N [] 0 [] [] +5 \N [] 0 [] [] +6 \N [] 0 [] [] +7 \N [] 0 [] [] +8 \N [] 0 [] [] +9 \N [] 0 [] [] +\N \N [] 0 [] [] +10 \N [] 0 [] [] +10 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +12 \N [] 0 [] [] +12 \N [] 0 [] [] +\N \N [] 0 [] [] +13 \N [] 0 [] [] +13 \N [] 0 [] [] +\N \N [] 0 [] [] +14 \N [] 0 [] [] +14 \N [] 0 [] [] +\N \N [] 0 [] [] +15 \N [] 0 [] [] +15 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +17 \N [] 0 [] [] +17 \N [] 0 [] [] +\N \N [] 0 [] [] +18 \N [] 0 [] [] +18 \N [] 0 [] [] +\N \N [] 0 [] [] +19 \N [] 0 [] [] +19 \N [] 0 [] [] +\N \N [20] 1 [20] [NULL] +\N \N ['str_21','str_21'] 2 [NULL,NULL] ['str_21','str_21'] +\N \N [22,22,22] 3 [22,22,22] [NULL,NULL,NULL] +\N \N [23,23,23,23] 4 [23,23,23,23] [NULL,NULL,NULL,NULL] +\N \N [24,24,24,24,24] 5 [24,24,24,24,24] [NULL,NULL,NULL,NULL,NULL] +\N \N [25,25,25,25,25,25] 6 [25,25,25,25,25,25] [NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [26,26,26,26,26,26,26] 7 [26,26,26,26,26,26,26] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 8 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [28,28,28,28,28,28,28,28,28] 9 [28,28,28,28,28,28,28,28,28] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [29,29,29,29,29,29,29,29,29,29] 10 [29,29,29,29,29,29,29,29,29,29] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [40] 1 [40] [NULL] +41 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [44,44,44,44,44] 5 [44,44,44,44,44] [NULL,NULL,NULL,NULL,NULL] +45 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 9 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [52,52,52] 3 [52,52,52] [NULL,NULL,NULL] +53 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [56,56,56,56,56,56,56] 7 [56,56,56,56,56,56,56] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +57 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [60] 1 [60] [NULL] +61 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [64,64,64,64,64] 5 [64,64,64,64,64] [NULL,NULL,NULL,NULL,NULL] +65 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [68,68,68,68,68,68,68,68,68] 9 [68,68,68,68,68,68,68,68,68] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +69 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [NULL,NULL,NULL] 3 [NULL,NULL,NULL] [NULL,NULL,NULL] +73 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [76,76,76,76,76,76,76] 7 [76,76,76,76,76,76,76] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +77 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +0 0 [] [] +1 0 [] [] +2 0 [] [] +3 0 [] [] +4 0 [] [] +5 0 [] [] +6 0 [] [] +7 0 [] [] +8 0 [] [] +9 0 [] [] +str_10 0 [] [] +10 0 [] [] +10 0 [] [] +[[0,1]] 0 [] [] +str_11 0 [] [] +str_11 0 [] [] +str_12 0 [] [] +12 0 [] [] +12 0 [] [] +str_13 0 [] [] +13 0 [] [] +13 0 [] [] +str_14 0 [] [] +14 0 [] [] +14 0 [] [] +str_15 0 [] [] +15 0 [] [] +15 0 [] [] +[[0,1,2,3,4,5,6]] 0 [] [] +str_16 0 [] [] +str_16 0 [] [] +str_17 0 [] [] +17 0 [] [] +17 0 [] [] +str_18 0 [] [] +18 0 [] [] +18 0 [] [] +str_19 0 [] [] +19 0 [] [] +19 0 [] [] +[20] 0 [] [20] +['str_21','str_21'] 0 [] [NULL,NULL] +[22,22,22] 0 [] [22,22,22] +[23,23,23,23] 0 [] [23,23,23,23] +[24,24,24,24,24] 0 [] [24,24,24,24,24] +[25,25,25,25,25,25] 0 [] [25,25,25,25,25,25] +[26,26,26,26,26,26,26] 0 [] [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 0 [] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] 0 [] [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] 0 [] [29,29,29,29,29,29,29,29,29,29] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +[40] 0 [] [40] +41 0 [] [] +\N 0 [] [] +str_43 0 [] [] +[44,44,44,44,44] 0 [] [44,44,44,44,44] +45 0 [] [] +\N 0 [] [] +str_47 0 [] [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 0 [] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +49 0 [] [] +\N 0 [] [] +str_51 0 [] [] +[52,52,52] 0 [] [52,52,52] +53 0 [] [] +\N 0 [] [] +str_55 0 [] [] +[56,56,56,56,56,56,56] 0 [] [56,56,56,56,56,56,56] +57 0 [] [] +\N 0 [] [] +str_59 0 [] [] +[60] 0 [] [60] +61 0 [] [] +\N 0 [] [] +str_63 0 [] [] +[64,64,64,64,64] 0 [] [64,64,64,64,64] +65 0 [] [] +\N 0 [] [] +str_67 0 [] [] +[68,68,68,68,68,68,68,68,68] 0 [] [68,68,68,68,68,68,68,68,68] +69 0 [] [] +\N 0 [] [] +str_71 0 [] [] +[NULL,NULL,NULL] 0 [] [NULL,NULL,NULL] +73 0 [] [] +\N 0 [] [] +str_75 0 [] [] +[76,76,76,76,76,76,76] 0 [] [76,76,76,76,76,76,76] +77 0 [] [] +\N 0 [] [] +str_79 0 [] [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [20] +[] 0 [NULL,NULL] +[] 0 [22,22,22] +[] 0 [23,23,23,23] +[] 0 [24,24,24,24,24] +[] 0 [25,25,25,25,25,25] +[] 0 [26,26,26,26,26,26,26] +[] 0 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[] 0 [28,28,28,28,28,28,28,28,28] +[] 0 [29,29,29,29,29,29,29,29,29,29] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [40] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [44,44,44,44,44] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [52,52,52] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [56,56,56,56,56,56,56] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [60] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [64,64,64,64,64] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [68,68,68,68,68,68,68,68,68] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [NULL,NULL,NULL] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [76,76,76,76,76,76,76] +[] 0 [] +[] 0 [] +[] 0 [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[2] [[0,1]] [[[],[]]] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[7] [[0,1,2,3,4,5,6]] [[[],[],[],[],[],[],[]]] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +Array(Array(Dynamic)) +Array(Variant(String, UInt64)) +LowCardinality(String) +None +String +UInt64 +36 +36 +20 +20 +0 +0 +2 +2 +20 +20 +2 +2 +20 +0 +0 0 \N [] +1 1 \N [] +2 2 \N [] +3 3 \N [] +4 4 \N [] +5 5 \N [] +6 6 \N [] +7 7 \N [] +8 8 \N [] +9 9 \N [] +str_10 \N str_10 [] +10 10 \N [] +10 10 \N [] +[[0,1]] \N \N [] +str_11 \N \N [] +str_11 \N str_11 [] +str_12 \N str_12 [] +12 12 \N [] +12 12 \N [] +str_13 \N str_13 [] +13 13 \N [] +13 13 \N [] +str_14 \N str_14 [] +14 14 \N [] +14 14 \N [] +str_15 \N str_15 [] +15 15 \N [] +15 15 \N [] +[[0,1,2,3,4,5,6]] \N \N [] +str_16 \N \N [] +str_16 \N str_16 [] +str_17 \N str_17 [] +17 17 \N [] +17 17 \N [] +str_18 \N str_18 [] +18 18 \N [] +18 18 \N [] +str_19 \N str_19 [] +19 19 \N [] +19 19 \N [] +[20] \N \N [20] +['str_21','str_21'] \N \N ['str_21','str_21'] +[22,22,22] \N \N [22,22,22] +[23,23,23,23] \N \N [23,23,23,23] +[24,24,24,24,24] \N \N [24,24,24,24,24] +[25,25,25,25,25,25] \N \N [25,25,25,25,25,25] +[26,26,26,26,26,26,26] \N \N [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] \N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] \N \N [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] \N \N [29,29,29,29,29,29,29,29,29,29] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +[40] \N \N [40] +41 41 \N [] +\N \N \N [] +str_43 \N str_43 [] +[44,44,44,44,44] \N \N [44,44,44,44,44] +45 45 \N [] +\N \N \N [] +str_47 \N str_47 [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] \N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 49 \N [] +\N \N \N [] +str_51 \N str_51 [] +[52,52,52] \N \N [52,52,52] +53 53 \N [] +\N \N \N [] +str_55 \N str_55 [] +[56,56,56,56,56,56,56] \N \N [56,56,56,56,56,56,56] +57 57 \N [] +\N \N \N [] +str_59 \N str_59 [] +[60] \N \N [60] +61 61 \N [] +\N \N \N [] +str_63 \N str_63 [] +[64,64,64,64,64] \N \N [64,64,64,64,64] +65 65 \N [] +\N \N \N [] +str_67 \N str_67 [] +[68,68,68,68,68,68,68,68,68] \N \N [68,68,68,68,68,68,68,68,68] +69 69 \N [] +\N \N \N [] +str_71 \N str_71 [] +[NULL,NULL,NULL] \N \N [NULL,NULL,NULL] +73 73 \N [] +\N \N \N [] +str_75 \N str_75 [] +[76,76,76,76,76,76,76] \N \N [76,76,76,76,76,76,76] +77 77 \N [] +\N \N \N [] +str_79 \N str_79 [] +0 \N [] +1 \N [] +2 \N [] +3 \N [] +4 \N [] +5 \N [] +6 \N [] +7 \N [] +8 \N [] +9 \N [] +\N str_10 [] +10 \N [] +10 \N [] +\N \N [] +\N \N [] +\N str_11 [] +\N str_12 [] +12 \N [] +12 \N [] +\N str_13 [] +13 \N [] +13 \N [] +\N str_14 [] +14 \N [] +14 \N [] +\N str_15 [] +15 \N [] +15 \N [] +\N \N [] +\N \N [] +\N str_16 [] +\N str_17 [] +17 \N [] +17 \N [] +\N str_18 [] +18 \N [] +18 \N [] +\N str_19 [] +19 \N [] +19 \N [] +\N \N [20] +\N \N ['str_21','str_21'] +\N \N [22,22,22] +\N \N [23,23,23,23] +\N \N [24,24,24,24,24] +\N \N [25,25,25,25,25,25] +\N \N [26,26,26,26,26,26,26] +\N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [28,28,28,28,28,28,28,28,28] +\N \N [29,29,29,29,29,29,29,29,29,29] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [40] +41 \N [] +\N \N [] +\N str_43 [] +\N \N [44,44,44,44,44] +45 \N [] +\N \N [] +\N str_47 [] +\N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 \N [] +\N \N [] +\N str_51 [] +\N \N [52,52,52] +53 \N [] +\N \N [] +\N str_55 [] +\N \N [56,56,56,56,56,56,56] +57 \N [] +\N \N [] +\N str_59 [] +\N \N [60] +61 \N [] +\N \N [] +\N str_63 [] +\N \N [64,64,64,64,64] +65 \N [] +\N \N [] +\N str_67 [] +\N \N [68,68,68,68,68,68,68,68,68] +69 \N [] +\N \N [] +\N str_71 [] +\N \N [NULL,NULL,NULL] +73 \N [] +\N \N [] +\N str_75 [] +\N \N [76,76,76,76,76,76,76] +77 \N [] +\N \N [] +\N str_79 [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +0 0 \N [] 0 [] +1 1 \N [] 0 [] +2 2 \N [] 0 [] +3 3 \N [] 0 [] +4 4 \N [] 0 [] +5 5 \N [] 0 [] +6 6 \N [] 0 [] +7 7 \N [] 0 [] +8 8 \N [] 0 [] +9 9 \N [] 0 [] +str_10 \N \N [] 0 [] +10 10 \N [] 0 [] +10 10 \N [] 0 [] +[[0,1]] \N \N [] 0 [] +str_11 \N \N [] 0 [] +str_11 \N \N [] 0 [] +str_12 \N \N [] 0 [] +12 12 \N [] 0 [] +12 12 \N [] 0 [] +str_13 \N \N [] 0 [] +13 13 \N [] 0 [] +13 13 \N [] 0 [] +str_14 \N \N [] 0 [] +14 14 \N [] 0 [] +14 14 \N [] 0 [] +str_15 \N \N [] 0 [] +15 15 \N [] 0 [] +15 15 \N [] 0 [] +[[0,1,2,3,4,5,6]] \N \N [] 0 [] +str_16 \N \N [] 0 [] +str_16 \N \N [] 0 [] +str_17 \N \N [] 0 [] +17 17 \N [] 0 [] +17 17 \N [] 0 [] +str_18 \N \N [] 0 [] +18 18 \N [] 0 [] +18 18 \N [] 0 [] +str_19 \N \N [] 0 [] +19 19 \N [] 0 [] +19 19 \N [] 0 [] +[20] \N \N [20] 1 [20] +['str_21','str_21'] \N \N ['str_21','str_21'] 2 [NULL,NULL] +[22,22,22] \N \N [22,22,22] 3 [22,22,22] +[23,23,23,23] \N \N [23,23,23,23] 4 [23,23,23,23] +[24,24,24,24,24] \N \N [24,24,24,24,24] 5 [24,24,24,24,24] +[25,25,25,25,25,25] \N \N [25,25,25,25,25,25] 6 [25,25,25,25,25,25] +[26,26,26,26,26,26,26] \N \N [26,26,26,26,26,26,26] 7 [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] \N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 8 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] \N \N [28,28,28,28,28,28,28,28,28] 9 [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] \N \N [29,29,29,29,29,29,29,29,29,29] 10 [29,29,29,29,29,29,29,29,29,29] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +[40] \N \N [40] 1 [40] +41 41 \N [] 0 [] +\N \N \N [] 0 [] +str_43 \N \N [] 0 [] +[44,44,44,44,44] \N \N [44,44,44,44,44] 5 [44,44,44,44,44] +45 45 \N [] 0 [] +\N \N \N [] 0 [] +str_47 \N \N [] 0 [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] \N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 9 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +49 49 \N [] 0 [] +\N \N \N [] 0 [] +str_51 \N \N [] 0 [] +[52,52,52] \N \N [52,52,52] 3 [52,52,52] +53 53 \N [] 0 [] +\N \N \N [] 0 [] +str_55 \N \N [] 0 [] +[56,56,56,56,56,56,56] \N \N [56,56,56,56,56,56,56] 7 [56,56,56,56,56,56,56] +57 57 \N [] 0 [] +\N \N \N [] 0 [] +str_59 \N \N [] 0 [] +[60] \N \N [60] 1 [60] +61 61 \N [] 0 [] +\N \N \N [] 0 [] +str_63 \N \N [] 0 [] +[64,64,64,64,64] \N \N [64,64,64,64,64] 5 [64,64,64,64,64] +65 65 \N [] 0 [] +\N \N \N [] 0 [] +str_67 \N \N [] 0 [] +[68,68,68,68,68,68,68,68,68] \N \N [68,68,68,68,68,68,68,68,68] 9 [68,68,68,68,68,68,68,68,68] +69 69 \N [] 0 [] +\N \N \N [] 0 [] +str_71 \N \N [] 0 [] +[NULL,NULL,NULL] \N \N [NULL,NULL,NULL] 3 [NULL,NULL,NULL] +73 73 \N [] 0 [] +\N \N \N [] 0 [] +str_75 \N \N [] 0 [] +[76,76,76,76,76,76,76] \N \N [76,76,76,76,76,76,76] 7 [76,76,76,76,76,76,76] +77 77 \N [] 0 [] +\N \N \N [] 0 [] +str_79 \N \N [] 0 [] +0 \N [] 0 [] [] +1 \N [] 0 [] [] +2 \N [] 0 [] [] +3 \N [] 0 [] [] +4 \N [] 0 [] [] +5 \N [] 0 [] [] +6 \N [] 0 [] [] +7 \N [] 0 [] [] +8 \N [] 0 [] [] +9 \N [] 0 [] [] +\N \N [] 0 [] [] +10 \N [] 0 [] [] +10 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +12 \N [] 0 [] [] +12 \N [] 0 [] [] +\N \N [] 0 [] [] +13 \N [] 0 [] [] +13 \N [] 0 [] [] +\N \N [] 0 [] [] +14 \N [] 0 [] [] +14 \N [] 0 [] [] +\N \N [] 0 [] [] +15 \N [] 0 [] [] +15 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +17 \N [] 0 [] [] +17 \N [] 0 [] [] +\N \N [] 0 [] [] +18 \N [] 0 [] [] +18 \N [] 0 [] [] +\N \N [] 0 [] [] +19 \N [] 0 [] [] +19 \N [] 0 [] [] +\N \N [20] 1 [20] [NULL] +\N \N ['str_21','str_21'] 2 [NULL,NULL] ['str_21','str_21'] +\N \N [22,22,22] 3 [22,22,22] [NULL,NULL,NULL] +\N \N [23,23,23,23] 4 [23,23,23,23] [NULL,NULL,NULL,NULL] +\N \N [24,24,24,24,24] 5 [24,24,24,24,24] [NULL,NULL,NULL,NULL,NULL] +\N \N [25,25,25,25,25,25] 6 [25,25,25,25,25,25] [NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [26,26,26,26,26,26,26] 7 [26,26,26,26,26,26,26] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 8 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [28,28,28,28,28,28,28,28,28] 9 [28,28,28,28,28,28,28,28,28] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [29,29,29,29,29,29,29,29,29,29] 10 [29,29,29,29,29,29,29,29,29,29] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [40] 1 [40] [NULL] +41 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [44,44,44,44,44] 5 [44,44,44,44,44] [NULL,NULL,NULL,NULL,NULL] +45 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 9 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [52,52,52] 3 [52,52,52] [NULL,NULL,NULL] +53 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [56,56,56,56,56,56,56] 7 [56,56,56,56,56,56,56] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +57 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [60] 1 [60] [NULL] +61 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [64,64,64,64,64] 5 [64,64,64,64,64] [NULL,NULL,NULL,NULL,NULL] +65 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [68,68,68,68,68,68,68,68,68] 9 [68,68,68,68,68,68,68,68,68] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +69 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [NULL,NULL,NULL] 3 [NULL,NULL,NULL] [NULL,NULL,NULL] +73 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [76,76,76,76,76,76,76] 7 [76,76,76,76,76,76,76] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +77 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +0 0 [] [] +1 0 [] [] +2 0 [] [] +3 0 [] [] +4 0 [] [] +5 0 [] [] +6 0 [] [] +7 0 [] [] +8 0 [] [] +9 0 [] [] +str_10 0 [] [] +10 0 [] [] +10 0 [] [] +[[0,1]] 0 [] [] +str_11 0 [] [] +str_11 0 [] [] +str_12 0 [] [] +12 0 [] [] +12 0 [] [] +str_13 0 [] [] +13 0 [] [] +13 0 [] [] +str_14 0 [] [] +14 0 [] [] +14 0 [] [] +str_15 0 [] [] +15 0 [] [] +15 0 [] [] +[[0,1,2,3,4,5,6]] 0 [] [] +str_16 0 [] [] +str_16 0 [] [] +str_17 0 [] [] +17 0 [] [] +17 0 [] [] +str_18 0 [] [] +18 0 [] [] +18 0 [] [] +str_19 0 [] [] +19 0 [] [] +19 0 [] [] +[20] 0 [] [20] +['str_21','str_21'] 0 [] [NULL,NULL] +[22,22,22] 0 [] [22,22,22] +[23,23,23,23] 0 [] [23,23,23,23] +[24,24,24,24,24] 0 [] [24,24,24,24,24] +[25,25,25,25,25,25] 0 [] [25,25,25,25,25,25] +[26,26,26,26,26,26,26] 0 [] [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 0 [] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] 0 [] [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] 0 [] [29,29,29,29,29,29,29,29,29,29] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +[40] 0 [] [40] +41 0 [] [] +\N 0 [] [] +str_43 0 [] [] +[44,44,44,44,44] 0 [] [44,44,44,44,44] +45 0 [] [] +\N 0 [] [] +str_47 0 [] [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 0 [] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +49 0 [] [] +\N 0 [] [] +str_51 0 [] [] +[52,52,52] 0 [] [52,52,52] +53 0 [] [] +\N 0 [] [] +str_55 0 [] [] +[56,56,56,56,56,56,56] 0 [] [56,56,56,56,56,56,56] +57 0 [] [] +\N 0 [] [] +str_59 0 [] [] +[60] 0 [] [60] +61 0 [] [] +\N 0 [] [] +str_63 0 [] [] +[64,64,64,64,64] 0 [] [64,64,64,64,64] +65 0 [] [] +\N 0 [] [] +str_67 0 [] [] +[68,68,68,68,68,68,68,68,68] 0 [] [68,68,68,68,68,68,68,68,68] +69 0 [] [] +\N 0 [] [] +str_71 0 [] [] +[NULL,NULL,NULL] 0 [] [NULL,NULL,NULL] +73 0 [] [] +\N 0 [] [] +str_75 0 [] [] +[76,76,76,76,76,76,76] 0 [] [76,76,76,76,76,76,76] +77 0 [] [] +\N 0 [] [] +str_79 0 [] [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [20] +[] 0 [NULL,NULL] +[] 0 [22,22,22] +[] 0 [23,23,23,23] +[] 0 [24,24,24,24,24] +[] 0 [25,25,25,25,25,25] +[] 0 [26,26,26,26,26,26,26] +[] 0 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[] 0 [28,28,28,28,28,28,28,28,28] +[] 0 [29,29,29,29,29,29,29,29,29,29] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [40] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [44,44,44,44,44] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [52,52,52] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [56,56,56,56,56,56,56] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [60] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [64,64,64,64,64] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [68,68,68,68,68,68,68,68,68] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [NULL,NULL,NULL] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [76,76,76,76,76,76,76] +[] 0 [] +[] 0 [] +[] 0 [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[2] [[0,1]] [[[],[]]] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[7] [[0,1,2,3,4,5,6]] [[[],[],[],[],[],[],[]]] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +Array(Array(Dynamic)) +Array(Variant(String, UInt64)) +LowCardinality(String) +None +String +UInt64 +36 +36 +20 +20 +0 +0 +2 +2 +20 +20 +2 +2 +20 +0 +0 0 \N [] +1 1 \N [] +2 2 \N [] +3 3 \N [] +4 4 \N [] +5 5 \N [] +6 6 \N [] +7 7 \N [] +8 8 \N [] +9 9 \N [] +str_10 \N str_10 [] +10 10 \N [] +10 10 \N [] +[[0,1]] \N \N [] +str_11 \N \N [] +str_11 \N str_11 [] +str_12 \N str_12 [] +12 12 \N [] +12 12 \N [] +str_13 \N str_13 [] +13 13 \N [] +13 13 \N [] +str_14 \N str_14 [] +14 14 \N [] +14 14 \N [] +str_15 \N str_15 [] +15 15 \N [] +15 15 \N [] +[[0,1,2,3,4,5,6]] \N \N [] +str_16 \N \N [] +str_16 \N str_16 [] +str_17 \N str_17 [] +17 17 \N [] +17 17 \N [] +str_18 \N str_18 [] +18 18 \N [] +18 18 \N [] +str_19 \N str_19 [] +19 19 \N [] +19 19 \N [] +[20] \N \N [20] +['str_21','str_21'] \N \N ['str_21','str_21'] +[22,22,22] \N \N [22,22,22] +[23,23,23,23] \N \N [23,23,23,23] +[24,24,24,24,24] \N \N [24,24,24,24,24] +[25,25,25,25,25,25] \N \N [25,25,25,25,25,25] +[26,26,26,26,26,26,26] \N \N [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] \N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] \N \N [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] \N \N [29,29,29,29,29,29,29,29,29,29] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +[40] \N \N [40] +41 41 \N [] +\N \N \N [] +str_43 \N str_43 [] +[44,44,44,44,44] \N \N [44,44,44,44,44] +45 45 \N [] +\N \N \N [] +str_47 \N str_47 [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] \N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 49 \N [] +\N \N \N [] +str_51 \N str_51 [] +[52,52,52] \N \N [52,52,52] +53 53 \N [] +\N \N \N [] +str_55 \N str_55 [] +[56,56,56,56,56,56,56] \N \N [56,56,56,56,56,56,56] +57 57 \N [] +\N \N \N [] +str_59 \N str_59 [] +[60] \N \N [60] +61 61 \N [] +\N \N \N [] +str_63 \N str_63 [] +[64,64,64,64,64] \N \N [64,64,64,64,64] +65 65 \N [] +\N \N \N [] +str_67 \N str_67 [] +[68,68,68,68,68,68,68,68,68] \N \N [68,68,68,68,68,68,68,68,68] +69 69 \N [] +\N \N \N [] +str_71 \N str_71 [] +[NULL,NULL,NULL] \N \N [NULL,NULL,NULL] +73 73 \N [] +\N \N \N [] +str_75 \N str_75 [] +[76,76,76,76,76,76,76] \N \N [76,76,76,76,76,76,76] +77 77 \N [] +\N \N \N [] +str_79 \N str_79 [] +0 \N [] +1 \N [] +2 \N [] +3 \N [] +4 \N [] +5 \N [] +6 \N [] +7 \N [] +8 \N [] +9 \N [] +\N str_10 [] +10 \N [] +10 \N [] +\N \N [] +\N \N [] +\N str_11 [] +\N str_12 [] +12 \N [] +12 \N [] +\N str_13 [] +13 \N [] +13 \N [] +\N str_14 [] +14 \N [] +14 \N [] +\N str_15 [] +15 \N [] +15 \N [] +\N \N [] +\N \N [] +\N str_16 [] +\N str_17 [] +17 \N [] +17 \N [] +\N str_18 [] +18 \N [] +18 \N [] +\N str_19 [] +19 \N [] +19 \N [] +\N \N [20] +\N \N ['str_21','str_21'] +\N \N [22,22,22] +\N \N [23,23,23,23] +\N \N [24,24,24,24,24] +\N \N [25,25,25,25,25,25] +\N \N [26,26,26,26,26,26,26] +\N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [28,28,28,28,28,28,28,28,28] +\N \N [29,29,29,29,29,29,29,29,29,29] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [40] +41 \N [] +\N \N [] +\N str_43 [] +\N \N [44,44,44,44,44] +45 \N [] +\N \N [] +\N str_47 [] +\N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 \N [] +\N \N [] +\N str_51 [] +\N \N [52,52,52] +53 \N [] +\N \N [] +\N str_55 [] +\N \N [56,56,56,56,56,56,56] +57 \N [] +\N \N [] +\N str_59 [] +\N \N [60] +61 \N [] +\N \N [] +\N str_63 [] +\N \N [64,64,64,64,64] +65 \N [] +\N \N [] +\N str_67 [] +\N \N [68,68,68,68,68,68,68,68,68] +69 \N [] +\N \N [] +\N str_71 [] +\N \N [NULL,NULL,NULL] +73 \N [] +\N \N [] +\N str_75 [] +\N \N [76,76,76,76,76,76,76] +77 \N [] +\N \N [] +\N str_79 [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +0 0 \N [] 0 [] +1 1 \N [] 0 [] +2 2 \N [] 0 [] +3 3 \N [] 0 [] +4 4 \N [] 0 [] +5 5 \N [] 0 [] +6 6 \N [] 0 [] +7 7 \N [] 0 [] +8 8 \N [] 0 [] +9 9 \N [] 0 [] +str_10 \N \N [] 0 [] +10 10 \N [] 0 [] +10 10 \N [] 0 [] +[[0,1]] \N \N [] 0 [] +str_11 \N \N [] 0 [] +str_11 \N \N [] 0 [] +str_12 \N \N [] 0 [] +12 12 \N [] 0 [] +12 12 \N [] 0 [] +str_13 \N \N [] 0 [] +13 13 \N [] 0 [] +13 13 \N [] 0 [] +str_14 \N \N [] 0 [] +14 14 \N [] 0 [] +14 14 \N [] 0 [] +str_15 \N \N [] 0 [] +15 15 \N [] 0 [] +15 15 \N [] 0 [] +[[0,1,2,3,4,5,6]] \N \N [] 0 [] +str_16 \N \N [] 0 [] +str_16 \N \N [] 0 [] +str_17 \N \N [] 0 [] +17 17 \N [] 0 [] +17 17 \N [] 0 [] +str_18 \N \N [] 0 [] +18 18 \N [] 0 [] +18 18 \N [] 0 [] +str_19 \N \N [] 0 [] +19 19 \N [] 0 [] +19 19 \N [] 0 [] +[20] \N \N [20] 1 [20] +['str_21','str_21'] \N \N ['str_21','str_21'] 2 [NULL,NULL] +[22,22,22] \N \N [22,22,22] 3 [22,22,22] +[23,23,23,23] \N \N [23,23,23,23] 4 [23,23,23,23] +[24,24,24,24,24] \N \N [24,24,24,24,24] 5 [24,24,24,24,24] +[25,25,25,25,25,25] \N \N [25,25,25,25,25,25] 6 [25,25,25,25,25,25] +[26,26,26,26,26,26,26] \N \N [26,26,26,26,26,26,26] 7 [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] \N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 8 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] \N \N [28,28,28,28,28,28,28,28,28] 9 [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] \N \N [29,29,29,29,29,29,29,29,29,29] 10 [29,29,29,29,29,29,29,29,29,29] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +[40] \N \N [40] 1 [40] +41 41 \N [] 0 [] +\N \N \N [] 0 [] +str_43 \N \N [] 0 [] +[44,44,44,44,44] \N \N [44,44,44,44,44] 5 [44,44,44,44,44] +45 45 \N [] 0 [] +\N \N \N [] 0 [] +str_47 \N \N [] 0 [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] \N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 9 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +49 49 \N [] 0 [] +\N \N \N [] 0 [] +str_51 \N \N [] 0 [] +[52,52,52] \N \N [52,52,52] 3 [52,52,52] +53 53 \N [] 0 [] +\N \N \N [] 0 [] +str_55 \N \N [] 0 [] +[56,56,56,56,56,56,56] \N \N [56,56,56,56,56,56,56] 7 [56,56,56,56,56,56,56] +57 57 \N [] 0 [] +\N \N \N [] 0 [] +str_59 \N \N [] 0 [] +[60] \N \N [60] 1 [60] +61 61 \N [] 0 [] +\N \N \N [] 0 [] +str_63 \N \N [] 0 [] +[64,64,64,64,64] \N \N [64,64,64,64,64] 5 [64,64,64,64,64] +65 65 \N [] 0 [] +\N \N \N [] 0 [] +str_67 \N \N [] 0 [] +[68,68,68,68,68,68,68,68,68] \N \N [68,68,68,68,68,68,68,68,68] 9 [68,68,68,68,68,68,68,68,68] +69 69 \N [] 0 [] +\N \N \N [] 0 [] +str_71 \N \N [] 0 [] +[NULL,NULL,NULL] \N \N [NULL,NULL,NULL] 3 [NULL,NULL,NULL] +73 73 \N [] 0 [] +\N \N \N [] 0 [] +str_75 \N \N [] 0 [] +[76,76,76,76,76,76,76] \N \N [76,76,76,76,76,76,76] 7 [76,76,76,76,76,76,76] +77 77 \N [] 0 [] +\N \N \N [] 0 [] +str_79 \N \N [] 0 [] +0 \N [] 0 [] [] +1 \N [] 0 [] [] +2 \N [] 0 [] [] +3 \N [] 0 [] [] +4 \N [] 0 [] [] +5 \N [] 0 [] [] +6 \N [] 0 [] [] +7 \N [] 0 [] [] +8 \N [] 0 [] [] +9 \N [] 0 [] [] +\N \N [] 0 [] [] +10 \N [] 0 [] [] +10 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +12 \N [] 0 [] [] +12 \N [] 0 [] [] +\N \N [] 0 [] [] +13 \N [] 0 [] [] +13 \N [] 0 [] [] +\N \N [] 0 [] [] +14 \N [] 0 [] [] +14 \N [] 0 [] [] +\N \N [] 0 [] [] +15 \N [] 0 [] [] +15 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +17 \N [] 0 [] [] +17 \N [] 0 [] [] +\N \N [] 0 [] [] +18 \N [] 0 [] [] +18 \N [] 0 [] [] +\N \N [] 0 [] [] +19 \N [] 0 [] [] +19 \N [] 0 [] [] +\N \N [20] 1 [20] [NULL] +\N \N ['str_21','str_21'] 2 [NULL,NULL] ['str_21','str_21'] +\N \N [22,22,22] 3 [22,22,22] [NULL,NULL,NULL] +\N \N [23,23,23,23] 4 [23,23,23,23] [NULL,NULL,NULL,NULL] +\N \N [24,24,24,24,24] 5 [24,24,24,24,24] [NULL,NULL,NULL,NULL,NULL] +\N \N [25,25,25,25,25,25] 6 [25,25,25,25,25,25] [NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [26,26,26,26,26,26,26] 7 [26,26,26,26,26,26,26] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 8 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [28,28,28,28,28,28,28,28,28] 9 [28,28,28,28,28,28,28,28,28] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [29,29,29,29,29,29,29,29,29,29] 10 [29,29,29,29,29,29,29,29,29,29] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [40] 1 [40] [NULL] +41 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [44,44,44,44,44] 5 [44,44,44,44,44] [NULL,NULL,NULL,NULL,NULL] +45 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 9 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [52,52,52] 3 [52,52,52] [NULL,NULL,NULL] +53 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [56,56,56,56,56,56,56] 7 [56,56,56,56,56,56,56] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +57 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [60] 1 [60] [NULL] +61 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [64,64,64,64,64] 5 [64,64,64,64,64] [NULL,NULL,NULL,NULL,NULL] +65 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [68,68,68,68,68,68,68,68,68] 9 [68,68,68,68,68,68,68,68,68] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +69 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [NULL,NULL,NULL] 3 [NULL,NULL,NULL] [NULL,NULL,NULL] +73 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [76,76,76,76,76,76,76] 7 [76,76,76,76,76,76,76] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +77 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +0 0 [] [] +1 0 [] [] +2 0 [] [] +3 0 [] [] +4 0 [] [] +5 0 [] [] +6 0 [] [] +7 0 [] [] +8 0 [] [] +9 0 [] [] +str_10 0 [] [] +10 0 [] [] +10 0 [] [] +[[0,1]] 0 [] [] +str_11 0 [] [] +str_11 0 [] [] +str_12 0 [] [] +12 0 [] [] +12 0 [] [] +str_13 0 [] [] +13 0 [] [] +13 0 [] [] +str_14 0 [] [] +14 0 [] [] +14 0 [] [] +str_15 0 [] [] +15 0 [] [] +15 0 [] [] +[[0,1,2,3,4,5,6]] 0 [] [] +str_16 0 [] [] +str_16 0 [] [] +str_17 0 [] [] +17 0 [] [] +17 0 [] [] +str_18 0 [] [] +18 0 [] [] +18 0 [] [] +str_19 0 [] [] +19 0 [] [] +19 0 [] [] +[20] 0 [] [20] +['str_21','str_21'] 0 [] [NULL,NULL] +[22,22,22] 0 [] [22,22,22] +[23,23,23,23] 0 [] [23,23,23,23] +[24,24,24,24,24] 0 [] [24,24,24,24,24] +[25,25,25,25,25,25] 0 [] [25,25,25,25,25,25] +[26,26,26,26,26,26,26] 0 [] [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 0 [] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] 0 [] [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] 0 [] [29,29,29,29,29,29,29,29,29,29] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +[40] 0 [] [40] +41 0 [] [] +\N 0 [] [] +str_43 0 [] [] +[44,44,44,44,44] 0 [] [44,44,44,44,44] +45 0 [] [] +\N 0 [] [] +str_47 0 [] [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 0 [] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +49 0 [] [] +\N 0 [] [] +str_51 0 [] [] +[52,52,52] 0 [] [52,52,52] +53 0 [] [] +\N 0 [] [] +str_55 0 [] [] +[56,56,56,56,56,56,56] 0 [] [56,56,56,56,56,56,56] +57 0 [] [] +\N 0 [] [] +str_59 0 [] [] +[60] 0 [] [60] +61 0 [] [] +\N 0 [] [] +str_63 0 [] [] +[64,64,64,64,64] 0 [] [64,64,64,64,64] +65 0 [] [] +\N 0 [] [] +str_67 0 [] [] +[68,68,68,68,68,68,68,68,68] 0 [] [68,68,68,68,68,68,68,68,68] +69 0 [] [] +\N 0 [] [] +str_71 0 [] [] +[NULL,NULL,NULL] 0 [] [NULL,NULL,NULL] +73 0 [] [] +\N 0 [] [] +str_75 0 [] [] +[76,76,76,76,76,76,76] 0 [] [76,76,76,76,76,76,76] +77 0 [] [] +\N 0 [] [] +str_79 0 [] [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [20] +[] 0 [NULL,NULL] +[] 0 [22,22,22] +[] 0 [23,23,23,23] +[] 0 [24,24,24,24,24] +[] 0 [25,25,25,25,25,25] +[] 0 [26,26,26,26,26,26,26] +[] 0 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[] 0 [28,28,28,28,28,28,28,28,28] +[] 0 [29,29,29,29,29,29,29,29,29,29] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [40] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [44,44,44,44,44] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [52,52,52] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [56,56,56,56,56,56,56] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [60] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [64,64,64,64,64] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [68,68,68,68,68,68,68,68,68] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [NULL,NULL,NULL] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [76,76,76,76,76,76,76] +[] 0 [] +[] 0 [] +[] 0 [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[2] [[0,1]] [[[],[]]] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[7] [[0,1,2,3,4,5,6]] [[[],[],[],[],[],[],[]]] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 new file mode 100644 index 00000000000..0c123d5f6fe --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 @@ -0,0 +1,46 @@ +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; + +drop table if exists test; + +{% for engine in ['Memory', 'MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000', 'MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1'] -%} + +create table test (id UInt64, d Dynamic(max_types=2)) engine={{ engine }}; + +insert into test select number, number from numbers(10); +insert into test select number, 'str_' || toString(number) from numbers(10, 10); +insert into test select number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1)) from numbers(20, 10); +insert into test select number, NULL from numbers(30, 10); +insert into test select number, multiIf(number % 4 == 3, 'str_' || toString(number), number % 4 == 2, NULL, number % 4 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1))) from numbers(40, 40); +insert into test select number, if(number % 5 == 1, [range((number % 10 + 1)::UInt64)]::Array(Array(Dynamic)), number) from numbers(10, 10); +insert into test select number, if(number % 5 == 1, ('str_' || number)::LowCardinality(String)::Dynamic, number::Dynamic) from numbers(10, 10); + +select distinct dynamicType(d) as type from test order by type; +select count() from test where dynamicType(d) == 'UInt64'; +select count() from test where d.UInt64 is not NULL; +select count() from test where dynamicType(d) == 'String'; +select count() from test where d.String is not NULL; +select count() from test where dynamicType(d) == 'Date'; +select count() from test where d.Date is not NULL; +select count() from test where dynamicType(d) == 'LowCardinality(String)'; +select count() from test where d.`LowCardinality(String)` is not NULL; +select count() from test where dynamicType(d) == 'Array(Variant(String, UInt64))'; +select count() from test where not empty(d.`Array(Variant(String, UInt64))`); +select count() from test where dynamicType(d) == 'Array(Array(Dynamic))'; +select count() from test where not empty(d.`Array(Array(Dynamic))`); +select count() from test where d is NULL; +select count() from test where not empty(d.`Tuple(a Array(Dynamic))`.a.String); + +select d, d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test order by id, d; +select d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test order by id, d; +select d.Int8, d.Date, d.`Array(String)` from test order by id, d; +select d, d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id, d; +select d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64, d.`Array(Variant(String, UInt64))`.String from test order by id, d; +select d, d.`Tuple(a UInt64, b String)`.a, d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id, d; +select d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Dynamic)`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id, d; +select d.`Array(Array(Dynamic))`.size1, d.`Array(Array(Dynamic))`.UInt64, d.`Array(Array(Dynamic))`.`Map(String, Tuple(a UInt64))`.values.a from test order by id, d; + +drop table test; + +{% endfor -%} diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.reference b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.reference new file mode 100644 index 00000000000..ca6c5dbba82 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.reference @@ -0,0 +1,20 @@ +Array(Array(Dynamic)) +Array(Variant(String, UInt64)) +LowCardinality(String) +None +String +UInt64 +360000 +360000 +200000 +200000 +0 +0 +20000 +20000 +200000 +200000 +20000 +20000 +200000 +0 diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.sql b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.sql new file mode 100644 index 00000000000..61dc8fca01a --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.sql @@ -0,0 +1,43 @@ +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan + +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; + +drop table if exists test; +create table test (id UInt64, d Dynamic(max_types=2)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; + +insert into test select number, number from numbers(100000) settings min_insert_block_size_rows=50000; +insert into test select number, 'str_' || toString(number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1)) from numbers(200000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, NULL from numbers(300000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, multiIf(number % 4 == 3, 'str_' || toString(number), number % 4 == 2, NULL, number % 4 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1))) from numbers(400000, 400000) settings min_insert_block_size_rows=50000; +insert into test select number, if (number % 5 == 1, [range((number % 10 + 1)::UInt64)]::Array(Array(Dynamic)), number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, if (number % 5 == 1, ('str_' || number)::LowCardinality(String)::Dynamic, number::Dynamic) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; + +select distinct dynamicType(d) as type from test order by type; +select count() from test where dynamicType(d) == 'UInt64'; +select count() from test where d.UInt64 is not NULL; +select count() from test where dynamicType(d) == 'String'; +select count() from test where d.String is not NULL; +select count() from test where dynamicType(d) == 'Date'; +select count() from test where d.Date is not NULL; +select count() from test where dynamicType(d) == 'LowCardinality(String)'; +select count() from test where d.`LowCardinality(String)` is not NULL; +select count() from test where dynamicType(d) == 'Array(Variant(String, UInt64))'; +select count() from test where not empty(d.`Array(Variant(String, UInt64))`); +select count() from test where dynamicType(d) == 'Array(Array(Dynamic))'; +select count() from test where not empty(d.`Array(Array(Dynamic))`); +select count() from test where d is NULL; +select count() from test where not empty(d.`Tuple(a Array(Dynamic))`.a.String); + +select d, d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.Int8, d.Date, d.`Array(String)` from test format Null; +select d, d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64, d.`Array(Variant(String, UInt64))`.String from test format Null; +select d, d.`Tuple(a UInt64, b String)`.a, d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Dynamic)`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.`Array(Array(Dynamic))`.size1, d.`Array(Array(Dynamic))`.UInt64, d.`Array(Array(Dynamic))`.`Map(String, Tuple(a UInt64))`.values.a from test format Null; + +drop table test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.reference index d0d777a5a38..b0be05f07a2 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.reference @@ -1,28 +1,66 @@ -50000 DateTime -60000 Date -70000 Array(UInt16) -80000 String -100000 None -100000 UInt64 -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -200000 Map(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -10000 Tuple(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -270000 String +50000 DateTime false +60000 Date false +70000 Array(UInt16) false +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +30000 String false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +100000 None false +100000 UInt64 true +110000 String false +200000 Map(UInt64, UInt64) false diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql index d2c787040e5..fb23e15738e 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql @@ -2,7 +2,7 @@ set allow_experimental_dynamic_type=1; drop table if exists test; -create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); @@ -12,22 +12,37 @@ insert into test select number, toDate(number) from numbers(60000); insert into test select number, toDateTime(number) from numbers(50000); insert into test select number, NULL from numbers(100000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final;; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, map(number, number) from numbers(200000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, tuple(number, number) from numbers(10000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, 'str_' || number from numbers(30000); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); + drop table test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.reference index d0d777a5a38..b0be05f07a2 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.reference @@ -1,28 +1,66 @@ -50000 DateTime -60000 Date -70000 Array(UInt16) -80000 String -100000 None -100000 UInt64 -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -200000 Map(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -10000 Tuple(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -270000 String +50000 DateTime false +60000 Date false +70000 Array(UInt16) false +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +30000 String false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +100000 None false +100000 UInt64 true +110000 String false +200000 Map(UInt64, UInt64) false diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql index f99bf771608..c098a3191e0 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql @@ -2,7 +2,7 @@ set allow_experimental_dynamic_type=1; drop table if exists test; -create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); @@ -12,22 +12,36 @@ insert into test select number, toDate(number) from numbers(60000); insert into test select number, toDateTime(number) from numbers(50000); insert into test select number, NULL from numbers(100000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final;; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, map(number, number) from numbers(200000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, tuple(number, number) from numbers(10000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, 'str_' || number from numbers(30000); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); drop table test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.reference index d0d777a5a38..b0be05f07a2 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.reference @@ -1,28 +1,66 @@ -50000 DateTime -60000 Date -70000 Array(UInt16) -80000 String -100000 None -100000 UInt64 -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -200000 Map(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -10000 Tuple(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -270000 String +50000 DateTime false +60000 Date false +70000 Array(UInt16) false +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +30000 String false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +100000 None false +100000 UInt64 true +110000 String false +200000 Map(UInt64, UInt64) false diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql index be81596d043..17b1e451143 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql @@ -2,7 +2,7 @@ set allow_experimental_dynamic_type=1; drop table if exists test; -create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); @@ -12,23 +12,36 @@ insert into test select number, toDate(number) from numbers(60000); insert into test select number, toDateTime(number) from numbers(50000); insert into test select number, NULL from numbers(100000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); -system start merges test; -optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); +system start merges test; optimize table test final;; +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, map(number, number) from numbers(200000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, tuple(number, number) from numbers(10000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, 'str_' || number from numbers(30000); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); drop table test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.reference index d0d777a5a38..b0be05f07a2 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.reference @@ -1,28 +1,66 @@ -50000 DateTime -60000 Date -70000 Array(UInt16) -80000 String -100000 None -100000 UInt64 -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -200000 Map(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -10000 Tuple(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -270000 String +50000 DateTime false +60000 Date false +70000 Array(UInt16) false +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +30000 String false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +100000 None false +100000 UInt64 true +110000 String false +200000 Map(UInt64, UInt64) false diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql index f6396af42a8..fd6c0109263 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql @@ -2,7 +2,7 @@ set allow_experimental_dynamic_type=1; drop table if exists test; -create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); @@ -12,22 +12,36 @@ insert into test select number, toDate(number) from numbers(60000); insert into test select number, toDateTime(number) from numbers(50000); insert into test select number, NULL from numbers(100000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final;; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, map(number, number) from numbers(200000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, tuple(number, number) from numbers(10000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, 'str_' || number from numbers(30000); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); drop table test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_small.reference.j2 b/tests/queries/0_stateless/03037_dynamic_merges_small.reference.j2 index 96a854630ed..7d3bc371e36 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_small.reference.j2 +++ b/tests/queries/0_stateless/03037_dynamic_merges_small.reference.j2 @@ -1,112 +1,264 @@ -5 DateTime -6 Date -7 Array(UInt16) -8 String -10 None -10 UInt64 -7 Array(UInt16) -10 None -10 UInt64 -19 String -7 Array(UInt16) -10 None -10 UInt64 -19 String -20 Map(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -1 Tuple(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -10 None -10 UInt64 -20 Map(UInt64, UInt64) -27 String -5 DateTime -6 Date -7 Array(UInt16) -8 String -10 None -10 UInt64 -7 Array(UInt16) -10 None -10 UInt64 -19 String -7 Array(UInt16) -10 None -10 UInt64 -19 String -20 Map(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -1 Tuple(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -10 None -10 UInt64 -20 Map(UInt64, UInt64) -27 String -5 DateTime -6 Date -7 Array(UInt16) -8 String -10 None -10 UInt64 -7 Array(UInt16) -10 None -10 UInt64 -19 String -7 Array(UInt16) -10 None -10 UInt64 -19 String -20 Map(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -1 Tuple(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -10 None -10 UInt64 -20 Map(UInt64, UInt64) -27 String -5 DateTime -6 Date -7 Array(UInt16) -8 String -10 None -10 UInt64 -7 Array(UInt16) -10 None -10 UInt64 -19 String -7 Array(UInt16) -10 None -10 UInt64 -19 String -20 Map(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -1 Tuple(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -10 None -10 UInt64 -20 Map(UInt64, UInt64) -27 String +5 DateTime false +6 Date false +7 Array(UInt16) false +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +3 String false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +10 None false +10 UInt64 true +11 String false +20 Map(UInt64, UInt64) false +5 DateTime false +6 Date false +7 Array(UInt16) false +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +3 String false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +10 None false +10 UInt64 true +11 String false +20 Map(UInt64, UInt64) false +5 DateTime false +6 Date false +7 Array(UInt16) false +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +3 String false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +10 None false +10 UInt64 true +11 String false +20 Map(UInt64, UInt64) false +5 DateTime false +6 Date false +7 Array(UInt16) false +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +3 String false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +10 None false +10 UInt64 true +11 String false +20 Map(UInt64, UInt64) false diff --git a/tests/queries/0_stateless/03037_dynamic_merges_small.sql.j2 b/tests/queries/0_stateless/03037_dynamic_merges_small.sql.j2 index 263e92be403..3778399d0a4 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_small.sql.j2 +++ b/tests/queries/0_stateless/03037_dynamic_merges_small.sql.j2 @@ -9,7 +9,7 @@ drop table if exists test; 'MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1', 'MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1'] -%} -create table test (id UInt64, d Dynamic(max_types=3)) engine={{ engine }}; +create table test (id UInt64, d Dynamic(max_types=2)) engine={{ engine }}; system stop merges test; insert into test select number, number from numbers(10); @@ -19,23 +19,37 @@ insert into test select number, toDate(number) from numbers(6); insert into test select number, toDateTime(number) from numbers(5); insert into test select number, NULL from numbers(10); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final;; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, map(number, number) from numbers(20); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, tuple(number, number) from numbers(1); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, 'str_' || number from numbers(3); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); drop table test; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference index 4be740f6050..a4c2df74a74 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference @@ -1,21 +1,63 @@ -16667 Tuple(a Dynamic(max_types=3)):Date -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):String -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 UInt64:None -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 UInt64:None -16667 Tuple(a Dynamic(max_types=3)):DateTime -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -133333 Tuple(a Dynamic(max_types=3)):None -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -116667 Tuple(a Dynamic(max_types=3)):String -133333 Tuple(a Dynamic(max_types=3)):None +6667 Tuple(a Dynamic(max_types=2)):DateTime false +13333 Tuple(a Dynamic(max_types=2)):IPv4 false +16667 Tuple(a Dynamic(max_types=2)):Date false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +16667 Tuple(a Dynamic(max_types=2)):DateTime false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +40000 Tuple(a Dynamic(max_types=2)):DateTime false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +63334 Tuple(a Dynamic(max_types=2)):DateTime true +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql index 81888946681..8ba192cb5db 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql @@ -6,24 +6,39 @@ set allow_experimental_dynamic_type = 1; set enable_named_columns_in_function_tuple = 0; drop table if exists test;; -create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); -insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000); -insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000); +insert into test select number, tuple(if(number % 3 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=2)) from numbers(100000); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, multiIf(number % 5 == 0, tuple(if(number % 3 == 0, toDateTime(number), toIPv4(number)))::Tuple(a Dynamic(max_types=2)), number % 5 == 1 or number % 5 == 2, number, 'str_' || number) from numbers(100000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; -insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000); -insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000); +system stop merges test; +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(200000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; + +system stop merges test; +insert into test select number, tuple(toDateTime(number))::Tuple(a Dynamic(max_types=2)) from numbers(40000); + +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; drop table test; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference index 4be740f6050..a4c2df74a74 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference @@ -1,21 +1,63 @@ -16667 Tuple(a Dynamic(max_types=3)):Date -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):String -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 UInt64:None -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 UInt64:None -16667 Tuple(a Dynamic(max_types=3)):DateTime -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -133333 Tuple(a Dynamic(max_types=3)):None -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -116667 Tuple(a Dynamic(max_types=3)):String -133333 Tuple(a Dynamic(max_types=3)):None +6667 Tuple(a Dynamic(max_types=2)):DateTime false +13333 Tuple(a Dynamic(max_types=2)):IPv4 false +16667 Tuple(a Dynamic(max_types=2)):Date false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +16667 Tuple(a Dynamic(max_types=2)):DateTime false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +40000 Tuple(a Dynamic(max_types=2)):DateTime false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +63334 Tuple(a Dynamic(max_types=2)):DateTime true +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql index ba58ca471a2..1ea7eefdd53 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql @@ -6,24 +6,39 @@ set allow_experimental_dynamic_type = 1; set enable_named_columns_in_function_tuple = 0; drop table if exists test;; -create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); -insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000); -insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000); +insert into test select number, tuple(if(number % 3 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=2)) from numbers(100000); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, multiIf(number % 5 == 0, tuple(if(number % 3 == 0, toDateTime(number), toIPv4(number)))::Tuple(a Dynamic(max_types=2)), number % 5 == 1 or number % 5 == 2, number, 'str_' || number) from numbers(100000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; -insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000); -insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000); +system stop merges test; +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(200000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; + +system stop merges test; +insert into test select number, tuple(toDateTime(number))::Tuple(a Dynamic(max_types=2)) from numbers(40000); + +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; drop table test; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 index ae07c164074..3d7e8b60f73 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 @@ -1,84 +1,256 @@ -2 Tuple(a Dynamic(max_types=3)):Date -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):String -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 UInt64:None -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 UInt64:None -2 Tuple(a Dynamic(max_types=3)):DateTime -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -13 Tuple(a Dynamic(max_types=3)):None -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -12 Tuple(a Dynamic(max_types=3)):String -13 Tuple(a Dynamic(max_types=3)):None -2 Tuple(a Dynamic(max_types=3)):Date -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):String -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 UInt64:None -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 UInt64:None -2 Tuple(a Dynamic(max_types=3)):DateTime -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -13 Tuple(a Dynamic(max_types=3)):None -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -12 Tuple(a Dynamic(max_types=3)):String -13 Tuple(a Dynamic(max_types=3)):None -2 Tuple(a Dynamic(max_types=3)):Date -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):String -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 UInt64:None -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 UInt64:None -2 Tuple(a Dynamic(max_types=3)):DateTime -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -13 Tuple(a Dynamic(max_types=3)):None -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -12 Tuple(a Dynamic(max_types=3)):String -13 Tuple(a Dynamic(max_types=3)):None -2 Tuple(a Dynamic(max_types=3)):Date -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):String -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 UInt64:None -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 UInt64:None -2 Tuple(a Dynamic(max_types=3)):DateTime -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -13 Tuple(a Dynamic(max_types=3)):None -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -12 Tuple(a Dynamic(max_types=3)):String -13 Tuple(a Dynamic(max_types=3)):None +test +2 Tuple(a Dynamic(max_types=2)):DateTime false +2 Tuple(a Dynamic(max_types=2)):IPv4 false +4 Tuple(a Dynamic(max_types=2)):Date false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +4 Tuple(a Dynamic(max_types=2)):DateTime false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +8 Tuple(a Dynamic(max_types=2)):DateTime false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +14 Tuple(a Dynamic(max_types=2)):DateTime true +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +test +2 Tuple(a Dynamic(max_types=2)):DateTime false +2 Tuple(a Dynamic(max_types=2)):IPv4 false +4 Tuple(a Dynamic(max_types=2)):Date false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +4 Tuple(a Dynamic(max_types=2)):DateTime false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +8 Tuple(a Dynamic(max_types=2)):DateTime false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +14 Tuple(a Dynamic(max_types=2)):DateTime true +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +test +2 Tuple(a Dynamic(max_types=2)):DateTime false +2 Tuple(a Dynamic(max_types=2)):IPv4 false +4 Tuple(a Dynamic(max_types=2)):Date false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +4 Tuple(a Dynamic(max_types=2)):DateTime false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +8 Tuple(a Dynamic(max_types=2)):DateTime false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String true +14 Tuple(a Dynamic(max_types=2)):DateTime false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +test +2 Tuple(a Dynamic(max_types=2)):DateTime false +2 Tuple(a Dynamic(max_types=2)):IPv4 false +4 Tuple(a Dynamic(max_types=2)):Date false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +4 Tuple(a Dynamic(max_types=2)):DateTime false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +8 Tuple(a Dynamic(max_types=2)):DateTime false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String true +14 Tuple(a Dynamic(max_types=2)):DateTime false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 index 7828c2af49c..8682b6cef81 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 @@ -10,25 +10,41 @@ drop table if exists test; 'MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1', 'MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1'] -%} -create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600; +select 'test'; +create table test (id UInt64, d Dynamic(max_types=2)) engine={{ engine }}; system stop merges test; -insert into test select number, number from numbers(10); -insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(10); -insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(5); +insert into test select number, number from numbers(20); +insert into test select number, tuple(if(number % 3 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=2)) from numbers(20); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=2)) from numbers(10); +insert into test select number, multiIf(number % 5 == 0, tuple(if(number % 3 == 0, toDateTime(number), toIPv4(number)))::Tuple(a Dynamic(max_types=2)), number % 5 == 1 or number % 5 == 2, number, 'str_' || number) from numbers(20); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; -insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(5); -insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(20); +system stop merges test; +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(10); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(40); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; + +system stop merges test; +insert into test select number, tuple(toDateTime(number))::Tuple(a Dynamic(max_types=2)) from numbers(8); + +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; drop table test; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference index 4be740f6050..a4c2df74a74 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference @@ -1,21 +1,63 @@ -16667 Tuple(a Dynamic(max_types=3)):Date -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):String -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 UInt64:None -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 UInt64:None -16667 Tuple(a Dynamic(max_types=3)):DateTime -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -133333 Tuple(a Dynamic(max_types=3)):None -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -116667 Tuple(a Dynamic(max_types=3)):String -133333 Tuple(a Dynamic(max_types=3)):None +6667 Tuple(a Dynamic(max_types=2)):DateTime false +13333 Tuple(a Dynamic(max_types=2)):IPv4 false +16667 Tuple(a Dynamic(max_types=2)):Date false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +16667 Tuple(a Dynamic(max_types=2)):DateTime false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +40000 Tuple(a Dynamic(max_types=2)):DateTime false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +63334 Tuple(a Dynamic(max_types=2)):DateTime true +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql index a53c5b0b2a5..c6a09036c30 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql @@ -6,24 +6,39 @@ set allow_experimental_dynamic_type = 1; set enable_named_columns_in_function_tuple = 0; drop table if exists test;; -create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); -insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000); -insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000); +insert into test select number, tuple(if(number % 3 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=2)) from numbers(100000); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, multiIf(number % 5 == 0, tuple(if(number % 3 == 0, toDateTime(number), toIPv4(number)))::Tuple(a Dynamic(max_types=2)), number % 5 == 1 or number % 5 == 2, number, 'str_' || number) from numbers(100000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; -insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000); -insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000); +system stop merges test; +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(200000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; + +system stop merges test; +insert into test select number, tuple(toDateTime(number))::Tuple(a Dynamic(max_types=2)) from numbers(40000); + +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; drop table test; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference index 4be740f6050..a4c2df74a74 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference @@ -1,21 +1,63 @@ -16667 Tuple(a Dynamic(max_types=3)):Date -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):String -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 UInt64:None -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 UInt64:None -16667 Tuple(a Dynamic(max_types=3)):DateTime -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -133333 Tuple(a Dynamic(max_types=3)):None -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -116667 Tuple(a Dynamic(max_types=3)):String -133333 Tuple(a Dynamic(max_types=3)):None +6667 Tuple(a Dynamic(max_types=2)):DateTime false +13333 Tuple(a Dynamic(max_types=2)):IPv4 false +16667 Tuple(a Dynamic(max_types=2)):Date false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +16667 Tuple(a Dynamic(max_types=2)):DateTime false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +40000 Tuple(a Dynamic(max_types=2)):DateTime false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +63334 Tuple(a Dynamic(max_types=2)):DateTime true +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql index 4256b010ec0..c1964c45d98 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql @@ -6,24 +6,39 @@ set allow_experimental_dynamic_type = 1; set enable_named_columns_in_function_tuple = 0; drop table if exists test;; -create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); -insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000); -insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000); +insert into test select number, tuple(if(number % 3 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=2)) from numbers(100000); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, multiIf(number % 5 == 0, tuple(if(number % 3 == 0, toDateTime(number), toIPv4(number)))::Tuple(a Dynamic(max_types=2)), number % 5 == 1 or number % 5 == 2, number, 'str_' || number) from numbers(100000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; -insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000); -insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000); +system stop merges test; +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(200000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; + +system stop merges test; +insert into test select number, tuple(toDateTime(number))::Tuple(a Dynamic(max_types=2)) from numbers(40000); + +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; drop table test; diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.reference index 2ec301b747b..9386548c74d 100644 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.reference +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.reference @@ -1,13 +1,13 @@ initial insert alter add column 1 -3 None +3 None false 0 0 \N \N \N 0 1 1 \N \N \N 0 2 2 \N \N \N 0 insert after alter add column 1 -4 String -4 UInt64 -7 None +4 String false +4 UInt64 false +7 None false 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 @@ -24,147 +24,158 @@ insert after alter add column 1 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 alter modify column 1 -7 None -8 String +4 String true +4 UInt64 true +7 None false 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 6 6 str_6 str_6 \N \N 0 7 7 str_7 str_7 \N \N 0 8 8 str_8 str_8 \N \N 0 9 9 \N \N \N \N 0 10 10 \N \N \N \N 0 11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 +12 12 12 \N 12 \N 0 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 insert after alter modify column 1 -8 None -11 String +1 Date true +5 String true +5 UInt64 true +8 None false 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 6 6 str_6 str_6 \N \N 0 7 7 str_7 str_7 \N \N 0 8 8 str_8 str_8 \N \N 0 9 9 \N \N \N \N 0 10 10 \N \N \N \N 0 11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 +12 12 12 \N 12 \N 0 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 +16 16 16 \N 16 \N 0 17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 alter modify column 2 -8 None -11 String +1 Date true +5 String true +5 UInt64 true +8 None false 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 6 6 str_6 str_6 \N \N 0 7 7 str_7 str_7 \N \N 0 8 8 str_8 str_8 \N \N 0 9 9 \N \N \N \N 0 10 10 \N \N \N \N 0 11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 +12 12 12 \N 12 \N 0 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 +16 16 16 \N 16 \N 0 17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 insert after alter modify column 2 -1 Date -1 UInt64 -9 None -12 String +1 String false +1 UInt64 false +2 Date true +5 String true +5 UInt64 true +9 None false 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 6 6 str_6 str_6 \N \N 0 7 7 str_7 str_7 \N \N 0 8 8 str_8 str_8 \N \N 0 9 9 \N \N \N \N 0 10 10 \N \N \N \N 0 11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 +12 12 12 \N 12 \N 0 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 +16 16 16 \N 16 \N 0 17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 19 19 \N \N \N \N 0 20 20 20 \N 20 \N 0 21 21 str_21 str_21 \N \N 0 22 22 1970-01-23 \N \N 1970-01-23 0 alter modify column 3 -1 Date -1 UInt64 -9 None -12 String +1 String false +1 UInt64 false +2 Date true +5 String true +5 UInt64 true +9 None false 0 0 0 \N 0 \N \N \N 0 1 1 1 \N 0 \N \N \N 0 2 2 2 \N 0 \N \N \N 0 -3 3 3 \N 0 3 \N \N 0 -4 4 4 \N 0 4 \N \N 0 -5 5 5 \N 0 5 \N \N 0 +3 3 3 \N 0 \N 3 \N 0 +4 4 4 \N 0 \N 4 \N 0 +5 5 5 \N 0 \N 5 \N 0 6 6 6 \N 0 str_6 \N \N 0 7 7 7 \N 0 str_7 \N \N 0 8 8 8 \N 0 str_8 \N \N 0 9 9 9 \N 0 \N \N \N 0 10 10 10 \N 0 \N \N \N 0 11 11 11 \N 0 \N \N \N 0 -12 12 12 \N 0 12 \N \N 0 +12 12 12 \N 0 \N 12 \N 0 13 13 13 \N 0 str_13 \N \N 0 14 14 14 \N 0 \N \N \N 0 15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 +16 16 16 \N 0 \N 16 \N 0 17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 +18 18 18 \N 0 \N \N 1970-01-19 0 19 19 19 \N 0 \N \N \N 0 20 20 20 \N 0 \N 20 \N 0 21 21 21 \N 0 str_21 \N \N 0 22 22 22 \N 0 \N \N 1970-01-23 0 insert after alter modify column 3 -1 Date -1 UInt64 -12 None -12 String +1 String false +1 UInt64 false +2 Date true +5 String true +5 UInt64 true +12 None false 0 0 0 \N 0 \N \N \N 0 1 1 1 \N 0 \N \N \N 0 2 2 2 \N 0 \N \N \N 0 -3 3 3 \N 0 3 \N \N 0 -4 4 4 \N 0 4 \N \N 0 -5 5 5 \N 0 5 \N \N 0 +3 3 3 \N 0 \N 3 \N 0 +4 4 4 \N 0 \N 4 \N 0 +5 5 5 \N 0 \N 5 \N 0 6 6 6 \N 0 str_6 \N \N 0 7 7 7 \N 0 str_7 \N \N 0 8 8 8 \N 0 str_8 \N \N 0 9 9 9 \N 0 \N \N \N 0 10 10 10 \N 0 \N \N \N 0 11 11 11 \N 0 \N \N \N 0 -12 12 12 \N 0 12 \N \N 0 +12 12 12 \N 0 \N 12 \N 0 13 13 13 \N 0 str_13 \N \N 0 14 14 14 \N 0 \N \N \N 0 15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 +16 16 16 \N 0 \N 16 \N 0 17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 +18 18 18 \N 0 \N \N 1970-01-19 0 19 19 19 \N 0 \N \N \N 0 20 20 20 \N 0 \N 20 \N 0 21 21 21 \N 0 str_21 \N \N 0 diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.sql b/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.sql index 4ab700306d4..de05ba36177 100644 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.sql @@ -9,7 +9,7 @@ insert into test select number, number from numbers(3); select 'alter add column 1'; alter table test add column d Dynamic(max_types=3) settings mutations_sync=1; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, d, d.String, d.UInt64, d.`Tuple(a UInt64)`.a from test order by x; select 'insert after alter add column 1'; @@ -17,37 +17,37 @@ insert into test select number, number, number from numbers(3, 3); insert into test select number, number, 'str_' || toString(number) from numbers(6, 3); insert into test select number, number, NULL from numbers(9, 3); insert into test select number, number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL) from numbers(12, 3); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; select 'alter modify column 1'; -alter table test modify column d Dynamic(max_types=1) settings mutations_sync=1; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +alter table test modify column d Dynamic(max_types=0) settings mutations_sync=1; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; select 'insert after alter modify column 1'; insert into test select number, number, multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, toDate(number), NULL) from numbers(15, 4); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; select 'alter modify column 2'; -alter table test modify column d Dynamic(max_types=3) settings mutations_sync=1; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +alter table test modify column d Dynamic(max_types=2) settings mutations_sync=1; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; select 'insert after alter modify column 2'; insert into test select number, number, multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, toDate(number), NULL) from numbers(19, 4); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; select 'alter modify column 3'; alter table test modify column y Dynamic settings mutations_sync=1; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, y.UInt64, y.String, y.`Tuple(a UInt64)`.a, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; select 'insert after alter modify column 3'; insert into test select number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL), NULL from numbers(23, 3); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, y.UInt64, y.String, y.`Tuple(a UInt64)`.a, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; drop table test; \ No newline at end of file diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.reference index c592528c3cd..d7123288280 100644 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.reference +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.reference @@ -24,48 +24,28 @@ insert after alter add column 1 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 alter modify column 1 +4 String +4 UInt64 7 None -8 String 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 6 6 str_6 str_6 \N \N 0 7 7 str_7 str_7 \N \N 0 8 8 str_8 str_8 \N \N 0 9 9 \N \N \N \N 0 10 10 \N \N \N \N 0 11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 +12 12 12 \N 12 \N 0 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 insert after alter modify column 1 -8 None -11 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 -17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -alter modify column 2 -4 UInt64 -7 String +1 Date +5 String +5 UInt64 8 None 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 @@ -83,13 +63,37 @@ alter modify column 2 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 +16 16 16 \N 16 \N 0 17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -insert after alter modify column 2 +18 18 1970-01-19 \N \N 1970-01-19 0 +alter modify column 2 1 Date +5 String 5 UInt64 -8 String +8 None +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 \N 16 \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 +insert after alter modify column 2 +2 Date +6 String +6 UInt64 9 None 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 @@ -107,17 +111,17 @@ insert after alter modify column 2 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 +16 16 16 \N 16 \N 0 17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 19 19 \N \N \N \N 0 20 20 20 \N 20 \N 0 21 21 str_21 str_21 \N \N 0 22 22 1970-01-23 \N \N 1970-01-23 0 alter modify column 3 -1 Date -5 UInt64 -8 String +2 Date +6 String +6 UInt64 9 None 0 0 0 \N 0 \N \N \N 0 1 1 1 \N 0 \N \N \N 0 @@ -135,17 +139,17 @@ alter modify column 3 13 13 13 \N 0 str_13 \N \N 0 14 14 14 \N 0 \N \N \N 0 15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 +16 16 16 \N 0 \N 16 \N 0 17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 +18 18 18 \N 0 \N \N 1970-01-19 0 19 19 19 \N 0 \N \N \N 0 20 20 20 \N 0 \N 20 \N 0 21 21 21 \N 0 str_21 \N \N 0 22 22 22 \N 0 \N \N 1970-01-23 0 insert after alter modify column 3 -1 Date -5 UInt64 -8 String +2 Date +6 String +6 UInt64 12 None 0 0 0 \N 0 \N \N \N 0 1 1 1 \N 0 \N \N \N 0 @@ -163,9 +167,9 @@ insert after alter modify column 3 13 13 13 \N 0 str_13 \N \N 0 14 14 14 \N 0 \N \N \N 0 15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 +16 16 16 \N 0 \N 16 \N 0 17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 +18 18 18 \N 0 \N \N 1970-01-19 0 19 19 19 \N 0 \N \N \N 0 20 20 20 \N 0 \N 20 \N 0 21 21 21 \N 0 str_21 \N \N 0 diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.reference index 2ec301b747b..d7123288280 100644 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.reference +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.reference @@ -24,147 +24,152 @@ insert after alter add column 1 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 alter modify column 1 +4 String +4 UInt64 7 None -8 String 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 6 6 str_6 str_6 \N \N 0 7 7 str_7 str_7 \N \N 0 8 8 str_8 str_8 \N \N 0 9 9 \N \N \N \N 0 10 10 \N \N \N \N 0 11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 +12 12 12 \N 12 \N 0 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 insert after alter modify column 1 -8 None -11 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 -17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -alter modify column 2 -8 None -11 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 -17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -insert after alter modify column 2 1 Date -1 UInt64 -9 None -12 String +5 String +5 UInt64 +8 None 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 6 6 str_6 str_6 \N \N 0 7 7 str_7 str_7 \N \N 0 8 8 str_8 str_8 \N \N 0 9 9 \N \N \N \N 0 10 10 \N \N \N \N 0 11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 +12 12 12 \N 12 \N 0 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 +16 16 16 \N 16 \N 0 17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 +alter modify column 2 +1 Date +5 String +5 UInt64 +8 None +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 \N 16 \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 +insert after alter modify column 2 +2 Date +6 String +6 UInt64 +9 None +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 \N 16 \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 19 19 \N \N \N \N 0 20 20 20 \N 20 \N 0 21 21 str_21 str_21 \N \N 0 22 22 1970-01-23 \N \N 1970-01-23 0 alter modify column 3 -1 Date -1 UInt64 +2 Date +6 String +6 UInt64 9 None -12 String 0 0 0 \N 0 \N \N \N 0 1 1 1 \N 0 \N \N \N 0 2 2 2 \N 0 \N \N \N 0 -3 3 3 \N 0 3 \N \N 0 -4 4 4 \N 0 4 \N \N 0 -5 5 5 \N 0 5 \N \N 0 +3 3 3 \N 0 \N 3 \N 0 +4 4 4 \N 0 \N 4 \N 0 +5 5 5 \N 0 \N 5 \N 0 6 6 6 \N 0 str_6 \N \N 0 7 7 7 \N 0 str_7 \N \N 0 8 8 8 \N 0 str_8 \N \N 0 9 9 9 \N 0 \N \N \N 0 10 10 10 \N 0 \N \N \N 0 11 11 11 \N 0 \N \N \N 0 -12 12 12 \N 0 12 \N \N 0 +12 12 12 \N 0 \N 12 \N 0 13 13 13 \N 0 str_13 \N \N 0 14 14 14 \N 0 \N \N \N 0 15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 +16 16 16 \N 0 \N 16 \N 0 17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 +18 18 18 \N 0 \N \N 1970-01-19 0 19 19 19 \N 0 \N \N \N 0 20 20 20 \N 0 \N 20 \N 0 21 21 21 \N 0 str_21 \N \N 0 22 22 22 \N 0 \N \N 1970-01-23 0 insert after alter modify column 3 -1 Date -1 UInt64 +2 Date +6 String +6 UInt64 12 None -12 String 0 0 0 \N 0 \N \N \N 0 1 1 1 \N 0 \N \N \N 0 2 2 2 \N 0 \N \N \N 0 -3 3 3 \N 0 3 \N \N 0 -4 4 4 \N 0 4 \N \N 0 -5 5 5 \N 0 5 \N \N 0 +3 3 3 \N 0 \N 3 \N 0 +4 4 4 \N 0 \N 4 \N 0 +5 5 5 \N 0 \N 5 \N 0 6 6 6 \N 0 str_6 \N \N 0 7 7 7 \N 0 str_7 \N \N 0 8 8 8 \N 0 str_8 \N \N 0 9 9 9 \N 0 \N \N \N 0 10 10 10 \N 0 \N \N \N 0 11 11 11 \N 0 \N \N \N 0 -12 12 12 \N 0 12 \N \N 0 +12 12 12 \N 0 \N 12 \N 0 13 13 13 \N 0 str_13 \N \N 0 14 14 14 \N 0 \N \N \N 0 15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 +16 16 16 \N 0 \N 16 \N 0 17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 +18 18 18 \N 0 \N \N 1970-01-19 0 19 19 19 \N 0 \N \N \N 0 20 20 20 \N 0 \N 20 \N 0 21 21 21 \N 0 str_21 \N \N 0 diff --git a/tests/queries/0_stateless/03041_dynamic_type_check_table.sh b/tests/queries/0_stateless/03041_dynamic_type_check_table.sh index c8bd533e253..da24b892cbd 100755 --- a/tests/queries/0_stateless/03041_dynamic_type_check_table.sh +++ b/tests/queries/0_stateless/03041_dynamic_type_check_table.sh @@ -13,7 +13,7 @@ function run() $CH_CLIENT -q "insert into test select number, number from numbers(3)" echo "alter add column" - $CH_CLIENT -q "alter table test add column d Dynamic(max_types=3) settings mutations_sync=1" + $CH_CLIENT -q "alter table test add column d Dynamic(max_types=2) settings mutations_sync=1" $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" $CH_CLIENT -q "select x, y, d, d.String, d.UInt64, d.\`Tuple(a UInt64)\`.a from test order by x" diff --git a/tests/queries/0_stateless/03150_dynamic_type_mv_insert.reference b/tests/queries/0_stateless/03150_dynamic_type_mv_insert.reference index 0b76d30953e..370e6352657 100644 --- a/tests/queries/0_stateless/03150_dynamic_type_mv_insert.reference +++ b/tests/queries/0_stateless/03150_dynamic_type_mv_insert.reference @@ -1,35 +1,35 @@ 1 2024-01-01 Date 2 1704056400 Decimal(18, 3) -3 1 String -4 2 String +3 1 Float32 +4 2 Float64 1 2024-01-01 Date 1 2024-01-01 Date 2 1704056400 Decimal(18, 3) 2 1704056400 Decimal(18, 3) -3 1 String -3 1 String -4 2 String -4 2 String - -1 2024-01-01 String -1 2024-01-01 String -2 1704056400 String -2 1704056400 String -3 1 String -3 1 String -4 2 String -4 2 String +3 1 Float32 +3 1 Float32 +4 2 Float64 +4 2 Float64 1 2024-01-01 Date -1 2024-01-01 String -1 2024-01-01 String +1 2024-01-01 Date 2 1704056400 Decimal(18, 3) -2 1704056400 String -2 1704056400 String -3 1 String -3 1 String -3 1 String -4 2 String -4 2 String -4 2 String +2 1704056400 Decimal(18, 3) +3 1 Float32 +3 1 Float32 +4 2 Float64 +4 2 Float64 + +1 2024-01-01 Date +1 2024-01-01 Date +1 2024-01-01 Date +2 1704056400 Decimal(18, 3) +2 1704056400 Decimal(18, 3) +2 1704056400 Decimal(18, 3) +3 1 Float32 +3 1 Float32 +3 1 Float32 +4 2 Float64 +4 2 Float64 +4 2 Float64 diff --git a/tests/queries/0_stateless/03150_dynamic_type_mv_insert.sql b/tests/queries/0_stateless/03150_dynamic_type_mv_insert.sql index ad5ea9512c6..71d5dd4abd1 100644 --- a/tests/queries/0_stateless/03150_dynamic_type_mv_insert.sql +++ b/tests/queries/0_stateless/03150_dynamic_type_mv_insert.sql @@ -1,5 +1,6 @@ SET allow_experimental_dynamic_type=1; +DROP TABLE IF EXISTS null_table; CREATE TABLE null_table ( n1 UInt8, @@ -7,9 +8,11 @@ CREATE TABLE null_table ) ENGINE = Null; +DROP VIEW IF EXISTS dummy_rmv; CREATE MATERIALIZED VIEW dummy_rmv TO to_table AS SELECT * FROM null_table; +DROP TABLE IF EXISTS to_table; CREATE TABLE to_table ( n1 UInt8, @@ -32,3 +35,7 @@ select ''; ALTER TABLE to_table MODIFY COLUMN n2 Dynamic(max_types=10); INSERT INTO null_table ( n1, n2 ) VALUES (1, '2024-01-01'), (2, toDateTime64('2024-01-01', 3, 'Asia/Istanbul')), (3, toFloat32(1)), (4, toFloat64(2)); SELECT *, dynamicType(n2) FROM to_table ORDER BY ALL; + +DROP TABLE null_table; +DROP VIEW dummy_rmv; +DROP TABLE to_table; diff --git a/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.reference b/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.reference index d96fbf658d8..2d3b2f118f6 100644 --- a/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.reference +++ b/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.reference @@ -1,26 +1,26 @@ -1 2024-01-01 Date -2 1704056400 String -3 1 String -4 2 String +1 2024-01-01 Date false +2 1704056400 Decimal(18, 3) false +3 1 Float32 true +4 2 Float64 true -1 2024-01-01 Date -1 2024-01-01 Date -2 1704056400 Decimal(18, 3) -2 1704056400 String -3 1 Float32 -3 1 String -4 2 Float64 -4 2 String +1 2024-01-01 Date false +1 2024-01-01 Date false +2 1704056400 Decimal(18, 3) false +2 1704056400 Decimal(18, 3) false +3 1 Float32 false +3 1 Float32 false +4 2 Float64 false +4 2 Float64 false -1 2024-01-01 String -1 2024-01-01 String -1 2024-01-01 String -2 1704056400 String -2 1704056400 String -2 1704056400 String -3 1 String -3 1 String -3 1 String -4 2 String -4 2 String -4 2 String +1 2024-01-01 Date true +1 2024-01-01 Date true +1 2024-01-01 Date true +2 1704056400 Decimal(18, 3) true +2 1704056400 Decimal(18, 3) true +2 1704056400 Decimal(18, 3) true +3 1 Float32 true +3 1 Float32 true +3 1 Float32 true +4 2 Float64 true +4 2 Float64 true +4 2 Float64 true diff --git a/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.sql b/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.sql index 632f3504fdb..e476d34a1db 100644 --- a/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.sql +++ b/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.sql @@ -11,16 +11,16 @@ CREATE TABLE to_table ENGINE = MergeTree ORDER BY n1; INSERT INTO to_table ( n1, n2 ) VALUES (1, '2024-01-01'), (2, toDateTime64('2024-01-01', 3, 'Asia/Istanbul')), (3, toFloat32(1)), (4, toFloat64(2)); -SELECT *, dynamicType(n2) FROM to_table ORDER BY ALL; +SELECT *, dynamicType(n2), isDynamicElementInSharedData(n2) FROM to_table ORDER BY ALL; select ''; ALTER TABLE to_table MODIFY COLUMN n2 Dynamic(max_types=5); INSERT INTO to_table ( n1, n2 ) VALUES (1, '2024-01-01'), (2, toDateTime64('2024-01-01', 3, 'Asia/Istanbul')), (3, toFloat32(1)), (4, toFloat64(2)); -SELECT *, dynamicType(n2) FROM to_table ORDER BY ALL; +SELECT *, dynamicType(n2), isDynamicElementInSharedData(n2) FROM to_table ORDER BY ALL; select ''; -ALTER TABLE to_table MODIFY COLUMN n2 Dynamic(max_types=1); +ALTER TABLE to_table MODIFY COLUMN n2 Dynamic(max_types=0); INSERT INTO to_table ( n1, n2 ) VALUES (1, '2024-01-01'), (2, toDateTime64('2024-01-01', 3, 'Asia/Istanbul')), (3, toFloat32(1)), (4, toFloat64(2)); -SELECT *, dynamicType(n2) FROM to_table ORDER BY ALL; +SELECT *, dynamicType(n2), isDynamicElementInSharedData(n2) FROM to_table ORDER BY ALL; ALTER TABLE to_table MODIFY COLUMN n2 Dynamic(max_types=500); -- { serverError UNEXPECTED_AST_STRUCTURE } diff --git a/tests/queries/0_stateless/03152_dynamic_type_simple.reference b/tests/queries/0_stateless/03152_dynamic_type_simple.reference index 5f243209ff3..e508bdd1990 100644 --- a/tests/queries/0_stateless/03152_dynamic_type_simple.reference +++ b/tests/queries/0_stateless/03152_dynamic_type_simple.reference @@ -7,7 +7,7 @@ string2 String \N None 42 Int64 42 Int64 string String -string String [1, 2] String +string String [1,2] Array(Int64) [1,2] Array(Int64) \N None ┌─d────────────────────────┬─dynamicType(d)─┬─d.Int64─┬─d.String─┬─────d.Date─┬─d.Float64─┬──────────d.DateTime─┬─d.Array(Int64)─┬─d.Array(String)──────────┐ 1. │ 42 │ Int64 │ 42 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ [] │ [] │ diff --git a/tests/queries/0_stateless/03152_dynamic_type_simple.sql b/tests/queries/0_stateless/03152_dynamic_type_simple.sql index fd5328faf15..ed24b213b1c 100644 --- a/tests/queries/0_stateless/03152_dynamic_type_simple.sql +++ b/tests/queries/0_stateless/03152_dynamic_type_simple.sql @@ -1,14 +1,17 @@ SET allow_experimental_dynamic_type=1; +DROP TABLE IF EXISTS test_max_types; CREATE TABLE test_max_types (d Dynamic(max_types=5)) ENGINE = Memory; INSERT INTO test_max_types VALUES ('string1'), (42), (3.14), ([1, 2]), (toDate('2021-01-01')), ('string2'); SELECT d, dynamicType(d) FROM test_max_types; SELECT ''; +DROP TABLE IF EXISTS test_nested_dynamic; CREATE TABLE test_nested_dynamic (d1 Dynamic, d2 Dynamic(max_types=2)) ENGINE = Memory; INSERT INTO test_nested_dynamic VALUES (NULL, 42), (42, 'string'), ('string', [1, 2]), ([1, 2], NULL); SELECT d1, dynamicType(d1), d2, dynamicType(d2) FROM test_nested_dynamic; +DROP TABLE IF EXISTS test_rapid_schema; CREATE TABLE test_rapid_schema (d Dynamic) ENGINE = Memory; INSERT INTO test_rapid_schema VALUES (42), ('string1'), (toDate('2021-01-01')), ([1, 2, 3]), (3.14), ('string2'), (toDateTime('2021-01-01 12:00:00')), (['array', 'of', 'strings']), (NULL), (toFloat64(42.42)); @@ -27,3 +30,8 @@ FROM FROM numbers(10000) ) ); + +DROP TABLE test_max_types; +DROP TABLE test_nested_dynamic; +DROP TABLE test_rapid_schema; + diff --git a/tests/queries/0_stateless/03153_dynamic_type_empty.sql b/tests/queries/0_stateless/03153_dynamic_type_empty.sql index 8e942fe6f6e..3a0c98e63ee 100644 --- a/tests/queries/0_stateless/03153_dynamic_type_empty.sql +++ b/tests/queries/0_stateless/03153_dynamic_type_empty.sql @@ -1,5 +1,7 @@ SET allow_experimental_dynamic_type=1; +DROP TABLE IF EXISTS test_null_empty; CREATE TABLE test_null_empty (d Dynamic) ENGINE = Memory; INSERT INTO test_null_empty VALUES ([]), ([1]), ([]), (['1']), ([]), (()),((1)), (()), (('1')), (()), ({}), ({1:2}), ({}), ({'1':'2'}), ({}); SELECT d, dynamicType(d) FROM test_null_empty; +DROP TABLE test_null_empty; diff --git a/tests/queries/0_stateless/03159_dynamic_type_all_types.sql b/tests/queries/0_stateless/03159_dynamic_type_all_types.sql index d302205ca23..fffea1bd0f5 100644 --- a/tests/queries/0_stateless/03159_dynamic_type_all_types.sql +++ b/tests/queries/0_stateless/03159_dynamic_type_all_types.sql @@ -6,7 +6,7 @@ SET allow_experimental_variant_type=1; SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE t (d Dynamic(max_types=255)) ENGINE = Memory; +CREATE TABLE t (d Dynamic(max_types=254)) ENGINE = Memory; -- Integer types: signed and unsigned integers (UInt8, UInt16, UInt32, UInt64, UInt128, UInt256, Int8, Int16, Int32, Int64, Int128, Int256) INSERT INTO t VALUES (-128::Int8), (-127::Int8), (-1::Int8), (0::Int8), (1::Int8), (126::Int8), (127::Int8); INSERT INTO t VALUES (-128::Int8), (-127::Int8), (-1::Int8), (0::Int8), (1::Int8), (126::Int8), (127::Int8); @@ -84,7 +84,7 @@ INSERT INTO t VALUES ([(1, (2, ['aa', 'bb']), [(3, 'cc'), (4, 'dd')]), (5, (6, [ SELECT dynamicType(d), d FROM t ORDER BY substring(dynamicType(d),1,1), length(dynamicType(d)), d; -CREATE TABLE t2 (d Dynamic(max_types=255)) ENGINE = Memory; +CREATE TABLE t2 (d Dynamic(max_types=254)) ENGINE = Memory; INSERT INTO t2 SELECT * FROM t; SELECT ''; diff --git a/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh b/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh index 9b57e5c8718..b9bab2bd70b 100755 --- a/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh +++ b/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -q "drop table if exists test" -$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 -q "create table test (id UInt64, d Dynamic(max_types=255)) engine=Memory" +$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 -q "create table test (id UInt64, d Dynamic(max_types=254)) engine=Memory" $CLICKHOUSE_CLIENT -q "insert into test select 0, NULL" $CLICKHOUSE_CLIENT -q "insert into test select 1, materialize(42)::UInt8" @@ -58,6 +58,6 @@ $CLICKHOUSE_CLIENT -q "insert into test select 47, materialize([[(20, 20), (50, $CLICKHOUSE_CLIENT -q "insert into test select 48, materialize([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]])::MultiPolygon" $CLICKHOUSE_CLIENT -q "insert into test select 49, materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])" -$CLICKHOUSE_CLIENT -q "select * from test format RowBinary" | $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --input-format RowBinary --structure 'id UInt64, d Dynamic(max_types=255)' -q "select d, dynamicType(d) from table order by id" +$CLICKHOUSE_CLIENT -q "select * from test format RowBinary" | $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --input-format RowBinary --structure 'id UInt64, d Dynamic(max_types=254)' -q "select d, dynamicType(d) from table order by id" $CLICKHOUSE_CLIENT -q "drop table test" diff --git a/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.reference b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.reference index 6d2c1334d6e..8d2470dea44 100644 --- a/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.reference +++ b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.reference @@ -1,10 +1,10 @@ -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +1 +2 +3 +4 +0 +1 +2 +3 +4 diff --git a/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql index 25f3bb0f4c8..939b49e1599 100644 --- a/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql +++ b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql @@ -1,6 +1,8 @@ set allow_experimental_dynamic_type=1; +drop table if exists test; create table test (d Dynamic) engine=Memory; insert into table test select * from numbers(5); -alter table test modify column d Dynamic(max_types=1); +alter table test modify column d Dynamic(max_types=0); select d.UInt64 from test settings enable_analyzer=1; select d.UInt64 from test settings enable_analyzer=0; +drop table test; From c1c32daf01bba08129dd17e2b3a108cd7e837528 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 9 Aug 2024 21:59:40 +0000 Subject: [PATCH 531/644] Fix tests --- .../Serializations/SerializationDynamic.cpp | 2 +- tests/queries/0_stateless/00000_test.sql | 43 ------------------- ...native_with_binary_encoded_types.reference | 4 +- ...ry_and_native_with_binary_encoded_types.sh | 6 +-- 4 files changed, 6 insertions(+), 49 deletions(-) delete mode 100644 tests/queries/0_stateless/00000_test.sql diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index 67b29750948..5fadb6e4de4 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -130,7 +130,7 @@ void SerializationDynamic::serializeBinaryBulkStatePrefix( for (const auto & variant: variants) { if (variant->getName() != ColumnDynamic::getSharedVariantTypeName()) - encodeDataType(dynamic_state->variant_type, *stream); + encodeDataType(variant, *stream); } } else diff --git a/tests/queries/0_stateless/00000_test.sql b/tests/queries/0_stateless/00000_test.sql deleted file mode 100644 index db9dd774484..00000000000 --- a/tests/queries/0_stateless/00000_test.sql +++ /dev/null @@ -1,43 +0,0 @@ -set allow_experimental_variant_type = 1; -set use_variant_as_common_type = 1; -set allow_experimental_dynamic_type = 1; -set enable_named_columns_in_function_tuple = 0; -drop table if exists test; -create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1; - - -system stop merges test; -insert into test select number, number from numbers(10); -insert into test select number, tuple(if(number % 3 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(10); -insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(5); -insert into test select number, multiIf(number % 5 == 0, tuple(if(number % 3 == 0, toDateTime(number), toIPv4(number)))::Tuple(a Dynamic(max_types=3)), number % 5 == 1 or number % 5 == 2, number, 'str_' || number) from numbers(10); - -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; -system start merges test; -optimize table test final; -select '---------------------'; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; - -system stop merges test; -insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(5); -insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(20); - -select '---------------------'; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; -system start merges test; -optimize table test final; -select '---------------------'; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; - -system stop merges test; -insert into test select number, tuple(toDateTime(number))::Tuple(a Dynamic(max_types=3)) from numbers(4); - -select '---------------------'; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; -system start merges test; -optimize table test final; -select '---------------------'; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; - -drop table test; - diff --git a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference index 1ba147f9627..7de0804e0f2 100644 --- a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference +++ b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference @@ -110,5 +110,5 @@ true Bool [{42:(1,[(2,{1:2})])}] Dynamic [{42:(1,[(2,{1:2})])}] Dynamic(max_types=10) [{42:(1,[(2,{1:2})])}] Dynamic(max_types=10) -[{42:(1,[(2,{1:2})])}] Dynamic(max_types=255) -[{42:(1,[(2,{1:2})])}] Dynamic(max_types=255) +[{42:(1,[(2,{1:2})])}] Dynamic(max_types=254) +[{42:(1,[(2,{1:2})])}] Dynamic(max_types=254) diff --git a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh index 0c585d36348..1e674a29072 100755 --- a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh +++ b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh @@ -6,8 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function test { - $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_binary_encode_types_in_binary_format=1 -q "select $1 as value format RowBinaryWithNamesAndTypes" | $CLICKHOUSE_LOCAL --input-format RowBinaryWithNamesAndTypes --input_format_binary_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" - $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_native_encode_types_in_binary_format=1 -q "select $1 as value format Native" | $CLICKHOUSE_LOCAL --input-format Native --input_format_native_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" + $CLICKHOUSE_LOCAL --stacktrace --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_binary_encode_types_in_binary_format=1 -q "select $1 as value format RowBinaryWithNamesAndTypes" | $CLICKHOUSE_LOCAL --input-format RowBinaryWithNamesAndTypes --input_format_binary_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" + $CLICKHOUSE_LOCAL --stacktrace --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_native_encode_types_in_binary_format=1 -q "select $1 as value format Native" | $CLICKHOUSE_LOCAL --input-format Native --input_format_native_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" } test "materialize(42)::UInt8" @@ -66,4 +66,4 @@ test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])" test "materialize(42::UInt32)::Variant(UInt32, String, Tuple(a UInt32, b Array(Map(String, String))))" test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])::Dynamic" test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])::Dynamic(max_types=10)" -test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])::Dynamic(max_types=255)" +test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])::Dynamic(max_types=254)" From 805a2e33bfedbdfc2393217e7b485761b1943e1d Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 10 Aug 2024 00:47:43 +0000 Subject: [PATCH 532/644] Fix unit tests --- src/Columns/tests/gtest_column_dynamic.cpp | 74 +++++++++++----------- 1 file changed, 37 insertions(+), 37 deletions(-) diff --git a/src/Columns/tests/gtest_column_dynamic.cpp b/src/Columns/tests/gtest_column_dynamic.cpp index 5445bd525d9..de76261229d 100644 --- a/src/Columns/tests/gtest_column_dynamic.cpp +++ b/src/Columns/tests/gtest_column_dynamic.cpp @@ -7,7 +7,7 @@ using namespace DB; TEST(ColumnDynamic, CreateEmpty) { - auto column = ColumnDynamic::create(255); + auto column = ColumnDynamic::create(254); ASSERT_TRUE(column->empty()); ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant(SharedVariant)"); ASSERT_EQ(column->getVariantInfo().variant_names.size(), 1); @@ -19,7 +19,7 @@ TEST(ColumnDynamic, CreateEmpty) TEST(ColumnDynamic, InsertDefault) { - auto column = ColumnDynamic::create(255); + auto column = ColumnDynamic::create(254); column->insertDefault(); ASSERT_TRUE(column->size() == 1); ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant(SharedVariant)"); @@ -34,7 +34,7 @@ TEST(ColumnDynamic, InsertDefault) TEST(ColumnDynamic, InsertFields) { - auto column = ColumnDynamic::create(255); + auto column = ColumnDynamic::create(254); column->insert(Field(42)); column->insert(Field(-42)); column->insert(Field("str1")); @@ -56,7 +56,7 @@ TEST(ColumnDynamic, InsertFields) ColumnDynamic::MutablePtr getDynamicWithManyVariants(size_t num_variants, Field tuple_element = Field(42)) { - auto column = ColumnDynamic::create(255); + auto column = ColumnDynamic::create(254); for (size_t i = 0; i != num_variants; ++i) { Tuple tuple; @@ -136,7 +136,7 @@ TEST(ColumnDynamic, InsertFieldsOverflow2) ColumnDynamic::MutablePtr getInsertFromColumn(size_t num = 1) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); for (size_t i = 0; i != num; ++i) { column_from->insert(Field(42)); @@ -170,13 +170,13 @@ void checkInsertFrom(const ColumnDynamic::MutablePtr & column_from, ColumnDynami TEST(ColumnDynamic, InsertFrom1) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); checkInsertFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertFrom2) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(42)); column_to->insert(Field(42.42)); column_to->insert(Field("str")); @@ -186,7 +186,7 @@ TEST(ColumnDynamic, InsertFrom2) TEST(ColumnDynamic, InsertFrom3) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(42)); column_to->insert(Field(42.42)); column_to->insert(Field("str")); @@ -197,7 +197,7 @@ TEST(ColumnDynamic, InsertFrom3) TEST(ColumnDynamic, InsertFromOverflow1) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(42.42)); column_from->insert(Field("str")); @@ -227,7 +227,7 @@ TEST(ColumnDynamic, InsertFromOverflow1) TEST(ColumnDynamic, InsertFromOverflow2) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(42.42)); @@ -251,7 +251,7 @@ TEST(ColumnDynamic, InsertFromOverflow3) column_from->insert(Field(42)); column_from->insert(Field(42.42)); - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(41)); column_to->insertFrom(*column_from, 0); @@ -297,13 +297,13 @@ void checkInsertManyFrom(const ColumnDynamic::MutablePtr & column_from, ColumnDy TEST(ColumnDynamic, InsertManyFrom1) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); checkInsertManyFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertManyFrom2) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(42)); column_to->insert(Field(42.42)); column_to->insert(Field("str")); @@ -313,7 +313,7 @@ TEST(ColumnDynamic, InsertManyFrom2) TEST(ColumnDynamic, InsertManyFrom3) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(42)); column_to->insert(Field(42.42)); column_to->insert(Field("str")); @@ -324,7 +324,7 @@ TEST(ColumnDynamic, InsertManyFrom3) TEST(ColumnDynamic, InsertManyFromOverflow1) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(42.42)); column_from->insert(Field("str")); @@ -361,7 +361,7 @@ TEST(ColumnDynamic, InsertManyFromOverflow1) TEST(ColumnDynamic, InsertManyFromOverflow2) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(42.42)); @@ -393,7 +393,7 @@ TEST(ColumnDynamic, InsertManyFromOverflow3) column_from->insert(Field(42)); column_from->insert(Field(42.42)); - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(41)); column_to->insertManyFrom(*column_from, 0, 2); @@ -441,13 +441,13 @@ void checkInsertRangeFrom(const ColumnDynamic::MutablePtr & column_from, ColumnD TEST(ColumnDynamic, InsertRangeFrom1) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); checkInsertRangeFrom(getInsertFromColumn(2), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertRangeFrom2) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(42)); column_to->insert(Field(42.42)); column_to->insert(Field("str1")); @@ -457,7 +457,7 @@ TEST(ColumnDynamic, InsertRangeFrom2) TEST(ColumnDynamic, InsertRangeFrom3) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(42)); column_to->insert(Field(42.42)); column_to->insert(Field("str1")); @@ -468,7 +468,7 @@ TEST(ColumnDynamic, InsertRangeFrom3) TEST(ColumnDynamic, InsertRangeFromOverflow1) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(43)); column_from->insert(Field(42.42)); @@ -494,7 +494,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow1) TEST(ColumnDynamic, InsertRangeFromOverflow2) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(43)); column_from->insert(Field(42.42)); @@ -516,7 +516,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow2) TEST(ColumnDynamic, InsertRangeFromOverflow3) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(43)); column_from->insert(Field(42.42)); @@ -539,7 +539,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow3) TEST(ColumnDynamic, InsertRangeFromOverflow4) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(42.42)); column_from->insert(Field("str")); @@ -561,7 +561,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow4) TEST(ColumnDynamic, InsertRangeFromOverflow5) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(43)); column_from->insert(Field(42.42)); @@ -587,7 +587,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow5) TEST(ColumnDynamic, InsertRangeFromOverflow6) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(43)); column_from->insert(Field(44)); @@ -619,7 +619,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow6) TEST(ColumnDynamic, InsertRangeFromOverflow7) { - auto column_from = ColumnDynamic::create(3); + auto column_from = ColumnDynamic::create(2); column_from->insert(Field(42.42)); column_from->insert(Field("str1")); column_from->insert(Field(42)); @@ -629,7 +629,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow7) column_from->insert(Field("str2")); column_from->insert(Field(Array({Field(42)}))); - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(42)); column_to->insertRangeFrom(*column_from, 0, 8); @@ -659,7 +659,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow7) TEST(ColumnDynamic, InsertRangeFromOverflow8) { - auto column_from = ColumnDynamic::create(3); + auto column_from = ColumnDynamic::create(2); column_from->insert(Field(42.42)); column_from->insert(Field("str1")); column_from->insert(Field(42)); @@ -669,7 +669,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow8) column_from->insert(Field("str2")); column_from->insert(Field(Array({Field(42)}))); - auto column_to = ColumnDynamic::create(3); + auto column_to = ColumnDynamic::create(2); column_to->insert(Field(42)); column_from->insert(Field("str1")); @@ -711,7 +711,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow9) column_from->insert(Field("str2")); column_from->insert(Field(Array({Field(42)}))); - auto column_to = ColumnDynamic::create(3); + auto column_to = ColumnDynamic::create(2); column_to->insert(Field(42)); column_to->insertRangeFrom(*column_from, 0, 9); @@ -743,7 +743,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow9) TEST(ColumnDynamic, SerializeDeserializeFromArena1) { - auto column = ColumnDynamic::create(255); + auto column = ColumnDynamic::create(254); column->insert(Field(42)); column->insert(Field(42.42)); column->insert(Field("str")); @@ -768,7 +768,7 @@ TEST(ColumnDynamic, SerializeDeserializeFromArena1) TEST(ColumnDynamic, SerializeDeserializeFromArena2) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(42.42)); column_from->insert(Field("str")); @@ -781,7 +781,7 @@ TEST(ColumnDynamic, SerializeDeserializeFromArena2) column_from->serializeValueIntoArena(2, arena, pos); column_from->serializeValueIntoArena(3, arena, pos); - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); pos = column_to->deserializeAndInsertFromArena(ref1.data); pos = column_to->deserializeAndInsertFromArena(pos); pos = column_to->deserializeAndInsertFromArena(pos); @@ -800,7 +800,7 @@ TEST(ColumnDynamic, SerializeDeserializeFromArena2) TEST(ColumnDynamic, SerializeDeserializeFromArenaOverflow1) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(42.42)); column_from->insert(Field("str")); @@ -846,7 +846,7 @@ TEST(ColumnDynamic, SerializeDeserializeFromArenaOverflow2) column_from->serializeValueIntoArena(3, arena, pos); column_from->serializeValueIntoArena(4, arena, pos); - auto column_to = ColumnDynamic::create(3); + auto column_to = ColumnDynamic::create(2); column_to->insert(Field(42.42)); pos = column_to->deserializeAndInsertFromArena(ref1.data); pos = column_to->deserializeAndInsertFromArena(pos); @@ -882,7 +882,7 @@ TEST(ColumnDynamic, skipSerializedInArena) auto ref4 = column_from->serializeValueIntoArena(3, arena, pos); const char * end = ref4.data + ref4.size; - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); pos = column_to->skipSerializedInArena(ref1.data); pos = column_to->skipSerializedInArena(pos); pos = column_to->skipSerializedInArena(pos); From c26b3cb4452931ee3bb3355b47dafb364744c9ab Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 10 Aug 2024 02:27:23 +0000 Subject: [PATCH 533/644] handle the case of packed storage --- src/Storages/MergeTree/MutateTask.cpp | 11 ++++++++--- .../03161_lightweight_delete_projection.sql | 4 +++- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 3d9f49c9a7a..0f0428287b6 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1554,6 +1554,10 @@ private: removed_projections.insert(command.column_name); } + bool lightweight_delete_mode = ctx->updated_header.has(RowExistsColumn::name); + bool lightweight_delete_drop = lightweight_delete_mode + && ctx->data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::DROP; + const auto & projections = ctx->metadata_snapshot->getProjections(); for (const auto & projection : projections) { @@ -1561,10 +1565,11 @@ private: continue; bool need_recalculate = - ctx->materialized_projections.contains(projection.name) + (ctx->materialized_projections.contains(projection.name) || (!is_full_part_storage && ctx->source_part->hasProjection(projection.name) - && !ctx->source_part->hasBrokenProjection(projection.name)); + && !ctx->source_part->hasBrokenProjection(projection.name))) + && !lightweight_delete_drop; if (need_recalculate) { @@ -1572,7 +1577,7 @@ private: } else { - if (!ctx->updated_header.has(RowExistsColumn::name) && ctx->source_part->checksums.has(projection.getDirectoryName())) + if (!lightweight_delete_mode && ctx->source_part->checksums.has(projection.getDirectoryName())) entries_to_hardlink.insert(projection.getDirectoryName()); } } diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 0b05326e2c1..da6427cbf22 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -1,5 +1,7 @@ +-- For cloud version, should also consider min_bytes_for_full_part_storage since packed storage exists, +-- but for less redundancy, just let CI test the parameter. -SET max_threads = 1, lightweight_deletes_sync = 2, alter_sync = 2; +SET lightweight_deletes_sync = 2, alter_sync = 2; DROP TABLE IF EXISTS users_compact; From 9b9fff4232d80e579b1d23ced8bfbb1b2c5e2147 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Sat, 10 Aug 2024 08:48:08 +0200 Subject: [PATCH 534/644] Push CI From e582118544f3c49c3c6600ac8fa252151714d25f Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 10 Aug 2024 13:09:40 +0200 Subject: [PATCH 535/644] review changes --- .../functions/type-conversion-functions.md | 916 +++++++++++++++--- 1 file changed, 801 insertions(+), 115 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 8e72fea7fdb..5c06e72f977 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -64,9 +64,8 @@ toInt8(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -251,9 +250,8 @@ toInt8OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `Int8` is unsuccessful. [Int8](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -314,9 +312,8 @@ toInt16(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -501,9 +498,8 @@ toInt16OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `Int16` is unsuccessful. [Int16](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -564,9 +560,8 @@ toInt32(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -750,9 +745,8 @@ toInt32OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `Int32` is unsuccessful. [Int32](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -813,9 +807,8 @@ toInt64(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported types: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -1000,9 +993,8 @@ toInt64OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `Int64` is unsuccessful. [Int64](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -1063,9 +1055,8 @@ toInt128(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -1312,9 +1303,8 @@ toInt256(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -1498,9 +1488,8 @@ toInt256OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `Int256` is unsuccessful. [Int256](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf` @@ -1561,9 +1550,8 @@ toUInt8(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -1748,9 +1736,8 @@ toUInt8OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `UInt8` is unsuccessful. [UInt8](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -1811,9 +1798,8 @@ toUInt16(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -1998,9 +1984,8 @@ toUInt16OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `UInt16` is unsuccessful. [UInt16](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -2061,9 +2046,8 @@ toUInt32(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -2249,9 +2233,8 @@ toUInt32OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `UInt32` is unsuccessful. [UInt32](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -2312,9 +2295,8 @@ toUInt64(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported types: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -2499,9 +2481,8 @@ toUInt64OrDefault(expr[, default]) - `defauult` (optional) — The default value to return if parsing to type `UInt64` is unsuccessful. [UInt64](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -2562,9 +2543,8 @@ toUInt128(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -2811,9 +2791,8 @@ toUInt256(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -2997,9 +2976,8 @@ toUInt256OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `UInt256` is unsuccessful. [UInt256](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf` @@ -3555,7 +3533,7 @@ toDecimal32(expr, S) **Arguments** - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). -- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `S` — Scale parameter between 0 and 9, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). Supported arguments: - Values or string representations of type (U)Int8/16/32/64/128/256. @@ -3566,7 +3544,7 @@ Unsupported arguments: - String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32('0xc0fe', 1);`. :::note -Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. Excessive digits in a fraction are discarded (not rounded). Excessive digits in the integer part will lead to an exception. ::: @@ -3619,20 +3597,20 @@ toDecimal32OrZero(expr, S) **Arguments** - `expr` — A String representation of a number. [String](../data-types/string.md). -- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `S` — Scale parameter between 0 and 9, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). Supported arguments: - String representations of type (U)Int8/16/32/64/128/256. - String representations of type Float32/64. Unsupported arguments: -- String representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of Float32/64 values `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32OrZero('0xc0fe', 1);`. :::note -Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. Excessive digits in a fraction are discarded (not rounded). -Excessive digits in the integer part will lead to an exception. +Excessive digits in the integer part will lead to an error. ::: **Returned value** @@ -3645,8 +3623,10 @@ Query: ``` sql SELECT - toDecimal32OrZero(toString(-1.111), 5) AS val, - toTypeName(val) + toDecimal32OrZero(toString(-1.111), 5) AS a, + toTypeName(a), + toDecimal32OrZero(toString('Inf'), 5) as b, + toTypeName(b) FORMAT Vertical; ``` @@ -3655,26 +3635,10 @@ Result: ```response Row 1: ────── -val: -1.111 -toTypeName(val): Decimal(9, 5) -``` - -Query: - -``` sql -SELECT - toDecimal32OrZero(toString(-1.111), 2) AS val, - toTypeName(val) -FORMAT Vertical; -``` - -Result: - -```response -Row 1: -────── -val: -1.11 -toTypeName(val): Decimal(9, 2) +a: -1.111 +toTypeName(a): Decimal(9, 5) +b: 0 +toTypeName(b): Decimal(9, 5) ``` **See also** @@ -3696,20 +3660,20 @@ toDecimal32OrNull(expr, S) **Arguments** - `expr` — A String representation of a number. [String](../data-types/string.md). -- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `S` — Scale parameter between 0 and 9, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). Supported arguments: - String representations of type (U)Int8/16/32/64/128/256. - String representations of type Float32/64. Unsupported arguments: -- String representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of Float32/64 values `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32OrNull('0xc0fe', 1);`. :::note -Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. Excessive digits in a fraction are discarded (not rounded). -Excessive digits in the integer part will lead to an exception. +Excessive digits in the integer part will lead to an error. ::: **Returned value** @@ -3722,8 +3686,10 @@ Query: ``` sql SELECT - toDecimal32OrNull(toString(-1.111), 5) AS val, - toTypeName(val) + toDecimal32OrNull(toString(-1.111), 5) AS a, + toTypeName(a), + toDecimal32OrNull(toString('Inf'), 5) as b, + toTypeName(b) FORMAT Vertical; ``` @@ -3732,26 +3698,10 @@ Result: ```response Row 1: ────── -val: -1.111 -toTypeName(val): Nullable(Decimal(9, 5)) -``` - -Query: - -``` sql -SELECT - toDecimal32OrNull(toString(-1.111), 2) AS val, - toTypeName(val) -FORMAT Vertical; -``` - -Result: - -```response -Row 1: -────── -val: -1.11 -toTypeName(val): Nullable(Decimal(9, 2)) +a: -1.111 +toTypeName(a): Nullable(Decimal(9, 5)) +b: ᴺᵁᴸᴸ +toTypeName(b): Nullable(Decimal(9, 5)) ``` **See also** @@ -3773,21 +3723,21 @@ toDecimal32OrDefault(expr, S[, default]) **Arguments** - `expr` — A String representation of a number. [String](../data-types/string.md). -- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). -- `default` (optional) — The default value to return if parsing to type `Decimal32(S)` is unsuccessful. [Decimal32(S](../data-types/decimal.md). +- `S` — Scale parameter between 0 and 9, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Decimal32(S)` is unsuccessful. [Decimal32(S)](../data-types/decimal.md). Supported arguments: - String representations of type (U)Int8/16/32/64/128/256. - String representations of type Float32/64. Unsupported arguments: -- String representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of Float32/64 values `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32OrDefault('0xc0fe', 1);`. :::note -Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. Excessive digits in a fraction are discarded (not rounded). -Excessive digits in the integer part will lead to an exception. +Excessive digits in the integer part will lead to an error. ::: **Returned value** @@ -3800,8 +3750,10 @@ Query: ``` sql SELECT - toDecimal32OrDefault(toString(-1.111), 5) AS val, - toTypeName(val) + toDecimal32OrDefault(toString(0.0001), 5) AS a, + toTypeName(a), + toDecimal32OrDefault('Inf', 0, CAST('-1', 'Decimal32(0)')) AS b, + toTypeName(b) FORMAT Vertical; ``` @@ -3810,16 +3762,125 @@ Result: ```response Row 1: ────── -val: -1.111 -toTypeName(val): Decimal(9, 5) +a: 0.0001 +toTypeName(a): Decimal(9, 5) +b: -1 +toTypeName(b): Decimal(9, 0) ``` +**See also** + +- [`toDecimal32`](#todecimal32). +- [`toDecimal32OrZero`](#todecimal32orzero). +- [`toDecimal32OrNull`](#todecimal32ornull). + +## toDecimal64 + +Converts an input value to a value of type [`Decimal(18, S)`](../data-types/decimal.md) with scale of `S`. Throws an exception in case of an error. + +**Syntax** + +```sql +toDecimal64(expr, S) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `S` — Scale parameter between 0 and 18, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- Values or string representations of type (U)Int8/16/32/64/128/256. +- Values or string representations of type Float32/64. + +Unsupported arguments: +- Values or string representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal64('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal64`: `( -1 * 10^(18 - S), 1 * 10^(18 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an exception. +::: + +**Returned value** + +- Value of type `Decimal(18, S)`. [Decimal64(S)](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT + toDecimal64(2, 1) AS a, toTypeName(a) AS type_a, + toDecimal64(4.2, 2) AS b, toTypeName(b) AS type_b, + toDecimal64('4.2', 3) AS c, toTypeName(c) AS type_c +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 2 +type_a: Decimal(18, 1) +b: 4.2 +type_b: Decimal(18, 2) +c: 4.2 +type_c: Decimal(18, 3) +``` + +**See also** + +- [`toDecimal64OrZero`](#todecimal64orzero). +- [`toDecimal64OrNull`](#todecimal64ornull). +- [`toDecimal64OrDefault`](#todecimal64ordefault). + +## toDecimal64OrZero + +Like [`toDecimal64`](#todecimal64), this function converts an input value to a value of type [Decimal(18, S)](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal64OrZero(expr, S) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 18, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal64OrZero('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal64`: `( -1 * 10^(18 - S), 1 * 10^(18 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Decimal(18, S)` if successful, otherwise `0` with `S` decimal places. [Decimal64(S)](../data-types/decimal.md). + +**Example** + Query: ``` sql SELECT - toDecimal32OrDefault(toString(-1.111), 2) AS val, - toTypeName(val) + toDecimal64OrZero(toString(0.0001), 18) AS a, + toTypeName(a), + toDecimal64OrZero(toString('Inf'), 18) as b, + toTypeName(b) FORMAT Vertical; ``` @@ -3828,16 +3889,61 @@ Result: ```response Row 1: ────── -val: -1.11 -toTypeName(val): Decimal(9, 2) +a: 0.0001 +toTypeName(a): Decimal(18, 18) +b: 0 +toTypeName(b): Decimal(18, 18) ``` +**See also** + +- [`toDecimal64`](#todecimal64). +- [`toDecimal64OrNull`](#todecimal64ornull). +- [`toDecimal64OrDefault`](#todecimal64ordefault). + +## toDecimal64OrNull + +Like [`toDecimal64`](#todecimal64), this function converts an input value to a value of type [Nullable(Decimal(18, S))](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal64OrNull(expr, S) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 18, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal64OrNull('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal64`: `( -1 * 10^(18 - S), 1 * 10^(18 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Nullable(Decimal(18, S))` if successful, otherwise value `NULL` of the same type. [Decimal64(S)](../data-types/decimal.md). + +**Examples** + Query: ``` sql SELECT - toDecimal32OrDefault('Inf', 2, CAST('0', 'Decimal32(2)')) AS val, - toTypeName(val) + toDecimal64OrNull(toString(0.0001), 18) AS a, + toTypeName(a), + toDecimal64OrNull(toString('Inf'), 18) as b, + toTypeName(b) FORMAT Vertical; ``` @@ -3846,10 +3952,590 @@ Result: ```response Row 1: ────── -val: 0 -toTypeName(val): Decimal(9, 2) +a: 0.0001 +toTypeName(a): Nullable(Decimal(18, 18)) +b: ᴺᵁᴸᴸ +toTypeName(b): Nullable(Decimal(18, 18)) ``` +**See also** + +- [`toDecimal64`](#todecimal64). +- [`toDecimal64OrZero`](#todecimal64orzero). +- [`toDecimal64OrDefault`](#todecimal64ordefault). + +## toDecimal64OrDefault + +Like [`toDecimal64`](#todecimal64), this function converts an input value to a value of type [Decimal(18, S)](../data-types/decimal.md) but returns the default value in case of an error. + +**Syntax** + +```sql +toDecimal64OrDefault(expr, S[, default]) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 18, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Decimal64(S)` is unsuccessful. [Decimal64(S)](../data-types/decimal.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal64OrDefault('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal64`: `( -1 * 10^(18 - S), 1 * 10^(18 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Decimal(18, S)` if successful, otherwise returns the default value if passed or `0` if not. [Decimal64(S)](../data-types/decimal.md). + +**Examples** + +Query: + +``` sql +SELECT + toDecimal64OrDefault(toString(0.0001), 18) AS a, + toTypeName(a), + toDecimal64OrDefault('Inf', 0, CAST('-1', 'Decimal64(0)')) AS b, + toTypeName(b) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 0.0001 +toTypeName(a): Decimal(18, 18) +b: -1 +toTypeName(b): Decimal(18, 0) +``` + +**See also** + +- [`toDecimal64`](#todecimal64). +- [`toDecimal64OrZero`](#todecimal64orzero). +- [`toDecimal64OrNull`](#todecimal64ornull). + +## toDecimal128 + +Converts an input value to a value of type [`Decimal(38, S)`](../data-types/decimal.md) with scale of `S`. Throws an exception in case of an error. + +**Syntax** + +```sql +toDecimal128(expr, S) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `S` — Scale parameter between 0 and 38, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- Values or string representations of type (U)Int8/16/32/64/128/256. +- Values or string representations of type Float32/64. + +Unsupported arguments: +- Values or string representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal128('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal128`: `( -1 * 10^(38 - S), 1 * 10^(38 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an exception. +::: + +**Returned value** + +- Value of type `Decimal(38, S)`. [Decimal128(S)](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT + toDecimal128(99, 1) AS a, toTypeName(a) AS type_a, + toDecimal128(99.67, 2) AS b, toTypeName(b) AS type_b, + toDecimal128('99.67', 3) AS c, toTypeName(c) AS type_c +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 99 +type_a: Decimal(38, 1) +b: 99.67 +type_b: Decimal(38, 2) +c: 99.67 +type_c: Decimal(38, 3) +``` + +**See also** + +- [`toDecimal128OrZero`](#todecimal128orzero). +- [`toDecimal128OrNull`](#todecimal128ornull). +- [`toDecimal128OrDefault`](#todecimal128ordefault). + +## toDecimal128OrZero + +Like [`toDecimal128`](#todecimal128), this function converts an input value to a value of type [Decimal(38, S)](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal128OrZero(expr, S) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 38, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal128OrZero('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal128`: `( -1 * 10^(38 - S), 1 * 10^(38 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Decimal(38, S)` if successful, otherwise `0` with `S` decimal places. [Decimal128(S)](../data-types/decimal.md). + +**Example** + +Query: + +``` sql +SELECT + toDecimal128OrZero(toString(0.0001), 38) AS a, + toTypeName(a), + toDecimal128OrZero(toString('Inf'), 38) as b, + toTypeName(b) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 0.0001 +toTypeName(a): Decimal(38, 38) +b: 0 +toTypeName(b): Decimal(38, 38) +``` + +**See also** + +- [`toDecimal128`](#todecimal128). +- [`toDecimal128OrNull`](#todecimal128ornull). +- [`toDecimal128OrDefault`](#todecimal128ordefault). + +## toDecimal128OrNull + +Like [`toDecimal128`](#todecimal128), this function converts an input value to a value of type [Nullable(Decimal(38, S))](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal128OrNull(expr, S) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 38, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal128OrNull('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal128`: `( -1 * 10^(38 - S), 1 * 10^(38 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Nullable(Decimal(38, S))` if successful, otherwise value `NULL` of the same type. [Decimal128(S)](../data-types/decimal.md). + +**Examples** + +Query: + +``` sql +SELECT + toDecimal128OrNull(toString(1/42), 38) AS a, + toTypeName(a), + toDecimal128OrNull(toString('Inf'), 38) as b, + toTypeName(b) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 0.023809523809523808 +toTypeName(a): Nullable(Decimal(38, 38)) +b: ᴺᵁᴸᴸ +toTypeName(b): Nullable(Decimal(38, 38)) +``` + +**See also** + +- [`toDecimal128`](#todecimal128). +- [`toDecimal128OrZero`](#todecimal128orzero). +- [`toDecimal128OrDefault`](#todecimal128ordefault). + +## toDecimal128OrDefault + +Like [`toDecimal128`](#todecimal128), this function converts an input value to a value of type [Decimal(38, S)](../data-types/decimal.md) but returns the default value in case of an error. + +**Syntax** + +```sql +toDecimal128OrDefault(expr, S[, default]) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 38, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Decimal128(S)` is unsuccessful. [Decimal128(S)](../data-types/decimal.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal128OrDefault('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal128`: `( -1 * 10^(38 - S), 1 * 10^(38 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Decimal(38, S)` if successful, otherwise returns the default value if passed or `0` if not. [Decimal128(S)](../data-types/decimal.md). + +**Examples** + +Query: + +``` sql +SELECT + toDecimal128OrDefault(toString(1/42), 18) AS a, + toTypeName(a), + toDecimal128OrDefault('Inf', 0, CAST('-1', 'Decimal128(0)')) AS b, + toTypeName(b) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 0.023809523809523808 +toTypeName(a): Decimal(38, 18) +b: -1 +toTypeName(b): Decimal(38, 0) +``` + +**See also** + +- [`toDecimal128`](#todecimal128). +- [`toDecimal128OrZero`](#todecimal128orzero). +- [`toDecimal128OrNull`](#todecimal128ornull). + +## toDecimal256 + +Converts an input value to a value of type [`Decimal(76, S)`](../data-types/decimal.md) with scale of `S`. Throws an exception in case of an error. + +**Syntax** + +```sql +toDecimal256(expr, S) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `S` — Scale parameter between 0 and 76, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- Values or string representations of type (U)Int8/16/32/64/128/256. +- Values or string representations of type Float32/64. + +Unsupported arguments: +- Values or string representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal256('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal256`: `( -1 * 10^(76 - S), 1 * 10^(76 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an exception. +::: + +**Returned value** + +- Value of type `Decimal(76, S)`. [Decimal256(S)](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT + toDecimal256(99, 1) AS a, toTypeName(a) AS type_a, + toDecimal256(99.67, 2) AS b, toTypeName(b) AS type_b, + toDecimal256('99.67', 3) AS c, toTypeName(c) AS type_c +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 99 +type_a: Decimal(76, 1) +b: 99.67 +type_b: Decimal(76, 2) +c: 99.67 +type_c: Decimal(76, 3) +``` + +**See also** + +- [`toDecimal256OrZero`](#todecimal256orzero). +- [`toDecimal256OrNull`](#todecimal256ornull). +- [`toDecimal256OrDefault`](#todecimal256ordefault). + +## toDecimal256OrZero + +Like [`toDecimal256`](#todecimal256), this function converts an input value to a value of type [Decimal(76, S)](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal256OrZero(expr, S) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 76, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal256OrZero('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal256`: `( -1 * 10^(76 - S), 1 * 10^(76 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Decimal(76, S)` if successful, otherwise `0` with `S` decimal places. [Decimal256(S)](../data-types/decimal.md). + +**Example** + +Query: + +``` sql +SELECT + toDecimal256OrZero(toString(0.0001), 76) AS a, + toTypeName(a), + toDecimal256OrZero(toString('Inf'), 76) as b, + toTypeName(b) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 0.0001 +toTypeName(a): Decimal(76, 76) +b: 0 +toTypeName(b): Decimal(76, 76) +``` + +**See also** + +- [`toDecimal256`](#todecimal256). +- [`toDecimal256OrNull`](#todecimal256ornull). +- [`toDecimal256OrDefault`](#todecimal256ordefault). + +## toDecimal256OrNull + +Like [`toDecimal256`](#todecimal256), this function converts an input value to a value of type [Nullable(Decimal(76, S))](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal256OrNull(expr, S) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 76, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal256OrNull('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal256`: `( -1 * 10^(76 - S), 1 * 10^(76 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Nullable(Decimal(76, S))` if successful, otherwise value `NULL` of the same type. [Decimal256(S)](../data-types/decimal.md). + +**Examples** + +Query: + +``` sql +SELECT + toDecimal256OrNull(toString(1/42), 76) AS a, + toTypeName(a), + toDecimal256OrNull(toString('Inf'), 76) as b, + toTypeName(b) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 0.023809523809523808 +toTypeName(a): Nullable(Decimal(76, 76)) +b: ᴺᵁᴸᴸ +toTypeName(b): Nullable(Decimal(76, 76)) +``` + +**See also** + +- [`toDecimal256`](#todecimal256). +- [`toDecimal256OrZero`](#todecimal256orzero). +- [`toDecimal256OrDefault`](#todecimal256ordefault). + +## toDecimal256OrDefault + +Like [`toDecimal256`](#todecimal256), this function converts an input value to a value of type [Decimal(76, S)](../data-types/decimal.md) but returns the default value in case of an error. + +**Syntax** + +```sql +toDecimal256OrDefault(expr, S[, default]) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 76, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Decimal256(S)` is unsuccessful. [Decimal256(S)](../data-types/decimal.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal256OrDefault('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal256`: `( -1 * 10^(76 - S), 1 * 10^(76 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Decimal(76, S)` if successful, otherwise returns the default value if passed or `0` if not. [Decimal256(S)](../data-types/decimal.md). + +**Examples** + +Query: + +``` sql +SELECT + toDecimal256OrDefault(toString(1/42), 76) AS a, + toTypeName(a), + toDecimal256OrDefault('Inf', 0, CAST('-1', 'Decimal256(0)')) AS b, + toTypeName(b) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 0.023809523809523808 +toTypeName(a): Decimal(76, 76) +b: -1 +toTypeName(b): Decimal(76, 0) +``` + +**See also** + +- [`toDecimal256`](#todecimal256). +- [`toDecimal256OrZero`](#todecimal256orzero). +- [`toDecimal256OrNull`](#todecimal256ornull). + ## toString Functions for converting between numbers, strings (but not fixed strings), dates, and dates with times. From d243feea2136bbfa5f1e943f64e5ebd851f2b103 Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 10 Aug 2024 11:21:28 +0000 Subject: [PATCH 536/644] Fix special builds --- src/Columns/ColumnDynamic.cpp | 8 ++++---- src/Columns/ColumnDynamic.h | 2 +- src/Columns/ColumnVariant.cpp | 2 +- src/Columns/ColumnVariant.h | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 454f7956f48..7246be29592 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -325,7 +325,7 @@ void ColumnDynamic::doInsertFrom(const IColumn & src_, size_t n) /// Check if we insert from shared variant and process it separately. if (src_global_discr == dynamic_src.getSharedVariantDiscriminator()) { - auto & src_shared_variant = dynamic_src.getSharedVariant(); + const auto & src_shared_variant = dynamic_src.getSharedVariant(); auto value = src_shared_variant.getDataAt(src_offset); /// Decode data type of this value. ReadBufferFromMemory buf(value.data, value.size); @@ -469,9 +469,9 @@ void ColumnDynamic::doInsertRangeFrom(const IColumn & src_, size_t start, size_t if (variant_info.variant_names.size() - 1 == max_dynamic_types) { auto shared_variant_discr = getSharedVariantDiscriminator(); - for (size_t i = 0; i != dynamic_src.variant_info.variant_names.size(); ++i) + for (const auto & variant_name : dynamic_src.variant_info.variant_names) { - auto it = variant_info.variant_name_to_discriminator.find(dynamic_src.variant_info.variant_names[i]); + auto it = variant_info.variant_name_to_discriminator.find(variant_name); if (it == variant_info.variant_name_to_discriminator.end()) other_to_new_discriminators.push_back(shared_variant_discr); else @@ -618,7 +618,7 @@ void ColumnDynamic::doInsertManyFrom(const IColumn & src_, size_t position, size /// Check if we insert from shared variant and process it separately. if (src_global_discr == dynamic_src.getSharedVariantDiscriminator()) { - auto & src_shared_variant = dynamic_src.getSharedVariant(); + const auto & src_shared_variant = dynamic_src.getSharedVariant(); auto value = src_shared_variant.getDataAt(src_offset); /// Decode data type of this value. ReadBufferFromMemory buf(value.data, value.size); diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 8b815e2b015..a595a990964 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -37,7 +37,7 @@ public: MERGE, /// Statistics were calculated during merge of several MergeTree parts. }; - Statistics(Source source_) : source(source_) {} + explicit Statistics(Source source_) : source(source_) {} /// Source of the statistics. Source source; diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index 7531e976926..0402e1a0690 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -476,7 +476,7 @@ void ColumnVariant::insertFromImpl(const DB::IColumn & src_, size_t n, const std } } -void ColumnVariant::insertRangeFromImpl(const DB::IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping, Discriminator * skip_discriminator) +void ColumnVariant::insertRangeFromImpl(const DB::IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping, const Discriminator * skip_discriminator) { const size_t num_variants = variants.size(); const auto & src = assert_cast(src_); diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 571a843d113..7c8093e385d 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -329,7 +329,7 @@ public: private: void insertFromImpl(const IColumn & src_, size_t n, const std::vector * global_discriminators_mapping); - void insertRangeFromImpl(const IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping, Discriminator * skip_discriminator); + void insertRangeFromImpl(const IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping, const Discriminator * skip_discriminator); void insertManyFromImpl(const IColumn & src_, size_t position, size_t length, const std::vector * global_discriminators_mapping); void initIdentityGlobalToLocalDiscriminatorsMapping(); From c1b5b908ba13f863de4e9621d3fe2c8139758650 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 10 Aug 2024 13:01:55 +0000 Subject: [PATCH 537/644] hide Field::get in private, only use Field::safeGet --- programs/client/Client.cpp | 2 +- programs/keeper-client/Commands.cpp | 6 +- .../AggregateFunctionGroupArray.cpp | 12 +- .../AggregateFunctionGroupArrayIntersect.cpp | 6 +- .../AggregateFunctionGroupArrayMoving.cpp | 6 +- .../AggregateFunctionGroupArraySorted.cpp | 6 +- .../AggregateFunctionGroupConcat.cpp | 10 +- .../AggregateFunctionGroupUniqArray.cpp | 6 +- ...AggregateFunctionKolmogorovSmirnovTest.cpp | 4 +- ...ateFunctionLargestTriangleThreeBuckets.cpp | 2 +- .../AggregateFunctionMannWhitney.cpp | 4 +- .../AggregateFunctionQuantile.h | 6 +- .../AggregateFunctionSumMap.cpp | 10 +- .../Passes/ComparisonTupleEliminationPass.cpp | 2 +- .../Passes/ConvertOrLikeChainPass.cpp | 2 +- .../Passes/FunctionToSubcolumnsPass.cpp | 6 +- src/Analyzer/Passes/FuseFunctionsPass.cpp | 2 +- .../Passes/IfTransformStringsToEnumPass.cpp | 10 +- .../Passes/NormalizeCountVariantsPass.cpp | 2 +- ...ateOrDateTimeConverterWithPreimagePass.cpp | 8 +- .../RewriteAggregateFunctionWithIfPass.cpp | 4 +- src/Analyzer/Passes/SumIfToCountIfPass.cpp | 6 +- src/Analyzer/QueryTreeBuilder.cpp | 2 +- src/Analyzer/Resolve/IdentifierResolver.cpp | 8 +- src/Analyzer/Resolve/QueryAnalyzer.cpp | 4 +- src/Analyzer/SetUtils.cpp | 8 +- src/Backups/BackupSettings.cpp | 4 +- src/Backups/DDLAdjustingForBackupVisitor.cpp | 4 +- src/Backups/RestoreSettings.cpp | 8 +- src/Backups/SettingsFieldOptionalString.cpp | 2 +- src/Backups/SettingsFieldOptionalUUID.cpp | 2 +- src/Client/QueryFuzzer.cpp | 20 +-- src/Client/Suggest.cpp | 2 +- src/Columns/ColumnAggregateFunction.cpp | 19 +-- src/Columns/ColumnArray.cpp | 6 +- src/Columns/ColumnDecimal.h | 2 +- src/Columns/ColumnFixedString.cpp | 4 +- src/Columns/ColumnMap.cpp | 10 +- src/Columns/ColumnObject.cpp | 4 +- src/Columns/ColumnString.h | 2 +- src/Columns/ColumnTuple.cpp | 6 +- src/Columns/ColumnVector.h | 4 +- src/Columns/tests/gtest_column_variant.cpp | 120 +++++++++--------- src/Columns/tests/gtest_low_cardinality.cpp | 6 +- src/Common/CollectionOfDerived.h | 2 +- src/Common/FieldBinaryEncoding.cpp | 2 +- src/Common/FieldVisitorSum.cpp | 2 +- src/Common/FieldVisitorSum.h | 2 +- src/Common/FieldVisitorToString.cpp | 2 +- src/Common/HashTable/HashMap.h | 2 +- src/Common/examples/arena_with_free_lists.cpp | 22 ++-- src/Core/ExternalTable.cpp | 2 +- src/Core/Field.h | 71 ++++------- src/Core/Range.cpp | 16 +-- src/Core/Settings.cpp | 2 +- src/Core/SettingsFields.cpp | 14 +- src/Core/SettingsQuirks.cpp | 6 +- src/Core/examples/field.cpp | 2 +- src/Core/tests/gtest_field.cpp | 28 ++-- src/DataTypes/DataTypeAggregateFunction.cpp | 4 +- src/DataTypes/DataTypeDynamic.cpp | 4 +- src/DataTypes/DataTypeEnum.cpp | 18 +-- src/DataTypes/DataTypeFixedString.cpp | 4 +- src/DataTypes/DataTypeObject.cpp | 2 +- src/DataTypes/DataTypesDecimal.cpp | 6 +- src/DataTypes/ObjectUtils.cpp | 2 +- .../Serializations/JSONDataParser.cpp | 4 +- .../SerializationAggregateFunction.cpp | 4 +- .../Serializations/SerializationArray.cpp | 4 +- .../SerializationDecimalBase.cpp | 2 +- .../SerializationFixedString.cpp | 4 +- .../SerializationIPv4andIPv6.cpp | 2 +- .../Serializations/SerializationMap.cpp | 4 +- .../Serializations/SerializationNumber.cpp | 2 +- .../Serializations/SerializationString.cpp | 4 +- .../Serializations/SerializationTuple.cpp | 4 +- .../Serializations/SerializationUUID.cpp | 2 +- src/DataTypes/registerDataTypeDateTime.cpp | 2 +- src/Databases/DDLLoadingDependencyVisitor.cpp | 4 +- src/Databases/DDLRenamingVisitor.cpp | 6 +- src/Databases/DatabaseReplicated.cpp | 4 +- .../MySQL/MaterializedMySQLSyncThread.cpp | 18 +-- .../MySQL/tests/gtest_mysql_binlog.cpp | 30 ++--- src/Dictionaries/CacheDictionaryStorage.h | 12 +- src/Dictionaries/DictionaryHelpers.h | 4 +- src/Dictionaries/FlatDictionary.cpp | 6 +- src/Dictionaries/HashedArrayDictionary.cpp | 8 +- src/Dictionaries/HashedDictionary.h | 8 +- .../HierarchyDictionariesUtils.cpp | 6 +- src/Dictionaries/IPAddressDictionary.cpp | 8 +- src/Dictionaries/MongoDBDictionarySource.cpp | 3 +- src/Dictionaries/PolygonDictionary.cpp | 10 +- src/Dictionaries/RangeHashedDictionary.h | 4 +- src/Dictionaries/RedisDictionarySource.cpp | 3 +- src/Disks/getOrCreateDiskFromAST.cpp | 2 +- src/Functions/DateTimeTransforms.h | 4 +- src/Functions/FunctionsConsistentHashing.h | 4 +- src/Functions/FunctionsConversion.cpp | 30 ++--- src/Functions/FunctionsJSON.cpp | 2 +- src/Functions/FunctionsLogical.cpp | 6 +- src/Functions/FunctionsRound.h | 6 +- src/Functions/IFunctionCustomWeek.h | 8 +- src/Functions/IFunctionDateOrDateTime.h | 16 +-- .../JSONPath/Parsers/ParserJSONPathRange.cpp | 4 +- src/Functions/MultiMatchAllIndicesImpl.h | 2 +- src/Functions/MultiMatchAnyImpl.h | 2 +- src/Functions/MultiSearchAllPositionsImpl.h | 2 +- src/Functions/MultiSearchFirstIndexImpl.h | 2 +- src/Functions/MultiSearchFirstPositionImpl.h | 2 +- src/Functions/MultiSearchImpl.h | 2 +- src/Functions/URL/cutURLParameter.cpp | 2 +- src/Functions/array/arrayElement.cpp | 38 +++--- src/Functions/array/mapOp.cpp | 2 +- src/Functions/getClientHTTPHeader.cpp | 2 +- src/Functions/multiIf.cpp | 2 +- src/Functions/nested.cpp | 4 +- src/IO/S3Common.cpp | 8 +- src/Interpreters/ActionsVisitor.cpp | 12 +- src/Interpreters/AddDefaultDatabaseVisitor.h | 2 +- .../ComparisonTupleEliminationVisitor.cpp | 2 +- .../ConvertFunctionOrLikeVisitor.cpp | 4 +- .../ConvertStringsToEnumVisitor.cpp | 12 +- src/Interpreters/DDLTask.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 4 +- src/Interpreters/InterpreterExplainQuery.cpp | 4 +- .../InterpreterKillQueryQuery.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/MutationsInterpreter.cpp | 2 +- ...OrDateTimeConverterWithPreimageVisitor.cpp | 8 +- ...OptimizeIfWithConstantConditionVisitor.cpp | 4 +- .../OptimizeShardingKeyRewriteInVisitor.cpp | 21 ++- .../RewriteCountVariantsVisitor.cpp | 2 +- src/Interpreters/TreeOptimizer.cpp | 2 +- src/Interpreters/WindowDescription.cpp | 8 +- src/Interpreters/convertFieldToType.cpp | 62 ++++----- .../evaluateConstantExpression.cpp | 4 +- .../replaceForPositionalArguments.cpp | 4 +- .../tests/gtest_comparison_graph.cpp | 4 +- src/Parsers/ASTFunction.cpp | 2 +- src/Parsers/ASTLiteral.cpp | 8 +- src/Parsers/Access/ParserCreateQuotaQuery.cpp | 2 +- src/Parsers/ExpressionElementParsers.cpp | 10 +- src/Parsers/ParserAlterQuery.cpp | 12 +- src/Parsers/ParserCheckQuery.cpp | 2 +- src/Parsers/ParserCreateQuery.cpp | 4 +- src/Parsers/ParserDictionary.cpp | 6 +- src/Parsers/ParserPartition.cpp | 2 +- src/Parsers/ParserSystemQuery.cpp | 8 +- src/Parsers/ParserUndropQuery.cpp | 2 +- src/Parsers/tests/gtest_dictionary_parser.cpp | 40 +++--- .../Formats/Impl/CHColumnToArrowColumn.cpp | 2 +- .../Impl/ConstantExpressionTemplate.cpp | 6 +- .../Impl/NativeORCBlockInputFormat.cpp | 4 +- .../Impl/PrometheusTextOutputFormat.cpp | 4 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 8 +- .../Algorithms/SummingSortedAlgorithm.cpp | 12 +- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 10 +- src/Processors/Sources/MySQLSource.cpp | 4 +- .../Transforms/FillingTransform.cpp | 14 +- src/Processors/Transforms/WindowTransform.cpp | 20 +-- src/Storages/AlterCommands.cpp | 8 +- src/Storages/ColumnsDescription.cpp | 2 +- ...pproximateNearestNeighborIndexesCommon.cpp | 12 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 8 +- src/Storages/MergeTree/KeyCondition.cpp | 12 +- src/Storages/MergeTree/MergeTreeData.cpp | 8 +- .../MergeTree/MergeTreeDataWriter.cpp | 4 +- .../MergeTree/MergeTreeIndexAnnoy.cpp | 6 +- .../MergeTree/MergeTreeIndexBloomFilter.cpp | 12 +- .../MergeTreeIndexBloomFilterText.cpp | 50 ++++---- .../MergeTree/MergeTreeIndexFullText.cpp | 42 +++--- .../MergeTree/MergeTreeIndexHypothesis.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 4 +- .../MergeTree/MergeTreeIndexUSearch.cpp | 8 +- src/Storages/MergeTree/MergeTreePartition.cpp | 2 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 6 +- .../MergeTree/registerStorageMergeTree.cpp | 2 +- src/Storages/Statistics/Statistics.cpp | 16 +-- .../Statistics/StatisticsCountMinSketch.cpp | 2 +- src/Storages/StorageFactory.cpp | 2 +- src/Storages/StorageFile.cpp | 6 +- src/Storages/StorageFuzzJSON.cpp | 2 +- src/Storages/StorageJoin.cpp | 4 +- src/Storages/StoragePostgreSQL.cpp | 4 +- src/Storages/System/StorageSystemColumns.cpp | 6 +- .../System/StorageSystemPartsBase.cpp | 2 +- src/Storages/System/StorageSystemPartsBase.h | 10 +- src/Storages/VirtualColumnUtils.h | 2 +- src/Storages/getStructureOfRemoteTable.cpp | 12 +- src/TableFunctions/TableFunctionExplain.cpp | 4 +- src/TableFunctions/TableFunctionFile.cpp | 2 +- .../TableFunctionMergeTreeIndex.cpp | 4 +- 192 files changed, 749 insertions(+), 774 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 1d99d223ee9..25c94c56aa6 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -223,7 +223,7 @@ std::vector Client::loadWarningMessages() size_t rows = packet.block.rows(); for (size_t i = 0; i < rows; ++i) - messages.emplace_back(column[i].get()); + messages.emplace_back(column[i].safeGet()); } continue; diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index df9da8e9613..7226bd82df7 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -95,7 +95,7 @@ void SetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co client->zookeeper->set( client->getAbsolutePath(query->args[0].safeGet()), query->args[1].safeGet(), - static_cast(query->args[2].get())); + static_cast(query->args[2].safeGet())); } bool CreateCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const @@ -494,7 +494,7 @@ void RMCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con { Int32 version{-1}; if (query->args.size() == 2) - version = static_cast(query->args[1].get()); + version = static_cast(query->args[1].safeGet()); client->zookeeper->remove(client->getAbsolutePath(query->args[0].safeGet()), version); } @@ -549,7 +549,7 @@ void ReconfigCommand::execute(const DB::ASTKeeperQuery * query, DB::KeeperClient String leaving; String new_members; - auto operation = query->args[0].get(); + auto operation = query->args[0].safeGet(); switch (operation) { case static_cast(ReconfigCommand::Operation::ADD): diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index 7034e6373b1..5cc9f725b46 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -780,12 +780,12 @@ AggregateFunctionPtr createAggregateFunctionGroupArray( if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); - if ((type == Field::Types::Int64 && parameters[0].get() < 0) || - (type == Field::Types::UInt64 && parameters[0].get() == 0)) + if ((type == Field::Types::Int64 && parameters[0].safeGet() < 0) || + (type == Field::Types::UInt64 && parameters[0].safeGet() == 0)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); has_limit = true; - max_elems = parameters[0].get(); + max_elems = parameters[0].safeGet(); } else throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, @@ -816,11 +816,11 @@ AggregateFunctionPtr createAggregateFunctionGroupArraySample( if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); - if ((type == Field::Types::Int64 && parameters[i].get() < 0) || - (type == Field::Types::UInt64 && parameters[i].get() == 0)) + if ((type == Field::Types::Int64 && parameters[i].safeGet() < 0) || + (type == Field::Types::UInt64 && parameters[i].safeGet() == 0)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); - return parameters[i].get(); + return parameters[i].safeGet(); }; UInt64 max_elems = get_parameter(0); diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp index 38f2fcb9fb9..36d00b1d9ec 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp @@ -83,16 +83,16 @@ public: if (version == 1) { for (size_t i = 0; i < arr_size; ++i) - set.insert(static_cast((*data_column)[offset + i].get())); + set.insert(static_cast((*data_column)[offset + i].safeGet())); } else if (!set.empty()) { typename State::Set new_set; for (size_t i = 0; i < arr_size; ++i) { - typename State::Set::LookupResult set_value = set.find(static_cast((*data_column)[offset + i].get())); + typename State::Set::LookupResult set_value = set.find(static_cast((*data_column)[offset + i].safeGet())); if (set_value != nullptr) - new_set.insert(static_cast((*data_column)[offset + i].get())); + new_set.insert(static_cast((*data_column)[offset + i].safeGet())); } set = std::move(new_set); } diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp index 026b8d1956f..2c3ac7f883e 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp @@ -269,12 +269,12 @@ AggregateFunctionPtr createAggregateFunctionMoving( if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name); - if ((type == Field::Types::Int64 && parameters[0].get() <= 0) || - (type == Field::Types::UInt64 && parameters[0].get() == 0)) + if ((type == Field::Types::Int64 && parameters[0].safeGet() <= 0) || + (type == Field::Types::UInt64 && parameters[0].safeGet() == 0)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name); limit_size = true; - max_elems = parameters[0].get(); + max_elems = parameters[0].safeGet(); } else throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, diff --git a/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp b/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp index d41d743e17a..27043ed6aa6 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp @@ -397,11 +397,11 @@ AggregateFunctionPtr createAggregateFunctionGroupArray( if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); - if ((type == Field::Types::Int64 && parameters[0].get() < 0) || - (type == Field::Types::UInt64 && parameters[0].get() == 0)) + if ((type == Field::Types::Int64 && parameters[0].safeGet() < 0) || + (type == Field::Types::UInt64 && parameters[0].safeGet() == 0)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); - max_elems = parameters[0].get(); + max_elems = parameters[0].safeGet(); } else throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, diff --git a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp index 5494ef74705..636ac80e350 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp @@ -247,7 +247,7 @@ AggregateFunctionPtr createAggregateFunctionGroupConcat( if (type != Field::Types::String) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First parameter for aggregate function {} should be string", name); - delimiter = parameters[0].get(); + delimiter = parameters[0].safeGet(); } if (parameters.size() == 2) { @@ -256,12 +256,12 @@ AggregateFunctionPtr createAggregateFunctionGroupConcat( if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number", name); - if ((type == Field::Types::Int64 && parameters[1].get() <= 0) || - (type == Field::Types::UInt64 && parameters[1].get() == 0)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number, got: {}", name, parameters[1].get()); + if ((type == Field::Types::Int64 && parameters[1].safeGet() <= 0) || + (type == Field::Types::UInt64 && parameters[1].safeGet() == 0)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number, got: {}", name, parameters[1].safeGet()); has_limit = true; - limit = parameters[1].get(); + limit = parameters[1].safeGet(); } if (has_limit) diff --git a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp index 7b4300b3568..5cbf449c946 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp @@ -323,12 +323,12 @@ AggregateFunctionPtr createAggregateFunctionGroupUniqArray( if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); - if ((type == Field::Types::Int64 && parameters[0].get() < 0) || - (type == Field::Types::UInt64 && parameters[0].get() == 0)) + if ((type == Field::Types::Int64 && parameters[0].safeGet() < 0) || + (type == Field::Types::UInt64 && parameters[0].safeGet() == 0)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); limit_size = true; - max_elems = parameters[0].get(); + max_elems = parameters[0].safeGet(); } else throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, diff --git a/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp b/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp index 04eebe9f485..28e8d37b8c8 100644 --- a/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp +++ b/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp @@ -238,7 +238,7 @@ public: if (params[0].getType() != Field::Types::String) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a String", getName()); - const auto & param = params[0].get(); + const auto & param = params[0].safeGet(); if (param == "two-sided") alternative = Alternative::TwoSided; else if (param == "less") @@ -255,7 +255,7 @@ public: if (params[1].getType() != Field::Types::String) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require second parameter to be a String", getName()); - method = params[1].get(); + method = params[1].safeGet(); if (method != "auto" && method != "exact" && method != "asymp" && method != "asymptotic") throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown method in aggregate function {}. " "It must be one of: 'auto', 'exact', 'asymp' (or 'asymptotic')", getName()); diff --git a/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp index 6d1e3c0f64b..813b13b6f7b 100644 --- a/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp +++ b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp @@ -181,7 +181,7 @@ public: throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a UInt64", getName()); - total_buckets = params[0].get(); + total_buckets = params[0].safeGet(); this->x_type = WhichDataType(arguments[0]).idx; this->y_type = WhichDataType(arguments[1]).idx; diff --git a/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp b/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp index f088737c340..fa90846650d 100644 --- a/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp +++ b/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp @@ -152,7 +152,7 @@ public: if (params[0].getType() != Field::Types::String) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a String", getName()); - const auto & param = params[0].get(); + const auto & param = params[0].safeGet(); if (param == "two-sided") alternative = Alternative::TwoSided; else if (param == "less") @@ -169,7 +169,7 @@ public: if (params[1].getType() != Field::Types::UInt64) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require second parameter to be a UInt64", getName()); - continuity_correction = static_cast(params[1].get()); + continuity_correction = static_cast(params[1].safeGet()); } String getName() const override diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.h b/src/AggregateFunctions/AggregateFunctionQuantile.h index 127dc06b642..423fd4bc569 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -117,7 +117,7 @@ public: throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} requires relative accuracy parameter with Float64 type", getName()); - relative_accuracy = relative_accuracy_field.get(); + relative_accuracy = relative_accuracy_field.safeGet(); if (relative_accuracy <= 0 || relative_accuracy >= 1 || isNaN(relative_accuracy)) throw Exception( @@ -147,9 +147,9 @@ public: ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} requires accuracy parameter with integer type", getName()); if (accuracy_field.getType() == Field::Types::Int64) - accuracy = accuracy_field.get(); + accuracy = accuracy_field.safeGet(); else - accuracy = accuracy_field.get(); + accuracy = accuracy_field.safeGet(); if (accuracy <= 0) throw Exception( diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 666136a91b7..9a94c3dfe1a 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -300,12 +300,12 @@ public: /// Compatibility with previous versions. if (value.getType() == Field::Types::Decimal32) { - auto source = value.get>(); + auto source = value.safeGet>(); value = DecimalField(source.getValue(), source.getScale()); } else if (value.getType() == Field::Types::Decimal64) { - auto source = value.get>(); + auto source = value.safeGet>(); value = DecimalField(source.getValue(), source.getScale()); } @@ -355,7 +355,7 @@ public: /// Compatibility with previous versions. if (value.getType() == Field::Types::Decimal128) { - auto source = value.get>(); + auto source = value.safeGet>(); WhichDataType value_type(values_types[col_idx]); if (value_type.isDecimal32()) { @@ -560,7 +560,7 @@ private: template bool compareImpl(FieldType & x) const { - auto val = rhs.get(); + auto val = rhs.safeGet(); if (val > x) { x = val; @@ -600,7 +600,7 @@ private: template bool compareImpl(FieldType & x) const { - auto val = rhs.get(); + auto val = rhs.safeGet(); if (val < x) { x = val; diff --git a/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp b/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp index 180470952cd..4bc4255e2e2 100644 --- a/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp +++ b/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp @@ -129,7 +129,7 @@ private: if (constant_node_value.getType() != Field::Types::Which::Tuple) return {}; - const auto & constant_tuple = constant_node_value.get(); + const auto & constant_tuple = constant_node_value.safeGet(); const auto & function_arguments_nodes = function_node_typed.getArguments().getNodes(); size_t function_arguments_nodes_size = function_arguments_nodes.size(); diff --git a/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp b/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp index 2b2ac95d7b9..6c4ce789993 100644 --- a/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp +++ b/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp @@ -89,7 +89,7 @@ public: if (!pattern || !isString(pattern->getResultType())) continue; - auto regexp = likePatternToRegexp(pattern->getValue().get()); + auto regexp = likePatternToRegexp(pattern->getValue().safeGet()); /// Case insensitive. Works with UTF-8 as well. if (is_ilike) regexp = "(?i)" + regexp; diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp index b968f43c6a6..1fc3eec6833 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -68,10 +68,10 @@ void optimizeFunctionEmpty(QueryTreeNodePtr &, FunctionNode & function_node, Col String getSubcolumnNameForElement(const Field & value, const DataTypeTuple & data_type_tuple) { if (value.getType() == Field::Types::String) - return value.get(); + return value.safeGet(); if (value.getType() == Field::Types::UInt64) - return data_type_tuple.getNameByPosition(value.get()); + return data_type_tuple.getNameByPosition(value.safeGet()); return ""; } @@ -79,7 +79,7 @@ String getSubcolumnNameForElement(const Field & value, const DataTypeTuple & dat String getSubcolumnNameForElement(const Field & value, const DataTypeVariant &) { if (value.getType() == Field::Types::String) - return value.get(); + return value.safeGet(); return ""; } diff --git a/src/Analyzer/Passes/FuseFunctionsPass.cpp b/src/Analyzer/Passes/FuseFunctionsPass.cpp index 0175e304a2b..f3b109a10ed 100644 --- a/src/Analyzer/Passes/FuseFunctionsPass.cpp +++ b/src/Analyzer/Passes/FuseFunctionsPass.cpp @@ -187,7 +187,7 @@ FunctionNodePtr createFusedQuantilesNode(std::vector & nodes /// Sort nodes and parameters in ascending order of quantile level std::vector permutation(nodes.size()); iota(permutation.data(), permutation.size(), size_t(0)); - std::sort(permutation.begin(), permutation.end(), [&](size_t i, size_t j) { return parameters[i].get() < parameters[j].get(); }); + std::sort(permutation.begin(), permutation.end(), [&](size_t i, size_t j) { return parameters[i].safeGet() < parameters[j].safeGet(); }); std::vector new_nodes; new_nodes.reserve(permutation.size()); diff --git a/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp b/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp index d966f129d08..f81327c5d55 100644 --- a/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp +++ b/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp @@ -134,8 +134,8 @@ public: return; std::set string_values; - string_values.insert(first_literal->getValue().get()); - string_values.insert(second_literal->getValue().get()); + string_values.insert(first_literal->getValue().safeGet()); + string_values.insert(second_literal->getValue().safeGet()); changeIfArguments(*function_if_node, string_values, context); wrapIntoToString(*function_node, std::move(modified_if_node), context); @@ -163,7 +163,7 @@ public: if (!isArray(literal_to->getResultType()) || !isString(literal_default->getResultType())) return; - auto array_to = literal_to->getValue().get(); + auto array_to = literal_to->getValue().safeGet(); if (array_to.empty()) return; @@ -178,9 +178,9 @@ public: std::set string_values; for (const auto & value : array_to) - string_values.insert(value.get()); + string_values.insert(value.safeGet()); - string_values.insert(literal_default->getValue().get()); + string_values.insert(literal_default->getValue().safeGet()); changeTransformArguments(*function_modified_transform_node, string_values, context); wrapIntoToString(*function_node, std::move(modified_transform_node), context); diff --git a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp index 3a8b6e75d40..02f1c93ea7f 100644 --- a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp +++ b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp @@ -54,7 +54,7 @@ public: } else if (function_node->getFunctionName() == "sum" && first_argument_constant_literal.getType() == Field::Types::UInt64 && - first_argument_constant_literal.get() == 1) + first_argument_constant_literal.safeGet() == 1) { function_node->getArguments().getNodes().clear(); resolveAggregateFunctionNodeByName(*function_node, "count"); diff --git a/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp b/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp index feb8bcc792d..0f33c302265 100644 --- a/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp +++ b/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp @@ -143,13 +143,13 @@ private: const auto & column_type = column_node_typed.getColumnType().get(); if (isDateOrDate32(column_type)) { - start_date_or_date_time = date_lut.dateToString(range.first.get()); - end_date_or_date_time = date_lut.dateToString(range.second.get()); + start_date_or_date_time = date_lut.dateToString(range.first.safeGet()); + end_date_or_date_time = date_lut.dateToString(range.second.safeGet()); } else if (isDateTime(column_type) || isDateTime64(column_type)) { - start_date_or_date_time = date_lut.timeToString(range.first.get()); - end_date_or_date_time = date_lut.timeToString(range.second.get()); + start_date_or_date_time = date_lut.timeToString(range.first.safeGet()); + end_date_or_date_time = date_lut.timeToString(range.second.safeGet()); } else [[unlikely]] return {}; diff --git a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp index a48e88132a6..091061ceb81 100644 --- a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp +++ b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp @@ -60,7 +60,7 @@ public: { const auto & second_const_value = second_const_node->getValue(); if (second_const_value.isNull() - || (lower_name == "sum" && isInt64OrUInt64FieldType(second_const_value.getType()) && second_const_value.get() == 0 + || (lower_name == "sum" && isInt64OrUInt64FieldType(second_const_value.getType()) && second_const_value.safeGet() == 0 && !if_node->getResultType()->isNullable())) { /// avg(if(cond, a, null)) -> avgIf(a::ResultTypeIf, cond) @@ -89,7 +89,7 @@ public: { const auto & first_const_value = first_const_node->getValue(); if (first_const_value.isNull() - || (lower_name == "sum" && isInt64OrUInt64FieldType(first_const_value.getType()) && first_const_value.get() == 0 + || (lower_name == "sum" && isInt64OrUInt64FieldType(first_const_value.getType()) && first_const_value.safeGet() == 0 && !if_node->getResultType()->isNullable())) { /// avg(if(cond, null, a) -> avgIf(a::ResultTypeIf, !cond)) diff --git a/src/Analyzer/Passes/SumIfToCountIfPass.cpp b/src/Analyzer/Passes/SumIfToCountIfPass.cpp index 1524629dc81..a987ced497a 100644 --- a/src/Analyzer/Passes/SumIfToCountIfPass.cpp +++ b/src/Analyzer/Passes/SumIfToCountIfPass.cpp @@ -66,7 +66,7 @@ public: resolveAggregateFunctionNodeByName(*function_node, "countIf"); - if (constant_value_literal.get() != 1) + if (constant_value_literal.safeGet() != 1) { /// Rewrite `sumIf(123, cond)` into `123 * countIf(cond)` node = getMultiplyFunction(std::move(multiplier_node), node); @@ -105,8 +105,8 @@ public: const auto & if_true_condition_constant_value_literal = if_true_condition_constant_node->getValue(); const auto & if_false_condition_constant_value_literal = if_false_condition_constant_node->getValue(); - auto if_true_condition_value = if_true_condition_constant_value_literal.get(); - auto if_false_condition_value = if_false_condition_constant_value_literal.get(); + auto if_true_condition_value = if_true_condition_constant_value_literal.safeGet(); + auto if_false_condition_value = if_false_condition_constant_value_literal.safeGet(); if (if_false_condition_value == 0) { diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index fb41826929f..9754897d54d 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -471,7 +471,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_express std::shared_ptr collator; if (order_by_element.getCollation()) - collator = std::make_shared(order_by_element.getCollation()->as().value.get()); + collator = std::make_shared(order_by_element.getCollation()->as().value.safeGet()); const auto & sort_expression_ast = order_by_element.children.at(0); auto sort_expression = buildExpression(sort_expression_ast, context); diff --git a/src/Analyzer/Resolve/IdentifierResolver.cpp b/src/Analyzer/Resolve/IdentifierResolver.cpp index 447bf825836..a79433ac130 100644 --- a/src/Analyzer/Resolve/IdentifierResolver.cpp +++ b/src/Analyzer/Resolve/IdentifierResolver.cpp @@ -1273,7 +1273,7 @@ QueryTreeNodePtr IdentifierResolver::matchArrayJoinSubcolumns( const auto & constant_node_value = constant_node.getValue(); if (constant_node_value.getType() == Field::Types::String) { - array_join_subcolumn_prefix = constant_node_value.get() + "."; + array_join_subcolumn_prefix = constant_node_value.safeGet() + "."; array_join_parent_column = argument_nodes.at(0).get(); } } @@ -1287,7 +1287,7 @@ QueryTreeNodePtr IdentifierResolver::matchArrayJoinSubcolumns( if (!second_argument || second_argument->getValue().getType() != Field::Types::String) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected constant string as second argument of getSubcolumn function {}", resolved_function->dumpTree()); - const auto & resolved_subcolumn_path = second_argument->getValue().get(); + const auto & resolved_subcolumn_path = second_argument->getValue().safeGet(); if (!startsWith(resolved_subcolumn_path, array_join_subcolumn_prefix)) return {}; @@ -1331,7 +1331,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveExpressionFromArrayJoinExpression size_t nested_function_arguments_size = nested_function_arguments.size(); const auto & nested_keys_names_constant_node = nested_function_arguments[0]->as(); - const auto & nested_keys_names = nested_keys_names_constant_node.getValue().get(); + const auto & nested_keys_names = nested_keys_names_constant_node.getValue().safeGet(); size_t nested_keys_names_size = nested_keys_names.size(); if (nested_keys_names_size == nested_function_arguments_size - 1) @@ -1344,7 +1344,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveExpressionFromArrayJoinExpression auto array_join_column = std::make_shared(array_join_column_expression_typed.getColumn(), array_join_column_expression_typed.getColumnSource()); - const auto & nested_key_name = nested_keys_names[i - 1].get(); + const auto & nested_key_name = nested_keys_names[i - 1].safeGet(); Identifier nested_identifier = Identifier(nested_key_name); array_join_resolved_expression = wrapExpressionNodeInTupleElement(array_join_column, nested_identifier, scope.context); break; diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 767d5c11075..2ce79b7bddd 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -746,11 +746,11 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_ UInt64 pos; if (constant_node->getValue().getType() == Field::Types::UInt64) { - pos = constant_node->getValue().get(); + pos = constant_node->getValue().safeGet(); } else // Int64 { - auto value = constant_node->getValue().get(); + auto value = constant_node->getValue().safeGet(); if (value > 0) pos = value; else diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index 0ecb3545225..095be1fb9b8 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -93,7 +93,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes& "Invalid type in set. Expected tuple, got {}", value.getTypeName()); - const auto & tuple = value.template get(); + const auto & tuple = value.template safeGet(); const DataTypePtr & value_type = value_types[collection_index]; const DataTypes & tuple_value_type = typeid_cast(value_type.get())->getElements(); @@ -169,15 +169,15 @@ Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const if (rhs_which_type.isArray()) { const DataTypeArray * value_array_type = assert_cast(value_type.get()); - size_t value_array_size = value.get().size(); + size_t value_array_size = value.safeGet().size(); DataTypes value_types(value_array_size, value_array_type->getNestedType()); - result_block = createBlockFromCollection(value.get(), value_types, set_element_types, transform_null_in); + result_block = createBlockFromCollection(value.safeGet(), value_types, set_element_types, transform_null_in); } else if (rhs_which_type.isTuple()) { const DataTypeTuple * value_tuple_type = assert_cast(value_type.get()); const DataTypes & value_types = value_tuple_type->getElements(); - result_block = createBlockFromCollection(value.get(), value_types, set_element_types, transform_null_in); + result_block = createBlockFromCollection(value.safeGet(), value_types, set_element_types, transform_null_in); } else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index e33880f88e3..7f353028346 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -125,7 +125,7 @@ std::vector BackupSettings::Util::clusterHostIDsFromAST(const IAST & as throw Exception( ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Setting cluster_host_ids has wrong format, must be array of arrays of string literals"); - const auto & replicas = array_of_replicas->value.get(); + const auto & replicas = array_of_replicas->value.safeGet(); res[i].resize(replicas.size()); for (size_t j = 0; j != replicas.size(); ++j) { @@ -134,7 +134,7 @@ std::vector BackupSettings::Util::clusterHostIDsFromAST(const IAST & as throw Exception( ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Setting cluster_host_ids has wrong format, must be array of arrays of string literals"); - res[i][j] = replica.get(); + res[i][j] = replica.safeGet(); } } } diff --git a/src/Backups/DDLAdjustingForBackupVisitor.cpp b/src/Backups/DDLAdjustingForBackupVisitor.cpp index 910831195a3..4dcbdcc1617 100644 --- a/src/Backups/DDLAdjustingForBackupVisitor.cpp +++ b/src/Backups/DDLAdjustingForBackupVisitor.cpp @@ -46,8 +46,8 @@ namespace if (zookeeper_path_ast && (zookeeper_path_ast->value.getType() == Field::Types::String) && replica_name_ast && (replica_name_ast->value.getType() == Field::Types::String)) { - String & zookeeper_path_arg = zookeeper_path_ast->value.get(); - String & replica_name_arg = replica_name_ast->value.get(); + String & zookeeper_path_arg = zookeeper_path_ast->value.safeGet(); + String & replica_name_arg = replica_name_ast->value.safeGet(); if (create.uuid != UUIDHelpers::Nil) { String table_uuid_str = toString(create.uuid); diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 7bbfd9ed751..4662305cdd6 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -31,7 +31,7 @@ namespace { if (field.getType() == Field::Types::String) { - const String & str = field.get(); + const String & str = field.safeGet(); if (str == "1" || boost::iequals(str, "true") || boost::iequals(str, "create")) { value = RestoreTableCreationMode::kCreate; @@ -54,7 +54,7 @@ namespace if (field.getType() == Field::Types::UInt64) { - UInt64 number = field.get(); + UInt64 number = field.safeGet(); if (number == 1) { value = RestoreTableCreationMode::kCreate; @@ -95,7 +95,7 @@ namespace { if (field.getType() == Field::Types::String) { - const String & str = field.get(); + const String & str = field.safeGet(); if (str == "1" || boost::iequals(str, "true") || boost::iequals(str, "create")) { value = RestoreAccessCreationMode::kCreate; @@ -118,7 +118,7 @@ namespace if (field.getType() == Field::Types::UInt64) { - UInt64 number = field.get(); + UInt64 number = field.safeGet(); if (number == 1) { value = RestoreAccessCreationMode::kCreate; diff --git a/src/Backups/SettingsFieldOptionalString.cpp b/src/Backups/SettingsFieldOptionalString.cpp index 573fd1e052c..684407a533d 100644 --- a/src/Backups/SettingsFieldOptionalString.cpp +++ b/src/Backups/SettingsFieldOptionalString.cpp @@ -19,7 +19,7 @@ SettingFieldOptionalString::SettingFieldOptionalString(const Field & field) if (field.getType() == Field::Types::String) { - value = field.get(); + value = field.safeGet(); return; } diff --git a/src/Backups/SettingsFieldOptionalUUID.cpp b/src/Backups/SettingsFieldOptionalUUID.cpp index 3f14608b206..0011f7f1073 100644 --- a/src/Backups/SettingsFieldOptionalUUID.cpp +++ b/src/Backups/SettingsFieldOptionalUUID.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes if (field.getType() == Field::Types::String) { - const String & str = field.get(); + const String & str = field.safeGet(); if (str.empty()) { value = std::nullopt; diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index f5b700ea529..cc7b37aad52 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -133,7 +133,7 @@ Field QueryFuzzer::fuzzField(Field field) if (type == Field::Types::String) { - auto & str = field.get(); + auto & str = field.safeGet(); UInt64 action = fuzz_rand() % 10; switch (action) { @@ -159,7 +159,7 @@ Field QueryFuzzer::fuzzField(Field field) } else if (type == Field::Types::Array) { - auto & arr = field.get(); + auto & arr = field.safeGet(); if (fuzz_rand() % 5 == 0 && !arr.empty()) { @@ -191,7 +191,7 @@ Field QueryFuzzer::fuzzField(Field field) } else if (type == Field::Types::Tuple) { - auto & arr = field.get(); + auto & arr = field.safeGet(); if (fuzz_rand() % 5 == 0 && !arr.empty()) { @@ -912,17 +912,17 @@ ASTPtr QueryFuzzer::fuzzLiteralUnderExpressionList(ASTPtr child) auto type = l->value.getType(); if (type == Field::Types::Which::String && fuzz_rand() % 7 == 0) { - String value = l->value.get(); + String value = l->value.safeGet(); child = makeASTFunction( "toFixedString", std::make_shared(value), std::make_shared(static_cast(value.size()))); } else if (type == Field::Types::Which::UInt64 && fuzz_rand() % 7 == 0) { - child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toUInt128" : "toUInt256", std::make_shared(l->value.get())); + child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toUInt128" : "toUInt256", std::make_shared(l->value.safeGet())); } else if (type == Field::Types::Which::Int64 && fuzz_rand() % 7 == 0) { - child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toInt128" : "toInt256", std::make_shared(l->value.get())); + child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toInt128" : "toInt256", std::make_shared(l->value.safeGet())); } else if (type == Field::Types::Which::Float64 && fuzz_rand() % 7 == 0) { @@ -930,22 +930,22 @@ ASTPtr QueryFuzzer::fuzzLiteralUnderExpressionList(ASTPtr child) if (decimal == 0) child = makeASTFunction( "toDecimal32", - std::make_shared(l->value.get()), + std::make_shared(l->value.safeGet()), std::make_shared(static_cast(fuzz_rand() % 9))); else if (decimal == 1) child = makeASTFunction( "toDecimal64", - std::make_shared(l->value.get()), + std::make_shared(l->value.safeGet()), std::make_shared(static_cast(fuzz_rand() % 18))); else if (decimal == 2) child = makeASTFunction( "toDecimal128", - std::make_shared(l->value.get()), + std::make_shared(l->value.safeGet()), std::make_shared(static_cast(fuzz_rand() % 38))); else child = makeASTFunction( "toDecimal256", - std::make_shared(l->value.get()), + std::make_shared(l->value.safeGet()), std::make_shared(static_cast(fuzz_rand() % 76))); } diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 0188ebc8173..affd620f83a 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -214,7 +214,7 @@ void Suggest::fillWordsFromBlock(const Block & block) Words new_words; new_words.reserve(rows); for (size_t i = 0; i < rows; ++i) - new_words.emplace_back(column[i].get()); + new_words.emplace_back(column[i].safeGet()); addWords(std::move(new_words)); } diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index e26fe790a8e..c41c340b069 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -426,9 +426,9 @@ MutableColumnPtr ColumnAggregateFunction::cloneEmpty() const Field ColumnAggregateFunction::operator[](size_t n) const { Field field = AggregateFunctionStateData(); - field.get().name = type_string; + field.safeGet().name = type_string; { - WriteBufferFromString buffer(field.get().data); + WriteBufferFromString buffer(field.safeGet().data); func->serialize(data[n], buffer, version); } return field; @@ -436,12 +436,7 @@ Field ColumnAggregateFunction::operator[](size_t n) const void ColumnAggregateFunction::get(size_t n, Field & res) const { - res = AggregateFunctionStateData(); - res.get().name = type_string; - { - WriteBufferFromString buffer(res.get().data); - func->serialize(data[n], buffer, version); - } + res = operator[](n); } StringRef ColumnAggregateFunction::getDataAt(size_t n) const @@ -521,7 +516,7 @@ void ColumnAggregateFunction::insert(const Field & x) "Inserting field of type {} into ColumnAggregateFunction. Expected {}", x.getTypeName(), Field::Types::AggregateFunctionState); - const auto & field_name = x.get().name; + const auto & field_name = x.safeGet().name; if (type_string != field_name) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot insert filed with type {} into column with type {}", field_name, type_string); @@ -529,7 +524,7 @@ void ColumnAggregateFunction::insert(const Field & x) ensureOwnership(); Arena & arena = createOrGetArena(); pushBackAndCreateState(data, arena, func.get()); - ReadBufferFromString read_buffer(x.get().data); + ReadBufferFromString read_buffer(x.safeGet().data); func->deserialize(data.back(), read_buffer, version, &arena); } @@ -538,14 +533,14 @@ bool ColumnAggregateFunction::tryInsert(const DB::Field & x) if (x.getType() != Field::Types::AggregateFunctionState) return false; - const auto & field_name = x.get().name; + const auto & field_name = x.safeGet().name; if (type_string != field_name) return false; ensureOwnership(); Arena & arena = createOrGetArena(); pushBackAndCreateState(data, arena, func.get()); - ReadBufferFromString read_buffer(x.get().data); + ReadBufferFromString read_buffer(x.safeGet().data); func->deserialize(data.back(), read_buffer, version, &arena); return true; } diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 19cce678cc7..5379adc0bf7 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -141,7 +141,7 @@ void ColumnArray::get(size_t n, Field & res) const size, max_array_size_as_field); res = Array(); - Array & res_arr = res.get(); + Array & res_arr = res.safeGet(); res_arr.reserve(size); for (size_t i = 0; i < size; ++i) @@ -309,7 +309,7 @@ void ColumnArray::updateHashFast(SipHash & hash) const void ColumnArray::insert(const Field & x) { - const Array & array = x.get(); + const Array & array = x.safeGet(); size_t size = array.size(); for (size_t i = 0; i < size; ++i) getData().insert(array[i]); @@ -321,7 +321,7 @@ bool ColumnArray::tryInsert(const Field & x) if (x.getType() != Field::Types::Which::Array) return false; - const Array & array = x.get(); + const Array & array = x.safeGet(); size_t size = array.size(); for (size_t i = 0; i < size; ++i) { diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 59bfbd2159c..07120f5f035 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -74,7 +74,7 @@ public: void insertData(const char * src, size_t /*length*/) override; void insertDefault() override { data.push_back(T()); } void insertManyDefaults(size_t length) override { data.resize_fill(data.size() + length); } - void insert(const Field & x) override { data.push_back(x.get()); } + void insert(const Field & x) override { data.push_back(x.safeGet()); } bool tryInsert(const Field & x) override; #if !defined(DEBUG_OR_SANITIZER_BUILD) void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 0bb3f7edb14..04e894ee5ab 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -59,7 +59,7 @@ bool ColumnFixedString::isDefaultAt(size_t index) const void ColumnFixedString::insert(const Field & x) { - const String & s = x.get(); + const String & s = x.safeGet(); insertData(s.data(), s.size()); } @@ -67,7 +67,7 @@ bool ColumnFixedString::tryInsert(const Field & x) { if (x.getType() != Field::Types::Which::String) return false; - const String & s = x.get(); + const String & s = x.safeGet(); if (s.size() > n) return false; insertData(s.data(), s.size()); diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 1025b4e77b9..6a6618bd81e 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -72,7 +72,7 @@ void ColumnMap::get(size_t n, Field & res) const size_t size = offsets[n] - offsets[n - 1]; res = Map(); - auto & map = res.get(); + auto & map = res.safeGet(); map.reserve(size); for (size_t i = 0; i < size; ++i) @@ -96,7 +96,7 @@ void ColumnMap::insertData(const char *, size_t) void ColumnMap::insert(const Field & x) { - const auto & map = x.get(); + const auto & map = x.safeGet(); nested->insert(Array(map.begin(), map.end())); } @@ -105,7 +105,7 @@ bool ColumnMap::tryInsert(const Field & x) if (x.getType() != Field::Types::Which::Map) return false; - const auto & map = x.get(); + const auto & map = x.safeGet(); return nested->tryInsert(Array(map.begin(), map.end())); } @@ -288,8 +288,8 @@ void ColumnMap::getExtremes(Field & min, Field & max) const /// Convert result Array fields to Map fields because client expect min and max field to have type Map - Array nested_min_value = nested_min.get(); - Array nested_max_value = nested_max.get(); + Array nested_min_value = nested_min.safeGet(); + Array nested_max_value = nested_max.safeGet(); Map map_min_value(nested_min_value.begin(), nested_min_value.end()); Map map_max_value(nested_max_value.begin(), nested_max_value.end()); diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index a6431007cb6..eb99bb4081b 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -698,7 +698,7 @@ void ColumnObject::forEachSubcolumnRecursively(RecursiveMutableColumnCallback ca void ColumnObject::insert(const Field & field) { - const auto & object = field.get(); + const auto & object = field.safeGet(); HashSet inserted_paths; size_t old_size = size(); @@ -754,7 +754,7 @@ void ColumnObject::get(size_t n, Field & res) const { assert(n < size()); res = Object(); - auto & object = res.get(); + auto & object = res.safeGet(); for (const auto & entry : subcolumns) { diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index c1012e1e55e..65aff9e6255 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -123,7 +123,7 @@ public: void insert(const Field & x) override { - const String & s = x.get(); + const String & s = x.safeGet(); const size_t old_size = chars.size(); const size_t size_to_append = s.size() + 1; const size_t new_size = old_size + size_to_append; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 4fc3f88a87c..251be8d9986 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -141,7 +141,7 @@ void ColumnTuple::get(size_t n, Field & res) const const size_t tuple_size = columns.size(); res = Tuple(); - Tuple & res_tuple = res.get(); + Tuple & res_tuple = res.safeGet(); res_tuple.reserve(tuple_size); for (size_t i = 0; i < tuple_size; ++i) @@ -169,7 +169,7 @@ void ColumnTuple::insertData(const char *, size_t) void ColumnTuple::insert(const Field & x) { - const auto & tuple = x.get(); + const auto & tuple = x.safeGet(); const size_t tuple_size = columns.size(); if (tuple.size() != tuple_size) @@ -185,7 +185,7 @@ bool ColumnTuple::tryInsert(const Field & x) if (x.getType() != Field::Types::Which::Tuple) return false; - const auto & tuple = x.get(); + const auto & tuple = x.safeGet(); const size_t tuple_size = columns.size(); if (tuple.size() != tuple_size) diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 2fe5b635bd2..a5e1ee4b462 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -85,7 +85,7 @@ public: void insertMany(const Field & field, size_t length) override { - data.resize_fill(data.size() + length, static_cast(field.get())); + data.resize_fill(data.size() + length, static_cast(field.safeGet())); } void insertData(const char * pos, size_t) override @@ -235,7 +235,7 @@ public: void insert(const Field & x) override { - data.push_back(static_cast(x.get())); + data.push_back(static_cast(x.safeGet())); } bool tryInsert(const DB::Field & x) override; diff --git a/src/Columns/tests/gtest_column_variant.cpp b/src/Columns/tests/gtest_column_variant.cpp index 25f276b9600..5e481b88409 100644 --- a/src/Columns/tests/gtest_column_variant.cpp +++ b/src/Columns/tests/gtest_column_variant.cpp @@ -108,10 +108,10 @@ void checkColumnVariant1(ColumnVariant * column) ASSERT_EQ(offsets[1], 0); ASSERT_EQ(offsets[3], 1); ASSERT_TRUE(column->isDefaultAt(2) && column->isDefaultAt(4)); - ASSERT_EQ((*column)[0].get(), 42); - ASSERT_EQ((*column)[1].get(), "Hello"); + ASSERT_EQ((*column)[0].safeGet(), 42); + ASSERT_EQ((*column)[1].safeGet(), "Hello"); ASSERT_TRUE((*column)[2].isNull()); - ASSERT_EQ((*column)[3].get(), "World"); + ASSERT_EQ((*column)[3].safeGet(), "World"); ASSERT_TRUE((*column)[4].isNull()); } @@ -209,9 +209,9 @@ TEST(ColumnVariant, CreateFromDiscriminatorsAndOneFullColumnNoNulls) ASSERT_EQ(offsets[0], 0); ASSERT_EQ(offsets[1], 1); ASSERT_EQ(offsets[2], 2); - ASSERT_EQ((*column)[0].get(), 0); - ASSERT_EQ((*column)[1].get(), 1); - ASSERT_EQ((*column)[2].get(), 2); + ASSERT_EQ((*column)[0].safeGet(), 0); + ASSERT_EQ((*column)[1].safeGet(), 1); + ASSERT_EQ((*column)[2].safeGet(), 2); } TEST(ColumnVariant, CreateFromDiscriminatorsAndOneFullColumnNoNullsWithLocalOrder) @@ -222,9 +222,9 @@ TEST(ColumnVariant, CreateFromDiscriminatorsAndOneFullColumnNoNullsWithLocalOrde ASSERT_EQ(offsets[0], 0); ASSERT_EQ(offsets[1], 1); ASSERT_EQ(offsets[2], 2); - ASSERT_EQ((*column)[0].get(), 0); - ASSERT_EQ((*column)[1].get(), 1); - ASSERT_EQ((*column)[2].get(), 2); + ASSERT_EQ((*column)[0].safeGet(), 0); + ASSERT_EQ((*column)[1].safeGet(), 1); + ASSERT_EQ((*column)[2].safeGet(), 2); ASSERT_EQ(column->localDiscriminatorAt(0), 2); ASSERT_EQ(column->localDiscriminatorAt(1), 2); ASSERT_EQ(column->localDiscriminatorAt(2), 2); @@ -331,9 +331,9 @@ TEST(ColumnVariant, CloneResizedGeneral1) ASSERT_EQ(offsets[0], 0); ASSERT_EQ(offsets[1], 0); ASSERT_EQ(offsets[3], 1); - ASSERT_EQ((*resized_column_variant)[0].get(), 42); - ASSERT_EQ((*resized_column_variant)[1].get(), "Hello"); - ASSERT_EQ((*resized_column_variant)[3].get(), 43); + ASSERT_EQ((*resized_column_variant)[0].safeGet(), 42); + ASSERT_EQ((*resized_column_variant)[1].safeGet(), "Hello"); + ASSERT_EQ((*resized_column_variant)[3].safeGet(), 43); } TEST(ColumnVariant, CloneResizedGeneral2) @@ -367,7 +367,7 @@ TEST(ColumnVariant, CloneResizedGeneral2) ASSERT_EQ(discriminators[2], ColumnVariant::NULL_DISCRIMINATOR); const auto & offsets = resized_column_variant->getOffsets(); ASSERT_EQ(offsets[0], 0); - ASSERT_EQ((*resized_column_variant)[0].get(), 42); + ASSERT_EQ((*resized_column_variant)[0].safeGet(), 42); } TEST(ColumnVariant, CloneResizedGeneral3) @@ -405,10 +405,10 @@ TEST(ColumnVariant, CloneResizedGeneral3) ASSERT_EQ(offsets[1], 0); ASSERT_EQ(offsets[2], 1); ASSERT_EQ(offsets[3], 1); - ASSERT_EQ((*resized_column_variant)[0].get(), 42); - ASSERT_EQ((*resized_column_variant)[1].get(), "Hello"); - ASSERT_EQ((*resized_column_variant)[2].get(), "World"); - ASSERT_EQ((*resized_column_variant)[3].get(), 43); + ASSERT_EQ((*resized_column_variant)[0].safeGet(), 42); + ASSERT_EQ((*resized_column_variant)[1].safeGet(), "Hello"); + ASSERT_EQ((*resized_column_variant)[2].safeGet(), "World"); + ASSERT_EQ((*resized_column_variant)[3].safeGet(), 43); } MutableColumnPtr createDiscriminators2() @@ -465,7 +465,7 @@ TEST(ColumnVariant, InsertFrom) auto column_from = createVariantColumn2(change_order); column_to->insertFrom(*column_from, 3); ASSERT_EQ(column_to->globalDiscriminatorAt(5), 0); - ASSERT_EQ((*column_to)[5].get(), 43); + ASSERT_EQ((*column_to)[5].safeGet(), 43); } } @@ -478,8 +478,8 @@ TEST(ColumnVariant, InsertRangeFromOneColumnNoNulls) column_to->insertRangeFrom(*column_from, 2, 2); ASSERT_EQ(column_to->globalDiscriminatorAt(7), 0); ASSERT_EQ(column_to->globalDiscriminatorAt(8), 0); - ASSERT_EQ((*column_to)[7].get(), 2); - ASSERT_EQ((*column_to)[8].get(), 3); + ASSERT_EQ((*column_to)[7].safeGet(), 2); + ASSERT_EQ((*column_to)[8].safeGet(), 3); } } @@ -494,9 +494,9 @@ TEST(ColumnVariant, InsertRangeFromGeneral) ASSERT_EQ(column_to->globalDiscriminatorAt(6), ColumnVariant::NULL_DISCRIMINATOR); ASSERT_EQ(column_to->globalDiscriminatorAt(7), 0); ASSERT_EQ(column_to->globalDiscriminatorAt(8), 1); - ASSERT_EQ((*column_to)[5].get(), "Hello"); - ASSERT_EQ((*column_to)[7].get(), 43); - ASSERT_EQ((*column_to)[8].get(), "World"); + ASSERT_EQ((*column_to)[5].safeGet(), "Hello"); + ASSERT_EQ((*column_to)[7].safeGet(), 43); + ASSERT_EQ((*column_to)[8].safeGet(), "World"); } } @@ -509,8 +509,8 @@ TEST(ColumnVariant, InsertManyFrom) column_to->insertManyFrom(*column_from, 3, 2); ASSERT_EQ(column_to->globalDiscriminatorAt(5), 0); ASSERT_EQ(column_to->globalDiscriminatorAt(6), 0); - ASSERT_EQ((*column_to)[5].get(), 43); - ASSERT_EQ((*column_to)[6].get(), 43); + ASSERT_EQ((*column_to)[5].safeGet(), 43); + ASSERT_EQ((*column_to)[6].safeGet(), 43); } } @@ -520,8 +520,8 @@ TEST(ColumnVariant, PopBackOneColumnNoNulls) column->popBack(3); ASSERT_EQ(column->size(), 2); ASSERT_EQ(column->getVariantByLocalDiscriminator(0).size(), 2); - ASSERT_EQ((*column)[0].get(), 0); - ASSERT_EQ((*column)[1].get(), 1); + ASSERT_EQ((*column)[0].safeGet(), 0); + ASSERT_EQ((*column)[1].safeGet(), 1); } TEST(ColumnVariant, PopBackGeneral) @@ -531,8 +531,8 @@ TEST(ColumnVariant, PopBackGeneral) ASSERT_EQ(column->size(), 3); ASSERT_EQ(column->getVariantByLocalDiscriminator(0).size(), 1); ASSERT_EQ(column->getVariantByLocalDiscriminator(1).size(), 1); - ASSERT_EQ((*column)[0].get(), 42); - ASSERT_EQ((*column)[1].get(), "Hello"); + ASSERT_EQ((*column)[0].safeGet(), 42); + ASSERT_EQ((*column)[1].safeGet(), "Hello"); ASSERT_TRUE((*column)[2].isNull()); } @@ -545,8 +545,8 @@ TEST(ColumnVariant, FilterOneColumnNoNulls) filter.push_back(1); auto filtered_column = column->filter(filter, -1); ASSERT_EQ(filtered_column->size(), 2); - ASSERT_EQ((*filtered_column)[0].get(), 0); - ASSERT_EQ((*filtered_column)[1].get(), 2); + ASSERT_EQ((*filtered_column)[0].safeGet(), 0); + ASSERT_EQ((*filtered_column)[1].safeGet(), 2); } TEST(ColumnVariant, FilterGeneral) @@ -562,7 +562,7 @@ TEST(ColumnVariant, FilterGeneral) filter.push_back(0); auto filtered_column = column->filter(filter, -1); ASSERT_EQ(filtered_column->size(), 3); - ASSERT_EQ((*filtered_column)[0].get(), "Hello"); + ASSERT_EQ((*filtered_column)[0].safeGet(), "Hello"); ASSERT_TRUE((*filtered_column)[1].isNull()); ASSERT_TRUE((*filtered_column)[2].isNull()); } @@ -577,9 +577,9 @@ TEST(ColumnVariant, PermuteAndIndexOneColumnNoNulls) permutation.push_back(0); auto permuted_column = column->permute(permutation, 3); ASSERT_EQ(permuted_column->size(), 3); - ASSERT_EQ((*permuted_column)[0].get(), 1); - ASSERT_EQ((*permuted_column)[1].get(), 3); - ASSERT_EQ((*permuted_column)[2].get(), 2); + ASSERT_EQ((*permuted_column)[0].safeGet(), 1); + ASSERT_EQ((*permuted_column)[1].safeGet(), 3); + ASSERT_EQ((*permuted_column)[2].safeGet(), 2); auto index = ColumnUInt64::create(); index->getData().push_back(1); @@ -588,9 +588,9 @@ TEST(ColumnVariant, PermuteAndIndexOneColumnNoNulls) index->getData().push_back(0); auto indexed_column = column->index(*index, 3); ASSERT_EQ(indexed_column->size(), 3); - ASSERT_EQ((*indexed_column)[0].get(), 1); - ASSERT_EQ((*indexed_column)[1].get(), 3); - ASSERT_EQ((*indexed_column)[2].get(), 2); + ASSERT_EQ((*indexed_column)[0].safeGet(), 1); + ASSERT_EQ((*indexed_column)[1].safeGet(), 3); + ASSERT_EQ((*indexed_column)[2].safeGet(), 2); } TEST(ColumnVariant, PermuteGeneral) @@ -603,9 +603,9 @@ TEST(ColumnVariant, PermuteGeneral) permutation.push_back(5); auto permuted_column = column->permute(permutation, 4); ASSERT_EQ(permuted_column->size(), 4); - ASSERT_EQ((*permuted_column)[0].get(), 43); - ASSERT_EQ((*permuted_column)[1].get(), "World"); - ASSERT_EQ((*permuted_column)[2].get(), "Hello"); + ASSERT_EQ((*permuted_column)[0].safeGet(), 43); + ASSERT_EQ((*permuted_column)[1].safeGet(), "World"); + ASSERT_EQ((*permuted_column)[2].safeGet(), "Hello"); ASSERT_TRUE((*permuted_column)[3].isNull()); } @@ -618,12 +618,12 @@ TEST(ColumnVariant, ReplicateOneColumnNoNull) offsets.push_back(6); auto replicated_column = column->replicate(offsets); ASSERT_EQ(replicated_column->size(), 6); - ASSERT_EQ((*replicated_column)[0].get(), 1); - ASSERT_EQ((*replicated_column)[1].get(), 1); - ASSERT_EQ((*replicated_column)[2].get(), 1); - ASSERT_EQ((*replicated_column)[3].get(), 2); - ASSERT_EQ((*replicated_column)[4].get(), 2); - ASSERT_EQ((*replicated_column)[5].get(), 2); + ASSERT_EQ((*replicated_column)[0].safeGet(), 1); + ASSERT_EQ((*replicated_column)[1].safeGet(), 1); + ASSERT_EQ((*replicated_column)[2].safeGet(), 1); + ASSERT_EQ((*replicated_column)[3].safeGet(), 2); + ASSERT_EQ((*replicated_column)[4].safeGet(), 2); + ASSERT_EQ((*replicated_column)[5].safeGet(), 2); } TEST(ColumnVariant, ReplicateGeneral) @@ -637,9 +637,9 @@ TEST(ColumnVariant, ReplicateGeneral) offsets.push_back(7); auto replicated_column = column->replicate(offsets); ASSERT_EQ(replicated_column->size(), 7); - ASSERT_EQ((*replicated_column)[0].get(), 42); - ASSERT_EQ((*replicated_column)[1].get(), "Hello"); - ASSERT_EQ((*replicated_column)[2].get(), "Hello"); + ASSERT_EQ((*replicated_column)[0].safeGet(), 42); + ASSERT_EQ((*replicated_column)[1].safeGet(), "Hello"); + ASSERT_EQ((*replicated_column)[2].safeGet(), "Hello"); ASSERT_TRUE((*replicated_column)[3].isNull()); ASSERT_TRUE((*replicated_column)[4].isNull()); ASSERT_TRUE((*replicated_column)[5].isNull()); @@ -657,13 +657,13 @@ TEST(ColumnVariant, ScatterOneColumnNoNulls) selector.push_back(1); auto columns = column->scatter(3, selector); ASSERT_EQ(columns[0]->size(), 2); - ASSERT_EQ((*columns[0])[0].get(), 0); - ASSERT_EQ((*columns[0])[1].get(), 3); + ASSERT_EQ((*columns[0])[0].safeGet(), 0); + ASSERT_EQ((*columns[0])[1].safeGet(), 3); ASSERT_EQ(columns[1]->size(), 2); - ASSERT_EQ((*columns[1])[0].get(), 1); - ASSERT_EQ((*columns[1])[1].get(), 4); + ASSERT_EQ((*columns[1])[0].safeGet(), 1); + ASSERT_EQ((*columns[1])[1].safeGet(), 4); ASSERT_EQ(columns[2]->size(), 1); - ASSERT_EQ((*columns[2])[0].get(), 2); + ASSERT_EQ((*columns[2])[0].safeGet(), 2); } TEST(ColumnVariant, ScatterGeneral) @@ -680,12 +680,12 @@ TEST(ColumnVariant, ScatterGeneral) auto columns = column->scatter(3, selector); ASSERT_EQ(columns[0]->size(), 3); - ASSERT_EQ((*columns[0])[0].get(), 42); - ASSERT_EQ((*columns[0])[1].get(), "Hello"); - ASSERT_EQ((*columns[0])[2].get(), 43); + ASSERT_EQ((*columns[0])[0].safeGet(), 42); + ASSERT_EQ((*columns[0])[1].safeGet(), "Hello"); + ASSERT_EQ((*columns[0])[2].safeGet(), 43); ASSERT_EQ(columns[1]->size(), 2); - ASSERT_EQ((*columns[1])[0].get(), "World"); - ASSERT_EQ((*columns[1])[1].get(), 44); + ASSERT_EQ((*columns[1])[0].safeGet(), "World"); + ASSERT_EQ((*columns[1])[1].safeGet(), 44); ASSERT_EQ(columns[2]->size(), 2); ASSERT_TRUE((*columns[2])[0].isNull()); ASSERT_TRUE((*columns[2])[1].isNull()); diff --git a/src/Columns/tests/gtest_low_cardinality.cpp b/src/Columns/tests/gtest_low_cardinality.cpp index 5e01279b7df..ce16d2cadb1 100644 --- a/src/Columns/tests/gtest_low_cardinality.cpp +++ b/src/Columns/tests/gtest_low_cardinality.cpp @@ -20,13 +20,13 @@ void testLowCardinalityNumberInsert(const DataTypePtr & data_type) Field value; column->get(0, value); - ASSERT_EQ(value.get(), 15); + ASSERT_EQ(value.safeGet(), 15); column->get(1, value); - ASSERT_EQ(value.get(), 20); + ASSERT_EQ(value.safeGet(), 20); column->get(2, value); - ASSERT_EQ(value.get(), 25); + ASSERT_EQ(value.safeGet(), 25); } TEST(ColumnLowCardinality, Insert) diff --git a/src/Common/CollectionOfDerived.h b/src/Common/CollectionOfDerived.h index 9f80ff727b4..bcbcc36c67a 100644 --- a/src/Common/CollectionOfDerived.h +++ b/src/Common/CollectionOfDerived.h @@ -168,7 +168,7 @@ private: records.emplace(it, type_idx, item); } - Records::const_iterator getImpl(std::type_index type_idx) const + typename Records::const_iterator getImpl(std::type_index type_idx) const { auto it = std::lower_bound(records.cbegin(), records.cend(), type_idx); diff --git a/src/Common/FieldBinaryEncoding.cpp b/src/Common/FieldBinaryEncoding.cpp index 6c1a8496fe6..23263c988c3 100644 --- a/src/Common/FieldBinaryEncoding.cpp +++ b/src/Common/FieldBinaryEncoding.cpp @@ -208,7 +208,7 @@ void FieldVisitorEncodeBinary::operator() (const Map & x, WriteBuffer & buf) con writeVarUInt(size, buf); for (size_t i = 0; i < size; ++i) { - const Tuple & key_and_value = x[i].get(); + const Tuple & key_and_value = x[i].safeGet(); Field::dispatch([&buf] (const auto & value) { FieldVisitorEncodeBinary()(value, buf); }, key_and_value[0]); Field::dispatch([&buf] (const auto & value) { FieldVisitorEncodeBinary()(value, buf); }, key_and_value[1]); } diff --git a/src/Common/FieldVisitorSum.cpp b/src/Common/FieldVisitorSum.cpp index b825f188586..af9503ac046 100644 --- a/src/Common/FieldVisitorSum.cpp +++ b/src/Common/FieldVisitorSum.cpp @@ -19,7 +19,7 @@ bool FieldVisitorSum::operator() (UInt64 & x) const return x != 0; } -bool FieldVisitorSum::operator() (Float64 & x) const { x += rhs.get(); return x != 0; } +bool FieldVisitorSum::operator() (Float64 & x) const { x += rhs.safeGet(); return x != 0; } bool FieldVisitorSum::operator() (Null &) const { diff --git a/src/Common/FieldVisitorSum.h b/src/Common/FieldVisitorSum.h index cbb4c4a1de3..d28676b5093 100644 --- a/src/Common/FieldVisitorSum.h +++ b/src/Common/FieldVisitorSum.h @@ -37,7 +37,7 @@ public: template bool operator() (DecimalField & x) const { - x += rhs.get>(); + x += rhs.safeGet>(); return x.getValue() != T(0); } diff --git a/src/Common/FieldVisitorToString.cpp b/src/Common/FieldVisitorToString.cpp index c4cb4266418..2148bac20d1 100644 --- a/src/Common/FieldVisitorToString.cpp +++ b/src/Common/FieldVisitorToString.cpp @@ -172,7 +172,7 @@ String FieldVisitorToString::operator() (const Object & x) const String convertFieldToString(const Field & field) { if (field.getType() == Field::Types::Which::String) - return field.get(); + return field.safeGet(); return applyVisitor(FieldVisitorToString(), field); } diff --git a/src/Common/HashTable/HashMap.h b/src/Common/HashTable/HashMap.h index a26797a687a..92621db5558 100644 --- a/src/Common/HashTable/HashMap.h +++ b/src/Common/HashTable/HashMap.h @@ -297,7 +297,7 @@ public: } /// Only inserts the value if key isn't already present - void ALWAYS_INLINE insertIfNotPresent(const Key & x, const Cell::Mapped & value) + void ALWAYS_INLINE insertIfNotPresent(const Key & x, const typename Cell::Mapped & value) { LookupResult it; bool inserted; diff --git a/src/Common/examples/arena_with_free_lists.cpp b/src/Common/examples/arena_with_free_lists.cpp index 6793d567aca..3a1304e2d94 100644 --- a/src/Common/examples/arena_with_free_lists.cpp +++ b/src/Common/examples/arena_with_free_lists.cpp @@ -174,19 +174,19 @@ struct Dictionary { switch (attribute.type) { - case AttributeUnderlyingTypeTest::UInt8: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingTypeTest::UInt16: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingTypeTest::UInt32: std::get>(attribute.arrays)[idx] = static_cast(value.get()); break; - case AttributeUnderlyingTypeTest::UInt64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingTypeTest::Int8: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingTypeTest::Int16: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingTypeTest::Int32: std::get>(attribute.arrays)[idx] = static_cast(value.get()); break; - case AttributeUnderlyingTypeTest::Int64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingTypeTest::Float32: std::get>(attribute.arrays)[idx] = static_cast(value.get()); break; - case AttributeUnderlyingTypeTest::Float64: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingTypeTest::UInt8: std::get>(attribute.arrays)[idx] = value.safeGet(); break; + case AttributeUnderlyingTypeTest::UInt16: std::get>(attribute.arrays)[idx] = value.safeGet(); break; + case AttributeUnderlyingTypeTest::UInt32: std::get>(attribute.arrays)[idx] = static_cast(value.safeGet()); break; + case AttributeUnderlyingTypeTest::UInt64: std::get>(attribute.arrays)[idx] = value.safeGet(); break; + case AttributeUnderlyingTypeTest::Int8: std::get>(attribute.arrays)[idx] = value.safeGet(); break; + case AttributeUnderlyingTypeTest::Int16: std::get>(attribute.arrays)[idx] = value.safeGet(); break; + case AttributeUnderlyingTypeTest::Int32: std::get>(attribute.arrays)[idx] = static_cast(value.safeGet()); break; + case AttributeUnderlyingTypeTest::Int64: std::get>(attribute.arrays)[idx] = value.safeGet(); break; + case AttributeUnderlyingTypeTest::Float32: std::get>(attribute.arrays)[idx] = static_cast(value.safeGet()); break; + case AttributeUnderlyingTypeTest::Float64: std::get>(attribute.arrays)[idx] = value.safeGet(); break; case AttributeUnderlyingTypeTest::String: { - const auto & string = value.get(); + const auto & string = value.safeGet(); auto & string_ref = std::get>(attribute.arrays)[idx]; const auto & null_value_ref = std::get(attribute.null_values); diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index af681cd5639..c2bcf6ec651 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -49,7 +49,7 @@ ExternalTableDataPtr BaseExternalTable::getData(ContextPtr context) { initReadBuffer(); initSampleBlock(); - auto input = context->getInputFormat(format, *read_buffer, sample_block, context->getSettingsRef().get("max_block_size").get()); + auto input = context->getInputFormat(format, *read_buffer, sample_block, context->getSettingsRef().get("max_block_size").safeGet()); auto data = std::make_unique(); data->pipe = std::make_unique(); diff --git a/src/Core/Field.h b/src/Core/Field.h index f1bb4a72b0d..689ac38a235 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -457,15 +457,6 @@ public: std::string_view getTypeName() const; bool isNull() const { return which == Types::Null; } - template - NearestFieldType> & get(); - - template - const auto & get() const - { - auto * mutable_this = const_cast *>(this); - return mutable_this->get(); - } bool isNegativeInfinity() const { return which == Types::Null && get().isNegativeInfinity(); } bool isPositiveInfinity() const { return which == Types::Null && get().isPositiveInfinity(); } @@ -681,6 +672,25 @@ private: Types::Which which; + /// This function is prone to type punning and should never be used outside of Field class, + /// whenever it is used within this class the stored type should be checked in advance. + template + NearestFieldType> & get() + { + // Before storing the value in the Field, we static_cast it to the field + // storage type, so here we return the value of storage type as well. + // Otherwise, it is easy to make a mistake of reinterpret_casting the stored + // value to a different and incompatible type. + // For example, a Float32 value is stored as Float64, and it is incorrect to + // return a reference to this value as Float32. + return *reinterpret_cast>*>(&storage); + } + + template + NearestFieldType> & get() const + { + return const_cast(this)->get(); + } /// Assuming there was no allocated state or it was deallocated (see destroy). template @@ -859,55 +869,18 @@ constexpr bool isInt64OrUInt64FieldType(Field::Types::Which t) || t == Field::Types::UInt64; } -constexpr bool isInt64OrUInt64orBoolFieldType(Field::Types::Which t) -{ - return t == Field::Types::Int64 - || t == Field::Types::UInt64 - || t == Field::Types::Bool; -} - -// Field value getter with type checking in debug builds. -template -NearestFieldType> & Field::get() -{ - // Before storing the value in the Field, we static_cast it to the field - // storage type, so here we return the value of storage type as well. - // Otherwise, it is easy to make a mistake of reinterpret_casting the stored - // value to a different and incompatible type. - // For example, a Float32 value is stored as Float64, and it is incorrect to - // return a reference to this value as Float32. - using StoredType = NearestFieldType>; - -#ifndef NDEBUG - // Disregard signedness when converting between int64 types. - constexpr Field::Types::Which target = TypeToEnum::value; - if (target != which - && (!isInt64OrUInt64orBoolFieldType(target) || !isInt64OrUInt64orBoolFieldType(which)) && target != Field::Types::IPv4) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Invalid Field get from type {} to type {}", which, target); -#endif - - StoredType * MAY_ALIAS ptr = reinterpret_cast(&storage); - - return *ptr; -} - - template auto & Field::safeGet() { const Types::Which target = TypeToEnum>>::value; - /// We allow converting int64 <-> uint64, int64 <-> bool, uint64 <-> bool in safeGet(). - if (target != which - && (!isInt64OrUInt64orBoolFieldType(target) || !isInt64OrUInt64orBoolFieldType(which))) - throw Exception(ErrorCodes::BAD_GET, - "Bad get: has {}, requested {}", getTypeName(), target); + /// bool is stored as uint64, will be returned as UInt64 when requested as bool or UInt64, as Int64 when requested as Int64 + if (target != which && !(which == Field::Types::Bool && (target == Field::Types::UInt64 || target == Field::Types::Int64))) + throw Exception(ErrorCodes::BAD_GET, "Bad get: has {}, requested {}", getTypeName(), target); return get(); } - template requires not_field_or_bool_or_stringlike Field::Field(T && rhs) diff --git a/src/Core/Range.cpp b/src/Core/Range.cpp index 956b96653a1..1a5ce1e012e 100644 --- a/src/Core/Range.cpp +++ b/src/Core/Range.cpp @@ -62,27 +62,27 @@ void Range::shrinkToIncludedIfPossible() { if (left.isExplicit() && !left_included) { - if (left.getType() == Field::Types::UInt64 && left.get() != std::numeric_limits::max()) + if (left.getType() == Field::Types::UInt64 && left.safeGet() != std::numeric_limits::max()) { - ++left.get(); + ++left.safeGet(); left_included = true; } - if (left.getType() == Field::Types::Int64 && left.get() != std::numeric_limits::max()) + if (left.getType() == Field::Types::Int64 && left.safeGet() != std::numeric_limits::max()) { - ++left.get(); + ++left.safeGet(); left_included = true; } } if (right.isExplicit() && !right_included) { - if (right.getType() == Field::Types::UInt64 && right.get() != std::numeric_limits::min()) + if (right.getType() == Field::Types::UInt64 && right.safeGet() != std::numeric_limits::min()) { - --right.get(); + --right.safeGet(); right_included = true; } - if (right.getType() == Field::Types::Int64 && right.get() != std::numeric_limits::min()) + if (right.getType() == Field::Types::Int64 && right.safeGet() != std::numeric_limits::min()) { - --right.get(); + --right.safeGet(); right_included = true; } } diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 9c9c9c1db00..45bd2b9eb42 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -118,7 +118,7 @@ void Settings::set(std::string_view name, const Field & value) { if (value.getType() != Field::Types::Which::String) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type of value for setting 'compatibility'. Expected String, got {}", value.getTypeName()); - applyCompatibilitySetting(value.get()); + applyCompatibilitySetting(value.safeGet()); } /// If we change setting that was changed by compatibility setting before /// we should remove it from settings_changed_by_compatibility_setting, diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 7d094e2a107..bb2ef58b214 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -53,29 +53,29 @@ namespace { if (f.getType() == Field::Types::String) { - return stringToNumber(f.get()); + return stringToNumber(f.safeGet()); } else if (f.getType() == Field::Types::UInt64) { T result; - if (!accurate::convertNumeric(f.get(), result)) + if (!accurate::convertNumeric(f.safeGet(), result)) throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Field value {} is out of range of {} type", f, demangle(typeid(T).name())); return result; } else if (f.getType() == Field::Types::Int64) { T result; - if (!accurate::convertNumeric(f.get(), result)) + if (!accurate::convertNumeric(f.safeGet(), result)) throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Field value {} is out of range of {} type", f, demangle(typeid(T).name())); return result; } else if (f.getType() == Field::Types::Bool) { - return T(f.get()); + return T(f.safeGet()); } else if (f.getType() == Field::Types::Float64) { - Float64 x = f.get(); + Float64 x = f.safeGet(); if constexpr (std::is_floating_point_v) { return T(x); @@ -120,7 +120,7 @@ namespace if (f.getType() == Field::Types::String) { /// Allow to parse Map from string field. For the convenience. - const auto & str = f.get(); + const auto & str = f.safeGet(); return stringToMap(str); } @@ -218,7 +218,7 @@ namespace UInt64 fieldToMaxThreads(const Field & f) { if (f.getType() == Field::Types::String) - return stringToMaxThreads(f.get()); + return stringToMaxThreads(f.safeGet()); else return fieldToNumber(f); } diff --git a/src/Core/SettingsQuirks.cpp b/src/Core/SettingsQuirks.cpp index 5541cc19653..3127a5ef36d 100644 --- a/src/Core/SettingsQuirks.cpp +++ b/src/Core/SettingsQuirks.cpp @@ -100,7 +100,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log) return current_value; }; - UInt64 max_threads = get_current_value("max_threads").get(); + UInt64 max_threads = get_current_value("max_threads").safeGet(); UInt64 max_threads_max_value = 256 * getNumberOfPhysicalCPUCores(); if (max_threads > max_threads_max_value) { @@ -120,7 +120,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log) "input_format_parquet_max_block_size"}; for (auto const & setting : block_rows_settings) { - if (auto block_size = get_current_value(setting).get(); + if (auto block_size = get_current_value(setting).safeGet(); block_size > max_sane_block_rows_size) { if (log) @@ -129,7 +129,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log) } } - if (auto max_block_size = get_current_value("max_block_size").get(); max_block_size == 0) + if (auto max_block_size = get_current_value("max_block_size").safeGet(); max_block_size == 0) { if (log) LOG_WARNING(log, "Sanity check: 'max_block_size' cannot be 0. Set to default value {}", DEFAULT_BLOCK_SIZE); diff --git a/src/Core/examples/field.cpp b/src/Core/examples/field.cpp index 110e11d0cb1..3064290e127 100644 --- a/src/Core/examples/field.cpp +++ b/src/Core/examples/field.cpp @@ -37,7 +37,7 @@ int main(int argc, char ** argv) std::cerr << applyVisitor(to_string, field) << std::endl; } - field.get().push_back(field); + field.safeGet().push_back(field); std::cerr << applyVisitor(to_string, field) << std::endl; std::cerr << (field < field2) << std::endl; diff --git a/src/Core/tests/gtest_field.cpp b/src/Core/tests/gtest_field.cpp index 5585442d835..7e778be9575 100644 --- a/src/Core/tests/gtest_field.cpp +++ b/src/Core/tests/gtest_field.cpp @@ -8,31 +8,31 @@ GTEST_TEST(Field, FromBool) { Field f{false}; ASSERT_EQ(f.getType(), Field::Types::Bool); - ASSERT_EQ(f.get(), 0); - ASSERT_EQ(f.get(), false); + ASSERT_EQ(f.safeGet(), 0); + ASSERT_EQ(f.safeGet(), false); } { Field f{true}; ASSERT_EQ(f.getType(), Field::Types::Bool); - ASSERT_EQ(f.get(), 1); - ASSERT_EQ(f.get(), true); + ASSERT_EQ(f.safeGet(), 1); + ASSERT_EQ(f.safeGet(), true); } { Field f; f = false; ASSERT_EQ(f.getType(), Field::Types::Bool); - ASSERT_EQ(f.get(), 0); - ASSERT_EQ(f.get(), false); + ASSERT_EQ(f.safeGet(), 0); + ASSERT_EQ(f.safeGet(), false); } { Field f; f = true; ASSERT_EQ(f.getType(), Field::Types::Bool); - ASSERT_EQ(f.get(), 1); - ASSERT_EQ(f.get(), true); + ASSERT_EQ(f.safeGet(), 1); + ASSERT_EQ(f.safeGet(), true); } } @@ -42,15 +42,15 @@ GTEST_TEST(Field, Move) Field f; f = Field{String{"Hello, world (1)"}}; - ASSERT_EQ(f.get(), "Hello, world (1)"); + ASSERT_EQ(f.safeGet(), "Hello, world (1)"); f = Field{String{"Hello, world (2)"}}; - ASSERT_EQ(f.get(), "Hello, world (2)"); + ASSERT_EQ(f.safeGet(), "Hello, world (2)"); f = Field{Array{Field{String{"Hello, world (3)"}}}}; - ASSERT_EQ(f.get()[0].get(), "Hello, world (3)"); + ASSERT_EQ(f.safeGet()[0].safeGet(), "Hello, world (3)"); f = String{"Hello, world (4)"}; - ASSERT_EQ(f.get(), "Hello, world (4)"); + ASSERT_EQ(f.safeGet(), "Hello, world (4)"); f = Array{Field{String{"Hello, world (5)"}}}; - ASSERT_EQ(f.get()[0].get(), "Hello, world (5)"); + ASSERT_EQ(f.safeGet()[0].safeGet(), "Hello, world (5)"); f = Array{String{"Hello, world (6)"}}; - ASSERT_EQ(f.get()[0].get(), "Hello, world (6)"); + ASSERT_EQ(f.safeGet()[0].safeGet(), "Hello, world (6)"); } diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index 09175617bf1..dbb713bc382 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -119,7 +119,7 @@ MutableColumnPtr DataTypeAggregateFunction::createColumn() const Field DataTypeAggregateFunction::getDefault() const { Field field = AggregateFunctionStateData(); - field.get().name = getName(); + field.safeGet().name = getName(); AlignedBuffer place_buffer(function->sizeOfData(), function->alignOfData()); AggregateDataPtr place = place_buffer.data(); @@ -128,7 +128,7 @@ Field DataTypeAggregateFunction::getDefault() const try { - WriteBufferFromString buffer_from_field(field.get().data); + WriteBufferFromString buffer_from_field(field.safeGet().data); function->serialize(place, buffer_from_field, version); } catch (...) diff --git a/src/DataTypes/DataTypeDynamic.cpp b/src/DataTypes/DataTypeDynamic.cpp index a1b1f8325f0..e0430272479 100644 --- a/src/DataTypes/DataTypeDynamic.cpp +++ b/src/DataTypes/DataTypeDynamic.cpp @@ -71,10 +71,10 @@ static DataTypePtr create(const ASTPtr & arguments) auto * literal = argument->arguments->children[1]->as(); - if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.get() == 0 || literal->value.get() > ColumnVariant::MAX_NESTED_COLUMNS) + if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.safeGet() == 0 || literal->value.safeGet() > ColumnVariant::MAX_NESTED_COLUMNS) throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'max_types' argument for Dynamic type should be a positive integer between 1 and 255"); - return std::make_shared(literal->value.get()); + return std::make_shared(literal->value.safeGet()); } void registerDataTypeDynamic(DataTypeFactory & factory) diff --git a/src/DataTypes/DataTypeEnum.cpp b/src/DataTypes/DataTypeEnum.cpp index 08e0c0d2045..b9a5a1a5a68 100644 --- a/src/DataTypes/DataTypeEnum.cpp +++ b/src/DataTypes/DataTypeEnum.cpp @@ -122,12 +122,12 @@ Field DataTypeEnum::castToName(const Field & value_or_name) const { if (value_or_name.getType() == Field::Types::String) { - this->getValue(value_or_name.get()); /// Check correctness - return value_or_name.get(); + this->getValue(value_or_name.safeGet()); /// Check correctness + return value_or_name.safeGet(); } else if (value_or_name.getType() == Field::Types::Int64) { - Int64 value = value_or_name.get(); + Int64 value = value_or_name.safeGet(); checkOverflow(value); return this->getNameForValue(static_cast(value)).toString(); } @@ -141,12 +141,12 @@ Field DataTypeEnum::castToValue(const Field & value_or_name) const { if (value_or_name.getType() == Field::Types::String) { - return this->getValue(value_or_name.get()); + return this->getValue(value_or_name.safeGet()); } else if (value_or_name.getType() == Field::Types::Int64 || value_or_name.getType() == Field::Types::UInt64) { - Int64 value = value_or_name.get(); + Int64 value = value_or_name.safeGet(); checkOverflow(value); this->getNameForValue(static_cast(value)); /// Check correctness return value; @@ -220,7 +220,7 @@ static void autoAssignNumberForEnum(const ASTPtr & arguments) "Elements of Enum data type must be of form: " "'name' = number or 'name', where name is string literal and number is an integer"); - literal_child_assign_num = value_literal->value.get(); + literal_child_assign_num = value_literal->value.safeGet(); } assign_number_child.emplace_back(child); } @@ -269,8 +269,8 @@ static DataTypePtr createExact(const ASTPtr & arguments) "Elements of Enum data type must be of form: " "'name' = number or 'name', where name is string literal and number is an integer"); - const String & field_name = name_literal->value.get(); - const auto value = value_literal->value.get(); + const String & field_name = name_literal->value.safeGet(); + const auto value = value_literal->value.safeGet(); if (value > std::numeric_limits::max() || value < std::numeric_limits::min()) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Value {} for element '{}' exceeds range of {}", @@ -302,7 +302,7 @@ static DataTypePtr create(const ASTPtr & arguments) "Elements of Enum data type must be of form: " "'name' = number or 'name', where name is string literal and number is an integer"); - Int64 value = value_literal->value.get(); + Int64 value = value_literal->value.safeGet(); if (value > std::numeric_limits::max() || value < std::numeric_limits::min()) return createExact(arguments); diff --git a/src/DataTypes/DataTypeFixedString.cpp b/src/DataTypes/DataTypeFixedString.cpp index 080ff8826a5..63d5245287f 100644 --- a/src/DataTypes/DataTypeFixedString.cpp +++ b/src/DataTypes/DataTypeFixedString.cpp @@ -51,11 +51,11 @@ static DataTypePtr create(const ASTPtr & arguments) "FixedString data type family must have exactly one argument - size in bytes"); const auto * argument = arguments->children[0]->as(); - if (!argument || argument->value.getType() != Field::Types::UInt64 || argument->value.get() == 0) + if (!argument || argument->value.getType() != Field::Types::UInt64 || argument->value.safeGet() == 0) throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "FixedString data type family must have a number (positive integer) as its argument"); - return std::make_shared(argument->value.get()); + return std::make_shared(argument->value.safeGet()); } diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 91b9bfcb2a5..5fd69688194 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -69,7 +69,7 @@ static DataTypePtr create(const ASTPtr & arguments) throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Object data type family must have a const string as its schema name parameter"); - return std::make_shared(literal->value.get(), is_nullable); + return std::make_shared(literal->value.safeGet(), is_nullable); } void registerDataTypeObject(DataTypeFactory & factory) diff --git a/src/DataTypes/DataTypesDecimal.cpp b/src/DataTypes/DataTypesDecimal.cpp index a427fd0717a..1d8f7711de1 100644 --- a/src/DataTypes/DataTypesDecimal.cpp +++ b/src/DataTypes/DataTypesDecimal.cpp @@ -80,14 +80,14 @@ static DataTypePtr create(const ASTPtr & arguments) const auto * precision_arg = arguments->children[0]->as(); if (!precision_arg || precision_arg->value.getType() != Field::Types::UInt64) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Decimal argument precision is invalid"); - precision = precision_arg->value.get(); + precision = precision_arg->value.safeGet(); if (arguments->children.size() == 2) { const auto * scale_arg = arguments->children[1]->as(); if (!scale_arg || !isInt64OrUInt64FieldType(scale_arg->value.getType())) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Decimal argument scale is invalid"); - scale = scale_arg->value.get(); + scale = scale_arg->value.safeGet(); } } @@ -107,7 +107,7 @@ static DataTypePtr createExact(const ASTPtr & arguments) "Decimal32 | Decimal64 | Decimal128 | Decimal256 data type family must have a one number as its argument"); UInt64 precision = DecimalUtils::max_precision; - UInt64 scale = scale_arg->value.get(); + UInt64 scale = scale_arg->value.safeGet(); return createDecimal(precision, scale); } diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 1d525e5987f..a7f2ff83873 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -104,7 +104,7 @@ Array createEmptyArrayField(size_t num_dimensions) for (size_t i = 1; i < num_dimensions; ++i) { current_array->push_back(Array()); - current_array = ¤t_array->back().get(); + current_array = ¤t_array->back().safeGet(); } return array; diff --git a/src/DataTypes/Serializations/JSONDataParser.cpp b/src/DataTypes/Serializations/JSONDataParser.cpp index 56641424396..0f74815f5b4 100644 --- a/src/DataTypes/Serializations/JSONDataParser.cpp +++ b/src/DataTypes/Serializations/JSONDataParser.cpp @@ -131,7 +131,7 @@ void JSONDataParser::traverseArrayElement(const Element & element, P auto nested_hash = getHashOfNestedPath(paths[i], values[i]); if (nested_hash) { - size_t array_size = values[i].template get().size(); + size_t array_size = values[i].template safeGet().size(); auto & current_nested_sizes = ctx.nested_sizes_by_path[*nested_hash]; if (current_nested_sizes.size() == ctx.current_size) @@ -154,7 +154,7 @@ void JSONDataParser::traverseArrayElement(const Element & element, P auto nested_hash = getHashOfNestedPath(paths[i], values[i]); if (nested_hash) { - size_t array_size = values[i].template get().size(); + size_t array_size = values[i].template safeGet().size(); auto & current_nested_sizes = ctx.nested_sizes_by_path[*nested_hash]; if (current_nested_sizes.empty()) diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp index 55f7641e058..41b198890e4 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp @@ -16,14 +16,14 @@ namespace DB void SerializationAggregateFunction::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { - const AggregateFunctionStateData & state = field.get(); + const AggregateFunctionStateData & state = field.safeGet(); writeBinary(state.data, ostr); } void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { field = AggregateFunctionStateData(); - AggregateFunctionStateData & s = field.get(); + AggregateFunctionStateData & s = field.safeGet(); readBinary(s.data, istr); s.name = type_name; } diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index b7d43332085..0a9c4529e23 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -29,7 +29,7 @@ static constexpr size_t MAX_ARRAYS_SIZE = 1ULL << 40; void SerializationArray::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { - const Array & a = field.get(); + const Array & a = field.safeGet(); writeVarUInt(a.size(), ostr); for (const auto & i : a) { @@ -51,7 +51,7 @@ void SerializationArray::deserializeBinary(Field & field, ReadBuffer & istr, con settings.binary.max_binary_string_size); field = Array(); - Array & arr = field.get(); + Array & arr = field.safeGet(); arr.reserve(size); for (size_t i = 0; i < size; ++i) nested->deserializeBinary(arr.emplace_back(), istr, settings); diff --git a/src/DataTypes/Serializations/SerializationDecimalBase.cpp b/src/DataTypes/Serializations/SerializationDecimalBase.cpp index 49dc042e872..8927f949368 100644 --- a/src/DataTypes/Serializations/SerializationDecimalBase.cpp +++ b/src/DataTypes/Serializations/SerializationDecimalBase.cpp @@ -13,7 +13,7 @@ namespace DB template void SerializationDecimalBase::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { - FieldType x = field.get>(); + FieldType x = field.safeGet>(); writeBinaryLittleEndian(x, ostr); } diff --git a/src/DataTypes/Serializations/SerializationFixedString.cpp b/src/DataTypes/Serializations/SerializationFixedString.cpp index f919dc16d33..688c71792fa 100644 --- a/src/DataTypes/Serializations/SerializationFixedString.cpp +++ b/src/DataTypes/Serializations/SerializationFixedString.cpp @@ -28,7 +28,7 @@ static constexpr size_t MAX_STRINGS_SIZE = 1ULL << 30; void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { - const String & s = field.get(); + const String & s = field.safeGet(); ostr.write(s.data(), std::min(s.size(), n)); if (s.size() < n) for (size_t i = s.size(); i < n; ++i) @@ -39,7 +39,7 @@ void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer void SerializationFixedString::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { field = String(); - String & s = field.get(); + String & s = field.safeGet(); s.resize(n); istr.readStrict(s.data(), n); } diff --git a/src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp b/src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp index dfcd24aff58..c1beceb4533 100644 --- a/src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp +++ b/src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp @@ -125,7 +125,7 @@ bool SerializationIP::tryDeserializeTextCSV(DB::IColumn & column, DB::ReadB template void SerializationIP::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { - IPv x = field.get(); + IPv x = field.safeGet(); if constexpr (std::is_same_v) writeBinary(x, ostr); else diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index 0bef3c7d79d..c722b3ac7a1 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -40,7 +40,7 @@ static IColumn & extractNestedColumn(IColumn & column) void SerializationMap::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & map = field.get(); + const auto & map = field.safeGet(); writeVarUInt(map.size(), ostr); for (const auto & elem : map) { @@ -63,7 +63,7 @@ void SerializationMap::deserializeBinary(Field & field, ReadBuffer & istr, const size, settings.binary.max_binary_string_size); field = Map(); - Map & map = field.get(); + Map & map = field.safeGet(); map.reserve(size); for (size_t i = 0; i < size; ++i) { diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index bdb4dfc6735..bfc13af8ca3 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -169,7 +169,7 @@ template void SerializationNumber::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { /// ColumnVector::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64 - typename ColumnVector::ValueType x = static_cast::ValueType>(field.get()); + typename ColumnVector::ValueType x = static_cast::ValueType>(field.safeGet()); writeBinaryLittleEndian(x, ostr); } diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index 9e523d0d745..ac5d4e3e128 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -32,7 +32,7 @@ namespace ErrorCodes void SerializationString::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { - const String & s = field.get(); + const String & s = field.safeGet(); if (settings.binary.max_binary_string_size && s.size() > settings.binary.max_binary_string_size) throw Exception( ErrorCodes::TOO_LARGE_STRING_SIZE, @@ -59,7 +59,7 @@ void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr, co settings.binary.max_binary_string_size); field = String(); - String & s = field.get(); + String & s = field.safeGet(); s.resize(size); istr.readStrict(s.data(), size); } diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 7a5227ca752..594a23ab507 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -34,7 +34,7 @@ static inline const IColumn & extractElementColumn(const IColumn & column, size_ void SerializationTuple::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & tuple = field.get(); + const auto & tuple = field.safeGet(); for (size_t element_index = 0; element_index < elems.size(); ++element_index) { const auto & serialization = elems[element_index]; @@ -47,7 +47,7 @@ void SerializationTuple::deserializeBinary(Field & field, ReadBuffer & istr, con const size_t size = elems.size(); field = Tuple(); - Tuple & tuple = field.get(); + Tuple & tuple = field.safeGet(); tuple.reserve(size); for (size_t i = 0; i < size; ++i) elems[i]->deserializeBinary(tuple.emplace_back(), istr, settings); diff --git a/src/DataTypes/Serializations/SerializationUUID.cpp b/src/DataTypes/Serializations/SerializationUUID.cpp index 58178a896dc..f18466ad8ad 100644 --- a/src/DataTypes/Serializations/SerializationUUID.cpp +++ b/src/DataTypes/Serializations/SerializationUUID.cpp @@ -137,7 +137,7 @@ bool SerializationUUID::tryDeserializeTextCSV(IColumn & column, ReadBuffer & ist void SerializationUUID::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { - UUID x = field.get(); + UUID x = field.safeGet(); writeBinaryLittleEndian(x, ostr); } diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index 84a52d4affb..9a632bd381b 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -55,7 +55,7 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume } } - return argument->value.get(); + return argument->value.safeGet(); } static DataTypePtr create(const ASTPtr & arguments) diff --git a/src/Databases/DDLLoadingDependencyVisitor.cpp b/src/Databases/DDLLoadingDependencyVisitor.cpp index 67bce915168..b91aa84ecd3 100644 --- a/src/Databases/DDLLoadingDependencyVisitor.cpp +++ b/src/Databases/DDLLoadingDependencyVisitor.cpp @@ -183,7 +183,7 @@ void DDLLoadingDependencyVisitor::extractTableNameFromArgument(const ASTFunction if (name->value.getType() != Field::Types::String) return; - auto maybe_qualified_name = QualifiedTableName::tryParseFromString(name->value.get()); + auto maybe_qualified_name = QualifiedTableName::tryParseFromString(name->value.safeGet()); if (!maybe_qualified_name) return; @@ -194,7 +194,7 @@ void DDLLoadingDependencyVisitor::extractTableNameFromArgument(const ASTFunction if (literal->value.getType() != Field::Types::String) return; - auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.get()); + auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.safeGet()); /// Just return if name if invalid if (!maybe_qualified_name) return; diff --git a/src/Databases/DDLRenamingVisitor.cpp b/src/Databases/DDLRenamingVisitor.cpp index 38e100e2470..7556223b30e 100644 --- a/src/Databases/DDLRenamingVisitor.cpp +++ b/src/Databases/DDLRenamingVisitor.cpp @@ -180,7 +180,7 @@ namespace if (database_name_field && table_name_field) { - QualifiedTableName qualified_name{database_name_field->get(), table_name_field->get()}; + QualifiedTableName qualified_name{database_name_field->safeGet(), table_name_field->safeGet()}; if (!qualified_name.database.empty() && !qualified_name.table.empty()) { auto new_qualified_name = data.renaming_map.getNewTableName(qualified_name); @@ -207,7 +207,7 @@ namespace if (literal->value.getType() != Field::Types::String) return; - auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.get()); + auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.safeGet()); /// Just return if name if invalid if (!maybe_qualified_name || maybe_qualified_name->database.empty() || maybe_qualified_name->table.empty()) return; @@ -247,7 +247,7 @@ namespace if (!literal || (literal->value.getType() != Field::Types::String)) return; - auto database_name = literal->value.get(); + auto database_name = literal->value.safeGet(); if (database_name.empty()) return; diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index f127ccbc224..602bb29aebc 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -802,8 +802,8 @@ void DatabaseReplicated::checkTableEngine(const ASTCreateQuery & query, ASTStora if (!arg1 || !arg2 || arg1->value.getType() != Field::Types::String || arg2->value.getType() != Field::Types::String) return; - String maybe_path = arg1->value.get(); - String maybe_replica = arg2->value.get(); + String maybe_path = arg1->value.safeGet(); + String maybe_replica = arg2->value.safeGet(); /// Looks like it's ReplicatedMergeTree with explicit zookeeper_path and replica_name arguments. /// Let's ensure that some macros are used. diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 04b4070d5af..1364e9ae2b2 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -736,11 +736,11 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Tuple & row_data = rows_data[index].get(); + const Tuple & row_data = rows_data[index].safeGet(); const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) - casted_column->insertValue(static_cast(value.template get())); + casted_column->insertValue(static_cast(value.template safeGet())); } }; @@ -776,17 +776,17 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Tuple & row_data = rows_data[index].get(); + const Tuple & row_data = rows_data[index].safeGet(); const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) { if (value.getType() == Field::Types::UInt64) - casted_int32_column->insertValue(static_cast(value.get())); + casted_int32_column->insertValue(static_cast(value.safeGet())); else if (value.getType() == Field::Types::Int64) { /// For MYSQL_TYPE_INT24 - const Int32 & num = static_cast(value.get()); + const Int32 & num = static_cast(value.safeGet()); casted_int32_column->insertValue(num & 0x800000 ? num | 0xFF000000 : num); } else @@ -798,7 +798,7 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Tuple & row_data = rows_data[index].get(); + const Tuple & row_data = rows_data[index].safeGet(); const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) @@ -812,12 +812,12 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Tuple & row_data = rows_data[index].get(); + const Tuple & row_data = rows_data[index].safeGet(); const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) { - const String & data = value.get(); + const String & data = value.safeGet(); casted_fixed_string_column->insertData(data.data(), data.size()); } } @@ -864,7 +864,7 @@ static inline size_t onUpdateData(const Row & rows_data, Block & buffer, size_t { writeable_rows_mask[index + 1] = true; writeable_rows_mask[index] = differenceSortingKeys( - rows_data[index].get(), rows_data[index + 1].get(), sorting_columns_index); + rows_data[index].safeGet(), rows_data[index + 1].safeGet(), sorting_columns_index); } for (size_t column = 0; column < buffer.columns() - 2; ++column) diff --git a/src/Databases/MySQL/tests/gtest_mysql_binlog.cpp b/src/Databases/MySQL/tests/gtest_mysql_binlog.cpp index 11299c5b8b1..6f1ba26ee33 100644 --- a/src/Databases/MySQL/tests/gtest_mysql_binlog.cpp +++ b/src/Databases/MySQL/tests/gtest_mysql_binlog.cpp @@ -281,12 +281,12 @@ static void testFile1(IBinlog & binlog, UInt64 timeout, bool filtered = false) ASSERT_EQ(write_event->table, "a"); ASSERT_EQ(write_event->rows.size(), 1); ASSERT_EQ(write_event->rows[0].getType(), Field::Types::Tuple); - auto row_data = write_event->rows[0].get(); + auto row_data = write_event->rows[0].safeGet(); ASSERT_EQ(row_data.size(), 4u); - ASSERT_EQ(row_data[0].get(), 1u); - ASSERT_EQ(row_data[1].get(), 1u); - ASSERT_EQ(row_data[2].get(), 1u); - ASSERT_EQ(row_data[3].get(), 1u); + ASSERT_EQ(row_data[0].safeGet(), 1u); + ASSERT_EQ(row_data[1].safeGet(), 1u); + ASSERT_EQ(row_data[2].safeGet(), 1u); + ASSERT_EQ(row_data[3].safeGet(), 1u); ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); ++count; @@ -342,18 +342,18 @@ static void testFile1(IBinlog & binlog, UInt64 timeout, bool filtered = false) ASSERT_EQ(update_event->table, "a"); ASSERT_EQ(update_event->rows.size(), 2); ASSERT_EQ(update_event->rows[0].getType(), Field::Types::Tuple); - row_data = update_event->rows[0].get(); + row_data = update_event->rows[0].safeGet(); ASSERT_EQ(row_data.size(), 4u); - ASSERT_EQ(row_data[0].get(), 1u); - ASSERT_EQ(row_data[1].get(), 1u); - ASSERT_EQ(row_data[2].get(), 1u); - ASSERT_EQ(row_data[3].get(), 1u); - row_data = update_event->rows[1].get(); + ASSERT_EQ(row_data[0].safeGet(), 1u); + ASSERT_EQ(row_data[1].safeGet(), 1u); + ASSERT_EQ(row_data[2].safeGet(), 1u); + ASSERT_EQ(row_data[3].safeGet(), 1u); + row_data = update_event->rows[1].safeGet(); ASSERT_EQ(row_data.size(), 4u); - ASSERT_EQ(row_data[0].get(), 1u); - ASSERT_EQ(row_data[1].get(), 2u); - ASSERT_EQ(row_data[2].get(), 1u); - ASSERT_EQ(row_data[3].get(), 1u); + ASSERT_EQ(row_data[0].safeGet(), 1u); + ASSERT_EQ(row_data[1].safeGet(), 2u); + ASSERT_EQ(row_data[2].safeGet(), 1u); + ASSERT_EQ(row_data[3].safeGet(), 1u); ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); ++count; diff --git a/src/Dictionaries/CacheDictionaryStorage.h b/src/Dictionaries/CacheDictionaryStorage.h index 47f99bd1093..781822533e9 100644 --- a/src/Dictionaries/CacheDictionaryStorage.h +++ b/src/Dictionaries/CacheDictionaryStorage.h @@ -395,13 +395,13 @@ private: } else if constexpr (std::is_same_v) { - const String & string_value = column_value.get(); + const String & string_value = column_value.safeGet(); StringRef inserted_value = copyStringInArena(arena, string_value); container.back() = inserted_value; } else { - container.back() = static_cast(column_value.get()); + container.back() = static_cast(column_value.safeGet()); } }); } @@ -441,7 +441,7 @@ private: } else if constexpr (std::is_same_v) { - const String & string_value = column_value.get(); + const String & string_value = column_value.safeGet(); StringRef inserted_value = copyStringInArena(arena, string_value); if (!cell_was_default) @@ -454,7 +454,7 @@ private: } else { - container[index_to_use] = static_cast(column_value.get()); + container[index_to_use] = static_cast(column_value.safeGet()); } }); } @@ -651,12 +651,12 @@ private: } else if constexpr (std::is_same_v) { - auto & value = default_value.get(); + auto & value = default_value.safeGet(); value_setter(value); } else { - value_setter(default_value.get()); + value_setter(default_value.safeGet()); } } else diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 64fc05e99ab..43fd39640c3 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -345,7 +345,7 @@ public: if (attribute_default_value.isNull()) default_value_is_null = true; else - default_value = static_cast(attribute_default_value.get()); + default_value = static_cast(attribute_default_value.safeGet()); } else { @@ -377,7 +377,7 @@ public: if constexpr (std::is_same_v) { Field field = (*default_values_column)[row]; - return field.get(); + return field.safeGet(); } else if constexpr (std::is_same_v) return default_values_column->getDataAt(row); diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 999160226d9..b0233766741 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -245,7 +245,7 @@ ColumnPtr FlatDictionary::getHierarchy(ColumnPtr key_column, const DataTypePtr & std::optional null_value; if (!dictionary_attribute.null_value.isNull()) - null_value = dictionary_attribute.null_value.get(); + null_value = dictionary_attribute.null_value.safeGet(); const ContainerType & parent_keys = std::get>(hierarchical_attribute.container); @@ -300,7 +300,7 @@ ColumnUInt8::Ptr FlatDictionary::isInHierarchy( std::optional null_value; if (!dictionary_attribute.null_value.isNull()) - null_value = dictionary_attribute.null_value.get(); + null_value = dictionary_attribute.null_value.safeGet(); const ContainerType & parent_keys = std::get>(hierarchical_attribute.container); @@ -701,7 +701,7 @@ void FlatDictionary::setAttributeValue(Attribute & attribute, const UInt64 key, return; } - auto & attribute_value = value.get(); + auto & attribute_value = value.safeGet(); auto & container = std::get>(attribute.container); loaded_keys[key] = true; diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index d7d50dfb0a6..8768be8e5ec 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -240,7 +240,7 @@ ColumnPtr HashedArrayDictionary::getHierarchy(Colu std::optional null_value; if (!dictionary_attribute.null_value.isNull()) - null_value = dictionary_attribute.null_value.get(); + null_value = dictionary_attribute.null_value.safeGet(); auto is_key_valid_func = [&, this](auto & key) @@ -313,7 +313,7 @@ ColumnUInt8::Ptr HashedArrayDictionary::isInHierar std::optional null_value; if (!dictionary_attribute.null_value.isNull()) - null_value = dictionary_attribute.null_value.get(); + null_value = dictionary_attribute.null_value.safeGet(); auto is_key_valid_func = [&](auto & key) @@ -581,13 +581,13 @@ void HashedArrayDictionary::blockToAttributes(cons if constexpr (std::is_same_v) { - String & value_to_insert = column_value_to_insert.get(); + String & value_to_insert = column_value_to_insert.safeGet(); StringRef string_in_arena_reference = copyStringInArena(*string_arenas[shard], value_to_insert); attribute_container.back() = string_in_arena_reference; } else { - auto value_to_insert = static_cast(column_value_to_insert.get()); + auto value_to_insert = static_cast(column_value_to_insert.safeGet()); attribute_container.back() = value_to_insert; } }; diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 3a2b61e5149..7e935fe4855 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -636,7 +636,7 @@ ColumnPtr HashedDictionary::getHierarchy(C std::optional null_value; if (!dictionary_attribute.null_value.isNull()) - null_value = dictionary_attribute.null_value.get(); + null_value = dictionary_attribute.null_value.safeGet(); const CollectionsHolder & child_key_to_parent_key_maps = std::get>(hierarchical_attribute.containers); @@ -710,7 +710,7 @@ ColumnUInt8::Ptr HashedDictionary::isInHie std::optional null_value; if (!dictionary_attribute.null_value.isNull()) - null_value = dictionary_attribute.null_value.get(); + null_value = dictionary_attribute.null_value.safeGet(); const CollectionsHolder & child_key_to_parent_key_maps = std::get>(hierarchical_attribute.containers); @@ -1004,13 +1004,13 @@ void HashedDictionary::blockToAttributes(c if constexpr (std::is_same_v) { - String & value_to_insert = column_value_to_insert.get(); + String & value_to_insert = column_value_to_insert.safeGet(); StringRef arena_value = copyStringInArena(*string_arenas[shard], value_to_insert); container.insert({key, arena_value}); } else { - auto value_to_insert = static_cast(column_value_to_insert.get()); + auto value_to_insert = static_cast(column_value_to_insert.safeGet()); container.insert({key, value_to_insert}); } diff --git a/src/Dictionaries/HierarchyDictionariesUtils.cpp b/src/Dictionaries/HierarchyDictionariesUtils.cpp index e1119982a34..de532ade26d 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.cpp +++ b/src/Dictionaries/HierarchyDictionariesUtils.cpp @@ -50,7 +50,7 @@ namespace std::optional null_value; if (!hierarchical_attribute.null_value.isNull()) - null_value = hierarchical_attribute.null_value.get(); + null_value = hierarchical_attribute.null_value.safeGet(); ColumnPtr key_to_request_column = ColumnVector::create(); auto * key_to_request_column_typed = static_cast *>(key_to_request_column->assumeMutable().get()); @@ -190,7 +190,7 @@ ColumnPtr getKeysHierarchyDefaultImplementation( std::optional null_value; if (!hierarchical_attribute.null_value.isNull()) - null_value = hierarchical_attribute.null_value.get(); + null_value = hierarchical_attribute.null_value.safeGet(); auto get_parent_key_func = [&](auto & key) { @@ -252,7 +252,7 @@ ColumnUInt8::Ptr getKeysIsInHierarchyDefaultImplementation( std::optional null_value; if (!hierarchical_attribute.null_value.isNull()) - null_value = hierarchical_attribute.null_value.get(); + null_value = hierarchical_attribute.null_value.safeGet(); auto get_parent_key_func = [&](auto & key) { diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 41fafcc162b..4f9e991752f 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -613,14 +613,14 @@ void IPAddressDictionary::calculateBytesAllocated() template void IPAddressDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { - attribute.null_values = null_value.isNull() ? T{} : T(null_value.get()); + attribute.null_values = null_value.isNull() ? T{} : T(null_value.safeGet()); attribute.maps.emplace>(); } template <> void IPAddressDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { - attribute.null_values = null_value.isNull() ? String() : null_value.get(); + attribute.null_values = null_value.isNull() ? String() : null_value.safeGet(); attribute.maps.emplace>(); attribute.string_arena = std::make_unique(); } @@ -976,13 +976,13 @@ void IPAddressDictionary::setAttributeValue(Attribute & attribute, const Field & if constexpr (std::is_same_v) { - const auto & string = value.get(); + const auto & string = value.safeGet(); const auto * string_in_arena = attribute.string_arena->insert(string.data(), string.size()); setAttributeValueImpl(attribute, StringRef{string_in_arena, string.size()}); } else { - setAttributeValueImpl(attribute, static_cast(value.get())); + setAttributeValueImpl(attribute, static_cast(value.safeGet())); } }; diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index 46910fa9f6a..c30a6f90e44 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -1,7 +1,6 @@ #include "MongoDBDictionarySource.h" #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" -#include "registerDictionaries.h" #include #include @@ -233,7 +232,7 @@ QueryPipeline MongoDBDictionarySource::loadKeys(const Columns & key_columns, con } case AttributeUnderlyingType::String: { - String loaded_str((*key_columns[attribute_index])[row_idx].get()); + String loaded_str((*key_columns[attribute_index])[row_idx].safeGet()); /// Convert string to ObjectID if (key_attribute.is_object_id) { diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index dfc920623e3..ff29ca1f6b8 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -141,7 +141,7 @@ ColumnPtr IPolygonDictionary::getColumn( { getItemsShortCircuitImpl( requested_key_points, - [&](size_t row) { return (*attribute_values_column)[row].get(); }, + [&](size_t row) { return (*attribute_values_column)[row].safeGet(); }, [&](Array & value) { result_column_typed.insert(value); }, default_mask.value()); } @@ -149,7 +149,7 @@ ColumnPtr IPolygonDictionary::getColumn( { getItemsImpl( requested_key_points, - [&](size_t row) { return (*attribute_values_column)[row].get(); }, + [&](size_t row) { return (*attribute_values_column)[row].safeGet(); }, [&](Array & value) { result_column_typed.insert(value); }, default_value_provider.value()); } @@ -432,16 +432,16 @@ void IPolygonDictionary::getItemsImpl( } else if constexpr (std::is_same_v) { - set_value(default_value.get()); + set_value(default_value.safeGet()); } else if constexpr (std::is_same_v) { - auto default_value_string = default_value.get(); + auto default_value_string = default_value.safeGet(); set_value(default_value_string); } else { - set_value(default_value.get>()); + set_value(default_value.safeGet>()); } } } diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index fc6c98990d0..c264b480bcb 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -906,13 +906,13 @@ void RangeHashedDictionary::setAttributeValue(Attribute & a if constexpr (std::is_same_v) { - const auto & string = value.get(); + const auto & string = value.safeGet(); StringRef string_ref = copyStringInArena(string_arena, string); value_to_insert = string_ref; } else { - value_to_insert = static_cast(value.get()); + value_to_insert = static_cast(value.safeGet()); } container.back() = value_to_insert; diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 1736cdff306..9db639a0ca4 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -1,7 +1,6 @@ #include "RedisDictionarySource.h" #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" -#include "registerDictionaries.h" #include #include @@ -160,7 +159,7 @@ namespace DB if (isInteger(type)) key << DB::toString(key_columns[i]->get64(row)); else if (isString(type)) - key << (*key_columns[i])[row].get(); + key << (*key_columns[i])[row].safeGet(); else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected type of key in Redis dictionary"); } diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index fd43f31a009..67bae372f87 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -113,7 +113,7 @@ std::string getOrCreateDiskFromDiskAST(const ASTPtr & disk_function, ContextPtr FlattenDiskConfigurationVisitor::Data data{context, attach}; FlattenDiskConfigurationVisitor{data}.visit(ast); - auto disk_name = assert_cast(*ast).value.get(); + auto disk_name = assert_cast(*ast).value.safeGet(); LOG_TRACE(getLogger("getOrCreateDiskFromDiskAST"), "Result disk name: {}", disk_name); return disk_name; } diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index a7bd398cdaa..5dbafe48c3a 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1196,7 +1196,7 @@ struct ToYearImpl { if (point.getType() != Field::Types::UInt64) return std::nullopt; - auto year = point.get(); + auto year = point.safeGet(); if (year < DATE_LUT_MIN_YEAR || year >= DATE_LUT_MAX_YEAR) return std::nullopt; const DateLUTImpl & date_lut = DateLUT::instance("UTC"); @@ -2001,7 +2001,7 @@ struct ToYYYYMMImpl { if (point.getType() != Field::Types::UInt64) return std::nullopt; - auto year_month = point.get(); + auto year_month = point.safeGet(); auto year = year_month / 100; auto month = year_month % 100; diff --git a/src/Functions/FunctionsConsistentHashing.h b/src/Functions/FunctionsConsistentHashing.h index 306b6395dc5..210bb69e16d 100644 --- a/src/Functions/FunctionsConsistentHashing.h +++ b/src/Functions/FunctionsConsistentHashing.h @@ -101,9 +101,9 @@ private: BucketsType num_buckets; if (buckets_field.getType() == Field::Types::Int64) - num_buckets = checkBucketsRange(buckets_field.get()); + num_buckets = checkBucketsRange(buckets_field.safeGet()); else if (buckets_field.getType() == Field::Types::UInt64) - num_buckets = checkBucketsRange(buckets_field.get()); + num_buckets = checkBucketsRange(buckets_field.safeGet()); else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of the second argument of function {}", diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 675283d011e..0760b929652 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -117,7 +117,7 @@ UInt32 extractToDecimalScale(const ColumnWithTypeAndName & named_column) Field field; named_column.column->get(0, field); - return static_cast(field.get()); + return static_cast(field.safeGet()); } @@ -2604,8 +2604,8 @@ struct ToNumberMonotonicity if (left.isNull() || right.isNull()) return {}; - Float64 left_float = left.get(); - Float64 right_float = right.get(); + Float64 left_float = left.safeGet(); + Float64 right_float = right.safeGet(); if (left_float >= static_cast(std::numeric_limits::min()) && left_float <= static_cast(std::numeric_limits::max()) @@ -2633,11 +2633,11 @@ struct ToNumberMonotonicity const bool left_in_first_half = left.isNull() ? from_is_unsigned - : (left.get() >= 0); + : (left.safeGet() >= 0); const bool right_in_first_half = right.isNull() ? !from_is_unsigned - : (right.get() >= 0); + : (right.safeGet() >= 0); /// Size of type is the same. if (size_of_from == size_of_to) @@ -2675,7 +2675,7 @@ struct ToNumberMonotonicity return {}; /// Function cannot be monotonic when left and right are not on the same ranges. - if (divideByRangeOfType(left.get()) != divideByRangeOfType(right.get())) + if (divideByRangeOfType(left.safeGet()) != divideByRangeOfType(right.safeGet())) return {}; if (to_is_unsigned) @@ -2683,7 +2683,7 @@ struct ToNumberMonotonicity else { // If To is signed, it's possible that the signedness is different after conversion. So we check it explicitly. - const bool is_monotonic = (T(left.get()) >= 0) == (T(right.get()) >= 0); + const bool is_monotonic = (T(left.safeGet()) >= 0) == (T(right.safeGet()) >= 0); return { .is_monotonic = is_monotonic }; } @@ -2707,13 +2707,13 @@ struct ToDateMonotonicity } else if ( ((left.getType() == Field::Types::UInt64 || left.isNull()) && (right.getType() == Field::Types::UInt64 || right.isNull()) - && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF))) + && ((left.isNull() || left.safeGet() < 0xFFFF) && (right.isNull() || right.safeGet() >= 0xFFFF))) || ((left.getType() == Field::Types::Int64 || left.isNull()) && (right.getType() == Field::Types::Int64 || right.isNull()) - && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF))) + && ((left.isNull() || left.safeGet() < 0xFFFF) && (right.isNull() || right.safeGet() >= 0xFFFF))) || (( (left.getType() == Field::Types::Float64 || left.isNull()) && (right.getType() == Field::Types::Float64 || right.isNull()) - && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF)))) + && ((left.isNull() || left.safeGet() < 0xFFFF) && (right.isNull() || right.safeGet() >= 0xFFFF)))) || !isNativeNumber(type)) { return {}; @@ -2768,16 +2768,16 @@ struct ToStringMonotonicity if (left.getType() == Field::Types::UInt64 && right.getType() == Field::Types::UInt64) { - return (left.get() == 0 && right.get() == 0) - || (floor(log10(left.get())) == floor(log10(right.get()))) + return (left.safeGet() == 0 && right.safeGet() == 0) + || (floor(log10(left.safeGet())) == floor(log10(right.safeGet()))) ? positive : not_monotonic; } if (left.getType() == Field::Types::Int64 && right.getType() == Field::Types::Int64) { - return (left.get() == 0 && right.get() == 0) - || (left.get() > 0 && right.get() > 0 && floor(log10(left.get())) == floor(log10(right.get()))) + return (left.safeGet() == 0 && right.safeGet() == 0) + || (left.safeGet() > 0 && right.safeGet() > 0 && floor(log10(left.safeGet())) == floor(log10(right.safeGet()))) ? positive : not_monotonic; } @@ -4673,7 +4673,7 @@ private: return [function_name] ( ColumnsWithTypeAndName & arguments, const DataTypePtr & res_type, const ColumnNullable * nullable_col, size_t /*input_rows_count*/) { - using ColumnEnumType = EnumType::ColumnType; + using ColumnEnumType = typename EnumType::ColumnType; const auto & first_col = arguments.front().column.get(); const auto & first_type = arguments.front().type.get(); diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index c35df8ba72d..1eaf0d1609a 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -243,7 +243,7 @@ private: } case MoveType::Index: { - Int64 index = (*arguments[j + 1].column)[row].get(); + Int64 index = (*arguments[j + 1].column)[row].safeGet(); if (!moveToElementByIndex(res_element, static_cast(index), key)) return false; break; diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index 65d7473b945..ff0cff09c9e 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -701,11 +701,11 @@ ColumnPtr FunctionAnyArityLogical::getConstantResultForNonConstArgum bool constant_value_bool = false; if (field_type == Field::Types::Float64) - constant_value_bool = static_cast(constant_field_value.get()); + constant_value_bool = static_cast(constant_field_value.safeGet()); else if (field_type == Field::Types::Int64) - constant_value_bool = static_cast(constant_field_value.get()); + constant_value_bool = static_cast(constant_field_value.safeGet()); else if (field_type == Field::Types::UInt64) - constant_value_bool = static_cast(constant_field_value.get()); + constant_value_bool = static_cast(constant_field_value.safeGet()); has_true_constant = has_true_constant || constant_value_bool; has_false_constant = has_false_constant || !constant_value_bool; diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 6b65a5feaec..ed7fe1a5de1 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -500,7 +500,7 @@ inline Scale getScaleArg(const ColumnConst* scale_col) { const auto & scale_field = scale_col->getField(); - Int64 scale64 = scale_field.get(); + Int64 scale64 = scale_field.safeGet(); validateScale(scale64); return scale64; @@ -632,7 +632,7 @@ public: Scale raw_scale = scale64; DecimalRoundingImpl::applyOne(value_col_typed->getElement(i), value_col_typed->getScale(), - reinterpret_cast::NativeT&>(col_res->getElement(i)), raw_scale); + reinterpret_cast::NativeT&>(col_res->getElement(i)), raw_scale); } } } @@ -854,7 +854,7 @@ private: using ValueType = typename Container::value_type; std::vector boundary_values(boundaries.size()); for (size_t i = 0; i < boundaries.size(); ++i) - boundary_values[i] = static_cast(boundaries[i].get()); + boundary_values[i] = static_cast(boundaries[i].safeGet()); ::sort(boundary_values.begin(), boundary_values.end()); boundary_values.erase(std::unique(boundary_values.begin(), boundary_values.end()), boundary_values.end()); diff --git a/src/Functions/IFunctionCustomWeek.h b/src/Functions/IFunctionCustomWeek.h index 51542c9cab1..ba0baa35819 100644 --- a/src/Functions/IFunctionCustomWeek.h +++ b/src/Functions/IFunctionCustomWeek.h @@ -50,15 +50,15 @@ public: if (checkAndGetDataType(&type)) { - return Transform::FactorTransform::execute(UInt16(left.get()), date_lut) - == Transform::FactorTransform::execute(UInt16(right.get()), date_lut) + return Transform::FactorTransform::execute(UInt16(left.safeGet()), date_lut) + == Transform::FactorTransform::execute(UInt16(right.safeGet()), date_lut) ? is_monotonic : is_not_monotonic; } else { - return Transform::FactorTransform::execute(UInt32(left.get()), date_lut) - == Transform::FactorTransform::execute(UInt32(right.get()), date_lut) + return Transform::FactorTransform::execute(UInt32(left.safeGet()), date_lut) + == Transform::FactorTransform::execute(UInt32(right.safeGet()), date_lut) ? is_monotonic : is_not_monotonic; } diff --git a/src/Functions/IFunctionDateOrDateTime.h b/src/Functions/IFunctionDateOrDateTime.h index 762b79bfafc..899aa2c305d 100644 --- a/src/Functions/IFunctionDateOrDateTime.h +++ b/src/Functions/IFunctionDateOrDateTime.h @@ -72,30 +72,30 @@ public: if (checkAndGetDataType(type_ptr)) { - return Transform::FactorTransform::execute(UInt16(left.get()), *date_lut) - == Transform::FactorTransform::execute(UInt16(right.get()), *date_lut) + return Transform::FactorTransform::execute(UInt16(left.safeGet()), *date_lut) + == Transform::FactorTransform::execute(UInt16(right.safeGet()), *date_lut) ? is_monotonic : is_not_monotonic; } else if (checkAndGetDataType(type_ptr)) { - return Transform::FactorTransform::execute(Int32(left.get()), *date_lut) - == Transform::FactorTransform::execute(Int32(right.get()), *date_lut) + return Transform::FactorTransform::execute(Int32(left.safeGet()), *date_lut) + == Transform::FactorTransform::execute(Int32(right.safeGet()), *date_lut) ? is_monotonic : is_not_monotonic; } else if (checkAndGetDataType(type_ptr)) { - return Transform::FactorTransform::execute(UInt32(left.get()), *date_lut) - == Transform::FactorTransform::execute(UInt32(right.get()), *date_lut) + return Transform::FactorTransform::execute(UInt32(left.safeGet()), *date_lut) + == Transform::FactorTransform::execute(UInt32(right.safeGet()), *date_lut) ? is_monotonic : is_not_monotonic; } else { assert(checkAndGetDataType(type_ptr)); - const auto & left_date_time = left.get(); + const auto & left_date_time = left.safeGet(); TransformDateTime64 transformer_left(left_date_time.getScale()); - const auto & right_date_time = right.get(); + const auto & right_date_time = right.safeGet(); TransformDateTime64 transformer_right(right_date_time.getScale()); return transformer_left.execute(left_date_time.getValue(), *date_lut) diff --git a/src/Functions/JSONPath/Parsers/ParserJSONPathRange.cpp b/src/Functions/JSONPath/Parsers/ParserJSONPathRange.cpp index fb74018b330..84ac0ff08f3 100644 --- a/src/Functions/JSONPath/Parsers/ParserJSONPathRange.cpp +++ b/src/Functions/JSONPath/Parsers/ParserJSONPathRange.cpp @@ -46,7 +46,7 @@ bool ParserJSONPathRange::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte { return false; } - range_indices.first = static_cast(number_ptr->as()->value.get()); + range_indices.first = static_cast(number_ptr->as()->value.safeGet()); if (pos->type == TokenType::Comma || pos->type == TokenType::ClosingSquareBracket) { @@ -63,7 +63,7 @@ bool ParserJSONPathRange::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte { return false; } - range_indices.second = static_cast(number_ptr->as()->value.get()); + range_indices.second = static_cast(number_ptr->as()->value.safeGet()); } else { diff --git a/src/Functions/MultiMatchAllIndicesImpl.h b/src/Functions/MultiMatchAllIndicesImpl.h index 3e9c8fba215..c1a2fb6be2d 100644 --- a/src/Functions/MultiMatchAllIndicesImpl.h +++ b/src/Functions/MultiMatchAllIndicesImpl.h @@ -75,7 +75,7 @@ struct MultiMatchAllIndicesImpl std::vector needles; needles.reserve(needles_arr.size()); for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + needles.emplace_back(needle.safeGet()); checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index 20b2150048b..ce6a054c064 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -89,7 +89,7 @@ struct MultiMatchAnyImpl std::vector needles; needles.reserve(needles_arr.size()); for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + needles.emplace_back(needle.safeGet()); checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); diff --git a/src/Functions/MultiSearchAllPositionsImpl.h b/src/Functions/MultiSearchAllPositionsImpl.h index cfe60e51bcd..6c2cd215638 100644 --- a/src/Functions/MultiSearchAllPositionsImpl.h +++ b/src/Functions/MultiSearchAllPositionsImpl.h @@ -33,7 +33,7 @@ struct MultiSearchAllPositionsImpl std::vector needles; needles.reserve(needles_arr.size()); for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + needles.emplace_back(needle.safeGet()); auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 { diff --git a/src/Functions/MultiSearchFirstIndexImpl.h b/src/Functions/MultiSearchFirstIndexImpl.h index 36a5fd514d9..3c6ec8ead44 100644 --- a/src/Functions/MultiSearchFirstIndexImpl.h +++ b/src/Functions/MultiSearchFirstIndexImpl.h @@ -44,7 +44,7 @@ struct MultiSearchFirstIndexImpl std::vector needles; needles.reserve(needles_arr.size()); for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + needles.emplace_back(needle.safeGet()); auto searcher = Impl::createMultiSearcherInBigHaystack(needles); diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index ccdd82a0ee5..b5c68ad664d 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -44,7 +44,7 @@ struct MultiSearchFirstPositionImpl std::vector needles; needles.reserve(needles_arr.size()); for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + needles.emplace_back(needle.safeGet()); auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 { diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index 467cc96a95f..3eb8e6fb627 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -44,7 +44,7 @@ struct MultiSearchImpl std::vector needles; needles.reserve(needles_arr.size()); for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + needles.emplace_back(needle.safeGet()); auto searcher = Impl::createMultiSearcherInBigHaystack(needles); diff --git a/src/Functions/URL/cutURLParameter.cpp b/src/Functions/URL/cutURLParameter.cpp index 3ab9cad1ea7..4439e79e962 100644 --- a/src/Functions/URL/cutURLParameter.cpp +++ b/src/Functions/URL/cutURLParameter.cpp @@ -156,7 +156,7 @@ public: for (size_t j = 0; j < num_needles; ++j) { auto field = col_needle_const_array->getData()[j]; - cutURL(res_data, field.get(), res_offset, cur_res_offset); + cutURL(res_data, field.safeGet(), res_offset, cur_res_offset); } } else diff --git a/src/Functions/array/arrayElement.cpp b/src/Functions/array/arrayElement.cpp index 227b29d5d9f..81f3f97979b 100644 --- a/src/Functions/array/arrayElement.cpp +++ b/src/Functions/array/arrayElement.cpp @@ -904,10 +904,10 @@ ColumnPtr FunctionArrayElement::executeNumberConst( return nullptr; if (index.getType() == Field::Types::UInt64 - || (index.getType() == Field::Types::Int64 && index.get() >= 0)) + || (index.getType() == Field::Types::Int64 && index.safeGet() >= 0)) { ArrayElementNumImpl::template vectorConst( - col_nested->getData(), col_array->getOffsets(), index.get() - 1, col_res_vec->getData(), builder); + col_nested->getData(), col_array->getOffsets(), index.safeGet() - 1, col_res_vec->getData(), builder); } else if (index.getType() == Field::Types::Int64) { @@ -972,14 +972,14 @@ FunctionArrayElement::executeStringConst(const ColumnsWithTypeAndName & argument auto col_res = ColumnString::create(); if (index.getType() == Field::Types::UInt64 - || (index.getType() == Field::Types::Int64 && index.get() >= 0)) + || (index.getType() == Field::Types::Int64 && index.safeGet() >= 0)) { if (builder) ArrayElementStringImpl::vectorConst( col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(), - index.get() - 1, + index.safeGet() - 1, col_res->getChars(), col_res->getOffsets(), builder); @@ -988,7 +988,7 @@ FunctionArrayElement::executeStringConst(const ColumnsWithTypeAndName & argument col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(), - index.get() - 1, + index.safeGet() - 1, col_res->getChars(), col_res->getOffsets(), builder); @@ -1000,7 +1000,7 @@ FunctionArrayElement::executeStringConst(const ColumnsWithTypeAndName & argument col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(), - -(UInt64(index.get()) + 1), + -(UInt64(index.safeGet()) + 1), col_res->getChars(), col_res->getOffsets(), builder); @@ -1009,7 +1009,7 @@ FunctionArrayElement::executeStringConst(const ColumnsWithTypeAndName & argument col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(), - -(UInt64(index.get()) + 1), + -(UInt64(index.safeGet()) + 1), col_res->getChars(), col_res->getOffsets(), builder); @@ -1046,7 +1046,7 @@ ColumnPtr FunctionArrayElement::executeArrayStringConst( auto res_offsets = ColumnArray::ColumnOffsets::create(); auto res_string_null_map = col_nullable ? ColumnUInt8::create() : nullptr; if (index.getType() == Field::Types::UInt64 - || (index.getType() == Field::Types::Int64 && index.get() >= 0)) + || (index.getType() == Field::Types::Int64 && index.safeGet() >= 0)) { if (col_nullable) ArrayElementArrayStringImpl::vectorConst( @@ -1055,7 +1055,7 @@ ColumnPtr FunctionArrayElement::executeArrayStringConst( col_nested_array->getOffsets(), col_nested_elem->getOffsets(), &string_null_map->getData(), - index.get() - 1, + index.safeGet() - 1, res_string->getChars(), res_offsets->getData(), res_string->getOffsets(), @@ -1068,7 +1068,7 @@ ColumnPtr FunctionArrayElement::executeArrayStringConst( col_nested_array->getOffsets(), col_nested_elem->getOffsets(), nullptr, - index.get() - 1, + index.safeGet() - 1, res_string->getChars(), res_offsets->getData(), res_string->getOffsets(), @@ -1084,7 +1084,7 @@ ColumnPtr FunctionArrayElement::executeArrayStringConst( col_nested_array->getOffsets(), col_nested_elem->getOffsets(), &string_null_map->getData(), - -(UInt64(index.get()) + 1), + -(UInt64(index.safeGet()) + 1), res_string->getChars(), res_offsets->getData(), res_string->getOffsets(), @@ -1097,7 +1097,7 @@ ColumnPtr FunctionArrayElement::executeArrayStringConst( col_nested_array->getOffsets(), col_nested_elem->getOffsets(), nullptr, - -(UInt64(index.get()) + 1), + -(UInt64(index.safeGet()) + 1), res_string->getChars(), res_offsets->getData(), res_string->getOffsets(), @@ -1153,7 +1153,7 @@ ColumnPtr FunctionArrayElement::executeArrayNumberConst( auto & res_offsets = res_array->getOffsets(); NullMap * res_null_map = res_nullable ? &res_nullable->getNullMapData() : nullptr; - if (index.getType() == Field::Types::UInt64 || (index.getType() == Field::Types::Int64 && index.get() >= 0)) + if (index.getType() == Field::Types::UInt64 || (index.getType() == Field::Types::Int64 && index.safeGet() >= 0)) { if (col_nullable) ArrayElementArrayNumImpl::template vectorConst( @@ -1161,7 +1161,7 @@ ColumnPtr FunctionArrayElement::executeArrayNumberConst( col_array->getOffsets(), col_nested_array->getOffsets(), null_map, - index.get() - 1, + index.safeGet() - 1, res_data->getData(), res_offsets, res_null_map, @@ -1172,7 +1172,7 @@ ColumnPtr FunctionArrayElement::executeArrayNumberConst( col_array->getOffsets(), col_nested_array->getOffsets(), null_map, - index.get() - 1, + index.safeGet() - 1, res_data->getData(), res_offsets, res_null_map, @@ -1392,12 +1392,12 @@ ColumnPtr FunctionArrayElement::executeGenericConst( auto col_res = col_nested.cloneEmpty(); if (index.getType() == Field::Types::UInt64 - || (index.getType() == Field::Types::Int64 && index.get() >= 0)) + || (index.getType() == Field::Types::Int64 && index.safeGet() >= 0)) ArrayElementGenericImpl::vectorConst( - col_nested, col_array->getOffsets(), index.get() - 1, *col_res, builder); + col_nested, col_array->getOffsets(), index.safeGet() - 1, *col_res, builder); else if (index.getType() == Field::Types::Int64) ArrayElementGenericImpl::vectorConst( - col_nested, col_array->getOffsets(), -(static_cast(index.get() + 1)), *col_res, builder); + col_nested, col_array->getOffsets(), -(static_cast(index.safeGet() + 1)), *col_res, builder); else throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal type of array index"); @@ -1789,7 +1789,7 @@ bool FunctionArrayElement::matchKeyToIndexStringConst( using DataColumn = std::decay_t; if (index.getType() != Field::Types::String) return false; - MatcherStringConst matcher{data_column, index.get()}; + MatcherStringConst matcher{data_column, index.safeGet()}; executeMatchKeyToIndex(offsets, matched_idxs, matcher); return true; }); diff --git a/src/Functions/array/mapOp.cpp b/src/Functions/array/mapOp.cpp index 86797cb5db0..614b01c2ac8 100644 --- a/src/Functions/array/mapOp.cpp +++ b/src/Functions/array/mapOp.cpp @@ -237,7 +237,7 @@ private: } arg.val_column->get(offset + j, temp_val); - ValType value = temp_val.get(); + ValType value = temp_val.safeGet(); if constexpr (op_type == OpTypes::ADD) { diff --git a/src/Functions/getClientHTTPHeader.cpp b/src/Functions/getClientHTTPHeader.cpp index 140f39d03b8..50a6275fc82 100644 --- a/src/Functions/getClientHTTPHeader.cpp +++ b/src/Functions/getClientHTTPHeader.cpp @@ -58,7 +58,7 @@ public: { Field header; source->get(row, header); - if (auto it = client_info.http_headers.find(header.get()); it != client_info.http_headers.end()) + if (auto it = client_info.http_headers.find(header.safeGet()); it != client_info.http_headers.end()) result->insert(it->second); else result->insertDefault(); diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index c4b675fcf6c..14b8b70b22c 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -200,7 +200,7 @@ public: if (value.isNull()) continue; - if (value.get() == 0) + if (value.safeGet() == 0) continue; instruction.condition_always_true = true; diff --git a/src/Functions/nested.cpp b/src/Functions/nested.cpp index bdaf57d65c9..85c342b5e7c 100644 --- a/src/Functions/nested.cpp +++ b/src/Functions/nested.cpp @@ -145,7 +145,7 @@ private: if (nested_names_field.getType() != Field::Types::Array) return {}; - const auto & nested_names_array = nested_names_field.get(); + const auto & nested_names_array = nested_names_field.safeGet(); Names nested_names; nested_names.reserve(nested_names_array.size()); @@ -155,7 +155,7 @@ private: if (nested_name_field.getType() != Field::Types::String) return {}; - nested_names.push_back(nested_name_field.get()); + nested_names.push_back(nested_name_field.safeGet()); } return nested_names; diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index a794cdbcf05..59040bf1fea 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -294,21 +294,21 @@ void RequestSettings::finishInit(const DB::Settings & settings, bool validate_se /// to avoid losing token bucket state on every config reload, /// which could lead to exceeding limit for short time. /// But it is good enough unless very high `burst` values are used. - if (UInt64 max_get_rps = isChanged("max_get_rps") ? get("max_get_rps").get() : settings.s3_max_get_rps) + if (UInt64 max_get_rps = isChanged("max_get_rps") ? get("max_get_rps").safeGet() : settings.s3_max_get_rps) { size_t default_max_get_burst = settings.s3_max_get_burst ? settings.s3_max_get_burst : (Throttler::default_burst_seconds * max_get_rps); - size_t max_get_burst = isChanged("max_get_burts") ? get("max_get_burst").get() : default_max_get_burst; + size_t max_get_burst = isChanged("max_get_burts") ? get("max_get_burst").safeGet() : default_max_get_burst; get_request_throttler = std::make_shared(max_get_rps, max_get_burst); } - if (UInt64 max_put_rps = isChanged("max_put_rps") ? get("max_put_rps").get() : settings.s3_max_put_rps) + if (UInt64 max_put_rps = isChanged("max_put_rps") ? get("max_put_rps").safeGet() : settings.s3_max_put_rps) { size_t default_max_put_burst = settings.s3_max_put_burst ? settings.s3_max_put_burst : (Throttler::default_burst_seconds * max_put_rps); - size_t max_put_burst = isChanged("max_put_burts") ? get("max_put_burst").get() : default_max_put_burst; + size_t max_put_burst = isChanged("max_put_burts") ? get("max_put_burst").safeGet() : default_max_put_burst; put_request_throttler = std::make_shared(max_put_rps, max_put_burst); } } diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index e1b7e92ee5d..368eb8174f0 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -131,7 +131,7 @@ static Block createBlockFromCollection(const Collection & collection, const Data throw Exception(ErrorCodes::INCORRECT_ELEMENT_OF_SET, "Invalid type in set. Expected tuple, got {}", String(value.getTypeName())); - const auto & tuple = value.template get(); + const auto & tuple = value.template safeGet(); size_t tuple_size = tuple.size(); if (tuple_size != columns_num) throw Exception(ErrorCodes::INCORRECT_ELEMENT_OF_SET, "Incorrect size of tuple in set: {} instead of {}", @@ -233,7 +233,7 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, Co "Invalid type of set. Expected tuple, got {}", function_result.getTypeName()); - tuple = &function_result.get(); + tuple = &function_result.safeGet(); } /// Tuple can be represented as a literal in AST. @@ -246,7 +246,7 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, Co "Invalid type in set. Expected tuple, got {}", literal->value.getTypeName()); - tuple = &literal->value.get(); + tuple = &literal->value.safeGet(); } assert(tuple || func); @@ -332,14 +332,14 @@ Block createBlockForSet( if (type_index == TypeIndex::Tuple) { const DataTypes & value_types = assert_cast(right_arg_type.get())->getElements(); - block = createBlockFromCollection(right_arg_value.get(), value_types, set_element_types, tranform_null_in); + block = createBlockFromCollection(right_arg_value.safeGet(), value_types, set_element_types, tranform_null_in); } else if (type_index == TypeIndex::Array) { const auto* right_arg_array_type = assert_cast(right_arg_type.get()); - size_t right_arg_array_size = right_arg_value.get().size(); + size_t right_arg_array_size = right_arg_value.safeGet().size(); DataTypes value_types(right_arg_array_size, right_arg_array_type->getNestedType()); - block = createBlockFromCollection(right_arg_value.get(), value_types, set_element_types, tranform_null_in); + block = createBlockFromCollection(right_arg_value.safeGet(), value_types, set_element_types, tranform_null_in); } else throw_unsupported_type(right_arg_type); diff --git a/src/Interpreters/AddDefaultDatabaseVisitor.h b/src/Interpreters/AddDefaultDatabaseVisitor.h index 356bffa75e9..302a5e55c53 100644 --- a/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -201,7 +201,7 @@ private: if (literal_value.getType() != Field::Types::String) continue; - auto dictionary_name = literal_value.get(); + auto dictionary_name = literal_value.safeGet(); auto qualified_dictionary_name = context->getExternalDictionariesLoader().qualifyDictionaryNameWithDatabase(dictionary_name, context); literal_value = qualified_dictionary_name.getFullName(); } diff --git a/src/Interpreters/ComparisonTupleEliminationVisitor.cpp b/src/Interpreters/ComparisonTupleEliminationVisitor.cpp index 4f06f345b96..b9f7f37b338 100644 --- a/src/Interpreters/ComparisonTupleEliminationVisitor.cpp +++ b/src/Interpreters/ComparisonTupleEliminationVisitor.cpp @@ -22,7 +22,7 @@ ASTs splitTuple(const ASTPtr & node) if (const auto * literal = node->as(); literal && literal->value.getType() == Field::Types::Tuple) { ASTs result; - const auto & tuple = literal->value.get(); + const auto & tuple = literal->value.safeGet(); for (const auto & child : tuple) result.emplace_back(std::make_shared(child)); return result; diff --git a/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp b/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp index 084bb0a1bb9..220355e0741 100644 --- a/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp +++ b/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp @@ -45,7 +45,7 @@ void ConvertFunctionOrLikeData::visit(ASTFunction & function, ASTPtr &) if (!identifier || !literal || literal->value.getType() != Field::Types::String) continue; - String regexp = likePatternToRegexp(literal->value.get()); + String regexp = likePatternToRegexp(literal->value.safeGet()); /// Case insensitive. Works with UTF-8 as well. if (is_ilike) regexp = "(?i)" + regexp; @@ -61,7 +61,7 @@ void ConvertFunctionOrLikeData::visit(ASTFunction & function, ASTPtr &) match->arguments->children.push_back(it->second); unique_elems.push_back(std::move(match)); } - it->second->value.get().push_back(regexp); + it->second->value.safeGet().push_back(regexp); } } diff --git a/src/Interpreters/ConvertStringsToEnumVisitor.cpp b/src/Interpreters/ConvertStringsToEnumVisitor.cpp index 7cc95dc521b..d35baa92900 100644 --- a/src/Interpreters/ConvertStringsToEnumVisitor.cpp +++ b/src/Interpreters/ConvertStringsToEnumVisitor.cpp @@ -33,8 +33,8 @@ String makeStringsEnum(const std::set & values) void changeIfArguments(ASTPtr & first, ASTPtr & second) { - String first_value = first->as()->value.get(); - String second_value = second->as()->value.get(); + String first_value = first->as()->value.safeGet(); + String second_value = second->as()->value.safeGet(); std::set values; values.insert(first_value); @@ -59,9 +59,9 @@ void changeTransformArguments(ASTPtr & array_to, ASTPtr & other) { std::set values; - for (const auto & item : array_to->as()->value.get()) - values.insert(item.get()); - values.insert(other->as()->value.get()); + for (const auto & item : array_to->as()->value.safeGet()) + values.insert(item.safeGet()); + values.insert(other->as()->value.safeGet()); String enum_string = makeStringsEnum(values); @@ -168,7 +168,7 @@ void ConvertStringsToEnumMatcher::visit(ASTFunction & function_node, Data & data if (literal_to->value.getTypeName() != "Array" || literal_other->value.getTypeName() != "String") return; - Array array_to = literal_to->value.get(); + Array array_to = literal_to->value.safeGet(); if (array_to.empty()) return; diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index fa197d59c13..6e08dd5e2cc 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -538,7 +538,7 @@ void DatabaseReplicatedTask::createSyncedNodeIfNeed(const ZooKeeperPtr & zookeep /// Bool type is really weird, sometimes it's Bool and sometimes it's UInt64... assert(value.getType() == Field::Types::Bool || value.getType() == Field::Types::UInt64); - if (!value.get()) + if (!value.safeGet()) return; zookeeper->createIfNotExists(getSyncedNodePath(), ""); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 971f90bd3cd..15c6aba8b62 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -689,7 +689,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( throw Exception(ErrorCodes::LOGICAL_ERROR, "Neither default value expression nor type is provided for a column"); if (col_decl.comment) - column.comment = col_decl.comment->as().value.get(); + column.comment = col_decl.comment->as().value.safeGet(); if (col_decl.codec) { @@ -1875,7 +1875,7 @@ void InterpreterCreateQuery::prepareOnClusterQuery(ASTCreateQuery & create, Cont if (has_explicit_zk_path_arg) { - String zk_path = create.storage->engine->arguments->children[0]->as()->value.get(); + String zk_path = create.storage->engine->arguments->children[0]->as()->value.safeGet(); Macros::MacroExpansionInfo info; info.table_id.uuid = create.uuid; info.ignore_unknown = true; diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index bedd9cb4a80..c820f999e0c 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -332,7 +332,7 @@ ExplainSettings checkAndGetSettings(const ASTPtr & ast_settings) if (settings.hasBooleanSetting(change.name)) { - auto value = change.value.get(); + auto value = change.value.safeGet(); if (value > 1) throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid value {} for setting \"{}\". " "Expected boolean type", value, change.name); @@ -341,7 +341,7 @@ ExplainSettings checkAndGetSettings(const ASTPtr & ast_settings) } else { - auto value = change.value.get(); + auto value = change.value.safeGet(); settings.setIntegerSetting(change.name, value); } } diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 7eb487ba7b3..2c579f3b468 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -334,7 +334,7 @@ BlockIO InterpreterKillQueryQuery::execute() for (size_t i = 0; i < moves_block.rows(); ++i) { table_id = StorageID{database_col.getDataAt(i).toString(), table_col.getDataAt(i).toString()}; - auto task_uuid = task_uuid_col[i].get(); + auto task_uuid = task_uuid_col[i].safeGet(); CancellationCode code = CancellationCode::Unknown; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cb42a8abf9c..0c79f4310ce 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1232,7 +1232,7 @@ SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery std::shared_ptr collator; if (order_by_elem.getCollation()) - collator = std::make_shared(order_by_elem.getCollation()->as().value.get()); + collator = std::make_shared(order_by_elem.getCollation()->as().value.safeGet()); if (order_by_elem.with_fill) { diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 57ad5caa4c7..dfffeb437d4 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -220,7 +220,7 @@ bool isStorageTouchedByMutations( Block tmp_block; while (executor.pull(tmp_block)); - auto count = (*block.getByName("count()").column)[0].get(); + auto count = (*block.getByName("count()").column)[0].safeGet(); return count != 0; } diff --git a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp index dd205ae6508..913f9900b77 100644 --- a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp +++ b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp @@ -42,13 +42,13 @@ ASTPtr generateOptimizedDateFilterAST(const String & comparator, const NameAndTy if (isDateOrDate32(column.type.get())) { - start_date_or_date_time = date_lut.dateToString(range.first.get()); - end_date_or_date_time = date_lut.dateToString(range.second.get()); + start_date_or_date_time = date_lut.dateToString(range.first.safeGet()); + end_date_or_date_time = date_lut.dateToString(range.second.safeGet()); } else if (isDateTime(column.type.get()) || isDateTime64(column.type.get())) { - start_date_or_date_time = date_lut.timeToString(range.first.get()); - end_date_or_date_time = date_lut.timeToString(range.second.get()); + start_date_or_date_time = date_lut.timeToString(range.first.safeGet()); + end_date_or_date_time = date_lut.timeToString(range.second.safeGet()); } else [[unlikely]] return {}; diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp index 48c9988b6fc..e9a663d53b0 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp @@ -24,7 +24,7 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v if (literal->value.getType() == Field::Types::Int64 || literal->value.getType() == Field::Types::UInt64) { - value = literal->value.get(); + value = literal->value.safeGet(); return true; } if (literal->value.getType() == Field::Types::Null) @@ -51,7 +51,7 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v { if (type_literal->value.getType() == Field::Types::String) { - const auto & type_str = type_literal->value.get(); + const auto & type_str = type_literal->value.safeGet(); if (type_str == "UInt8" || type_str == "Nullable(UInt8)") return tryExtractConstValueFromCondition(expr_list->children.at(0), value); } diff --git a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp index 54515ea072a..6a0522b0676 100644 --- a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp +++ b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp @@ -72,7 +72,7 @@ bool shardContains( if (sharding_value.isNull()) return false; - UInt64 value = sharding_value.get(); + UInt64 value = sharding_value.safeGet(); const auto shard_num = data.slots[value % data.slots.size()] + 1; return data.shard_info.shard_num == shard_num; } @@ -120,11 +120,20 @@ void OptimizeShardingKeyRewriteInMatcher::visit(ASTFunction & function, Data & d else if (auto * tuple_literal = right->as(); tuple_literal && tuple_literal->value.getType() == Field::Types::Tuple) { - auto & tuple = tuple_literal->value.get(); - std::erase_if(tuple, [&](auto & child) + auto & tuple = tuple_literal->value.safeGet(); + if (tuple.size() > 1) { - return tuple.size() > 1 && !shardContains(child, name, data); - }); + Tuple new_tuple; + + for (const auto & child : tuple) + if (shardContains(child, name, data)) + new_tuple.emplace_back(std::move(child)); + + if (new_tuple.size() == 0) + new_tuple.emplace_back(std::move(tuple.back())); + + tuple_literal->value = std::move(new_tuple); + } } } @@ -159,7 +168,7 @@ public: { if (isTuple(constant->getResultType())) { - const auto & tuple = constant->getValue().get(); + const auto & tuple = constant->getValue().safeGet(); Tuple new_tuple; new_tuple.reserve(tuple.size()); diff --git a/src/Interpreters/RewriteCountVariantsVisitor.cpp b/src/Interpreters/RewriteCountVariantsVisitor.cpp index 4a541c3765a..272e1ac735f 100644 --- a/src/Interpreters/RewriteCountVariantsVisitor.cpp +++ b/src/Interpreters/RewriteCountVariantsVisitor.cpp @@ -53,7 +53,7 @@ void RewriteCountVariantsVisitor::visit(ASTFunction & func) { if (first_arg_literal->value.getType() == Field::Types::UInt64) { - auto constant = first_arg_literal->value.get(); + auto constant = first_arg_literal->value.safeGet(); if (constant == 1 && !context->getSettingsRef().aggregate_functions_null_for_empty) transform = true; } diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index b872eb94fde..6483dd3be48 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -184,7 +184,7 @@ void optimizeGroupBy(ASTSelectQuery * select_query, ContextPtr context) const auto & value = group_exprs[i]->as()->value; if (value.getType() == Field::Types::UInt64) { - auto pos = value.get(); + auto pos = value.safeGet(); if (pos > 0 && pos <= select_query->select()->children.size()) keep_position = true; } diff --git a/src/Interpreters/WindowDescription.cpp b/src/Interpreters/WindowDescription.cpp index 31a881001e3..b1e12ff8048 100644 --- a/src/Interpreters/WindowDescription.cpp +++ b/src/Interpreters/WindowDescription.cpp @@ -94,8 +94,8 @@ void WindowFrame::checkValid() const if (begin_type == BoundaryType::Offset && !((begin_offset.getType() == Field::Types::UInt64 || begin_offset.getType() == Field::Types::Int64) - && begin_offset.get() >= 0 - && begin_offset.get() < INT_MAX)) + && begin_offset.safeGet() >= 0 + && begin_offset.safeGet() < INT_MAX)) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Frame start offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", @@ -107,8 +107,8 @@ void WindowFrame::checkValid() const if (end_type == BoundaryType::Offset && !((end_offset.getType() == Field::Types::UInt64 || end_offset.getType() == Field::Types::Int64) - && end_offset.get() >= 0 - && end_offset.get() < INT_MAX)) + && end_offset.safeGet() >= 0 + && end_offset.safeGet() < INT_MAX)) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Frame end offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 1a7c166c6a5..f3957c3b69b 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -57,7 +57,7 @@ template Field convertNumericTypeImpl(const Field & from) { To result; - if (!accurate::convertNumeric(from.get(), result)) + if (!accurate::convertNumeric(from.safeGet(), result)) return {}; return result; } @@ -88,7 +88,7 @@ Field convertNumericType(const Field & from, const IDataType & type) template Field convertIntToDecimalType(const Field & from, const DataTypeDecimal & type) { - From value = from.get(); + From value = from.safeGet(); if (!type.canStoreWhole(value)) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Number is too big to place in {}", type.getName()); @@ -100,7 +100,7 @@ Field convertIntToDecimalType(const Field & from, const DataTypeDecimal & typ template Field convertStringToDecimalType(const Field & from, const DataTypeDecimal & type) { - const String & str_value = from.get(); + const String & str_value = from.safeGet(); T value = type.parseFromString(str_value); return DecimalField(value, type.getScale()); } @@ -108,7 +108,7 @@ Field convertStringToDecimalType(const Field & from, const DataTypeDecimal & template Field convertDecimalToDecimalType(const Field & from, const DataTypeDecimal & type) { - auto field = from.get>(); + auto field = from.safeGet>(); T value = convertDecimals, DataTypeDecimal>(field.getValue(), field.getScale(), type.getScale()); return DecimalField(value, type.getScale()); } @@ -116,7 +116,7 @@ Field convertDecimalToDecimalType(const Field & from, const DataTypeDecimal & template Field convertFloatToDecimalType(const Field & from, const DataTypeDecimal & type) { - From value = from.get(); + From value = from.safeGet(); if (!type.canStoreWhole(value)) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Number is too big to place in {}", type.getName()); @@ -182,24 +182,24 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID /// Conversion between Date and DateTime and vice versa. if (which_type.isDate() && which_from_type.isDateTime()) { - return static_cast(static_cast(*from_type_hint).getTimeZone().toDayNum(src.get()).toUnderType()); + return static_cast(static_cast(*from_type_hint).getTimeZone().toDayNum(src.safeGet()).toUnderType()); } else if (which_type.isDate32() && which_from_type.isDateTime()) { - return static_cast(static_cast(*from_type_hint).getTimeZone().toDayNum(src.get()).toUnderType()); + return static_cast(static_cast(*from_type_hint).getTimeZone().toDayNum(src.safeGet()).toUnderType()); } else if (which_type.isDateTime() && which_from_type.isDate()) { - return static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); + return static_cast(type).getTimeZone().fromDayNum(DayNum(src.safeGet())); } else if (which_type.isDateTime() && which_from_type.isDate32()) { - return static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); + return static_cast(type).getTimeZone().fromDayNum(DayNum(src.safeGet())); } else if (which_type.isDateTime64() && which_from_type.isDate()) { const auto & date_time64_type = static_cast(type); - const auto value = date_time64_type.getTimeZone().fromDayNum(DayNum(src.get())); + const auto value = date_time64_type.getTimeZone().fromDayNum(DayNum(src.safeGet())); return DecimalField( DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, date_time64_type.getScaleMultiplier()), date_time64_type.getScale()); @@ -207,7 +207,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID else if (which_type.isDateTime64() && which_from_type.isDate32()) { const auto & date_time64_type = static_cast(type); - const auto value = date_time64_type.getTimeZone().fromDayNum(ExtendedDayNum(static_cast(src.get()))); + const auto value = date_time64_type.getTimeZone().fromDayNum(ExtendedDayNum(static_cast(src.safeGet()))); return DecimalField( DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, date_time64_type.getScaleMultiplier()), date_time64_type.getScale()); @@ -253,7 +253,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID if (which_type.isDateTime64() && src.getType() == Field::Types::Decimal64) { - const auto & from_type = src.get(); + const auto & from_type = src.safeGet(); const auto & to_type = static_cast(type); const auto scale_from = from_type.getScale(); @@ -318,7 +318,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID if (which_from_type.isFixedString() && assert_cast(from_type_hint)->getN() == IPV6_BINARY_LENGTH) { const auto col = type.createColumn(); - ReadBufferFromString in_buffer(src.get()); + ReadBufferFromString in_buffer(src.safeGet()); type.getDefaultSerialization()->deserializeBinary(*col, in_buffer, {}); return (*col)[0]; } @@ -330,7 +330,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID if (which_type.isFixedString()) { size_t n = assert_cast(type).getN(); - const auto & src_str = src.get(); + const auto & src_str = src.safeGet(); if (src_str.size() < n) { String src_str_extended = src_str; @@ -347,7 +347,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID { if (src.getType() == Field::Types::Array) { - const Array & src_arr = src.get(); + const Array & src_arr = src.safeGet(); size_t src_arr_size = src_arr.size(); const auto & element_type = *(type_array->getNestedType()); @@ -370,7 +370,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID { if (src.getType() == Field::Types::Tuple) { - const auto & src_tuple = src.get(); + const auto & src_tuple = src.safeGet(); size_t src_tuple_size = src_tuple.size(); size_t dst_tuple_size = type_tuple->getElements().size(); @@ -415,7 +415,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID const auto & key_type = *type_map->getKeyType(); const auto & value_type = *type_map->getValueType(); - const auto & map = src.get(); + const auto & map = src.safeGet(); size_t map_size = map.size(); Map res(map_size); @@ -424,7 +424,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID for (size_t i = 0; i < map_size; ++i) { - const auto & map_entry = map[i].get(); + const auto & map_entry = map[i].safeGet(); const auto & key = map_entry[0]; const auto & value = map_entry[1]; @@ -453,7 +453,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID "Cannot convert {} to {}", src.getTypeName(), agg_func_type->getName()); - const auto & name = src.get().name; + const auto & name = src.safeGet().name; if (agg_func_type->getName() != name) throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert {} to {}", name, agg_func_type->getName()); @@ -468,7 +468,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID if (src.getType() == Field::Types::Tuple && from_type_tuple && from_type_tuple->haveExplicitNames()) { const auto & names = from_type_tuple->getElementNames(); - const auto & tuple = src.get(); + const auto & tuple = src.safeGet(); if (names.size() != tuple.size()) throw Exception(ErrorCodes::TYPE_MISMATCH, @@ -485,10 +485,10 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID if (src.getType() == Field::Types::Map) { Object object; - const auto & map = src.get(); + const auto & map = src.safeGet(); for (const auto & element : map) { - const auto & map_entry = element.get(); + const auto & map_entry = element.safeGet(); const auto & key = map_entry[0]; const auto & value = map_entry[1]; @@ -496,7 +496,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert from Map with key of type {} to Object", key.getTypeName()); - object[key.get()] = value; + object[key.safeGet()] = value; } return object; @@ -537,7 +537,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID } const auto col = type_to_parse->createColumn(); - ReadBufferFromString in_buffer(src.get()); + ReadBufferFromString in_buffer(src.safeGet()); try { type_to_parse->getDefaultSerialization()->deserializeWholeText(*col, in_buffer, FormatSettings{}); @@ -545,9 +545,9 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID catch (Exception & e) { if (e.code() == ErrorCodes::UNEXPECTED_DATA_AFTER_PARSED_VALUE) - throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert string '{}' to type {}", src.get(), type.getName()); + throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert string '{}' to type {}", src.safeGet(), type.getName()); - e.addMessage(fmt::format("while converting '{}' to {}", src.get(), type.getName())); + e.addMessage(fmt::format("while converting '{}' to {}", src.safeGet(), type.getName())); throw; } @@ -610,7 +610,7 @@ Field convertFieldToTypeOrThrow(const Field & from_value, const IDataType & to_t template static bool decimalEqualsFloat(Field field, Float64 float_value) { - auto decimal_field = field.get>(); + auto decimal_field = field.safeGet>(); auto decimal_to_float = DecimalUtils::convertTo(decimal_field.getValue(), decimal_field.getScale()); return decimal_to_float == float_value; } @@ -629,13 +629,13 @@ std::optional convertFieldToTypeStrict(const Field & from_value, const ID { /// Convert back to Float64 and compare if (result_value.getType() == Field::Types::Decimal32) - return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; + return decimalEqualsFloat(result_value, from_value.safeGet()) ? result_value : std::optional{}; if (result_value.getType() == Field::Types::Decimal64) - return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; + return decimalEqualsFloat(result_value, from_value.safeGet()) ? result_value : std::optional{}; if (result_value.getType() == Field::Types::Decimal128) - return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; + return decimalEqualsFloat(result_value, from_value.safeGet()) ? result_value : std::optional{}; if (result_value.getType() == Field::Types::Decimal256) - return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; + return decimalEqualsFloat(result_value, from_value.safeGet()) ? result_value : std::optional{}; throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown decimal type {}", result_value.getTypeName()); } diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 4bfc80af1fe..d4bb0cc2f8a 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -297,7 +297,7 @@ namespace { if (tuple_literal->value.getType() == Field::Types::Tuple) { - const auto & tuple = tuple_literal->value.get(); + const auto & tuple = tuple_literal->value.safeGet(); for (const auto & child : tuple) { const auto dnf = analyzeEquals(identifier, child, expr); @@ -792,7 +792,7 @@ std::optional evaluateExpressionOverConstantCondition(const ASTPtr & nod else if (const auto * literal = node->as()) { // Check if it's always true or false. - if (literal->value.getType() == Field::Types::UInt64 && literal->value.get() == 0) + if (literal->value.getType() == Field::Types::UInt64 && literal->value.safeGet() == 0) return {result}; else return {}; diff --git a/src/Interpreters/replaceForPositionalArguments.cpp b/src/Interpreters/replaceForPositionalArguments.cpp index 3d60723a167..ee967f45c74 100644 --- a/src/Interpreters/replaceForPositionalArguments.cpp +++ b/src/Interpreters/replaceForPositionalArguments.cpp @@ -35,11 +35,11 @@ bool replaceForPositionalArguments(ASTPtr & argument, const ASTSelectQuery * sel if (which == Field::Types::UInt64) { - pos = ast_literal->value.get(); + pos = ast_literal->value.safeGet(); } else if (which == Field::Types::Int64) { - auto value = ast_literal->value.get(); + auto value = ast_literal->value.safeGet(); if (value > 0) pos = value; else diff --git a/src/Interpreters/tests/gtest_comparison_graph.cpp b/src/Interpreters/tests/gtest_comparison_graph.cpp index ac24a8de368..5f93bb983c1 100644 --- a/src/Interpreters/tests/gtest_comparison_graph.cpp +++ b/src/Interpreters/tests/gtest_comparison_graph.cpp @@ -29,7 +29,7 @@ TEST(ComparisonGraph, Bounds) const auto & [lower, strict] = *res; - ASSERT_EQ(lower.get(), 3); + ASSERT_EQ(lower.safeGet(), 3); ASSERT_TRUE(strict); } @@ -39,7 +39,7 @@ TEST(ComparisonGraph, Bounds) const auto & [upper, strict] = *res; - ASSERT_EQ(upper.get(), 7); + ASSERT_EQ(upper.safeGet(), 7); ASSERT_TRUE(strict); } diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index cd9e910d45a..d42728addb7 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -522,7 +522,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format if (tuple_arguments_valid && lit_right) { if (isInt64OrUInt64FieldType(lit_right->value.getType()) - && lit_right->value.get() >= 0) + && lit_right->value.safeGet() >= 0) { if (frame.need_parens) settings.ostr << '('; diff --git a/src/Parsers/ASTLiteral.cpp b/src/Parsers/ASTLiteral.cpp index 8dedc5dc95d..515f4f0cb9f 100644 --- a/src/Parsers/ASTLiteral.cpp +++ b/src/Parsers/ASTLiteral.cpp @@ -73,8 +73,8 @@ void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const /// Special case for very large arrays and tuples. Instead of listing all elements, will use hash of them. /// (Otherwise column name will be too long, that will lead to significant slowdown of expression analysis.) auto type = value.getType(); - if ((type == Field::Types::Array && value.get().size() > min_elements_for_hashing) - || (type == Field::Types::Tuple && value.get().size() > min_elements_for_hashing)) + if ((type == Field::Types::Array && value.safeGet().size() > min_elements_for_hashing) + || (type == Field::Types::Tuple && value.safeGet().size() > min_elements_for_hashing)) { SipHash hash; applyVisitor(FieldVisitorHash(hash), value); @@ -92,7 +92,7 @@ void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const /// for tons of literals as it creates temporary String. if (value.getType() == Field::Types::String) { - writeQuoted(value.get(), ostr); + writeQuoted(value.safeGet(), ostr); } else { @@ -110,7 +110,7 @@ void ASTLiteral::appendColumnNameImplLegacy(WriteBuffer & ostr) const /// Special case for very large arrays. Instead of listing all elements, will use hash of them. /// (Otherwise column name will be too long, that will lead to significant slowdown of expression analysis.) auto type = value.getType(); - if ((type == Field::Types::Array && value.get().size() > min_elements_for_hashing)) + if ((type == Field::Types::Array && value.safeGet().size() > min_elements_for_hashing)) { SipHash hash; applyVisitor(FieldVisitorHash(hash), value); diff --git a/src/Parsers/Access/ParserCreateQuotaQuery.cpp b/src/Parsers/Access/ParserCreateQuotaQuery.cpp index ddfdbe38903..ddf4e9ecda5 100644 --- a/src/Parsers/Access/ParserCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ParserCreateQuotaQuery.cpp @@ -114,7 +114,7 @@ namespace T fieldToNumber(const Field & f) { if (f.getType() == Field::Types::String) - return parseWithSizeSuffix(boost::algorithm::trim_copy(f.get())); + return parseWithSizeSuffix(boost::algorithm::trim_copy(f.safeGet())); else return applyVisitor(FieldVisitorConvertToNumber(), f); } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 9927acdcf17..de395d120d7 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -314,7 +314,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex ASTPtr ast_uuid; if (!uuid_p.parse(pos, ast_uuid, expected)) return false; - uuid = parseFromString(ast_uuid->as()->value.get()); + uuid = parseFromString(ast_uuid->as()->value.safeGet()); } if (parts.size() == 1) node = std::make_shared(parts[0], std::move(params)); @@ -1626,7 +1626,7 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e if (!parser_string_literal.parse(pos, ast_prefix_name, expected)) return false; - column_name_prefix = ast_prefix_name->as().value.get(); + column_name_prefix = ast_prefix_name->as().value.safeGet(); } if (with_open_round_bracket) @@ -1689,7 +1689,7 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e auto res = std::make_shared(); if (regexp_node) - res->setPattern(regexp_node->as().value.get()); + res->setPattern(regexp_node->as().value.safeGet()); else res->children = std::move(identifiers); res->is_strict = is_strict; @@ -1861,7 +1861,7 @@ static bool parseColumnsMatcherBody(IParser::Pos & pos, ASTPtr & node, Expected else { auto regexp_matcher = std::make_shared(); - regexp_matcher->setPattern(regexp_node->as().value.get()); + regexp_matcher->setPattern(regexp_node->as().value.safeGet()); if (!transformers->children.empty()) { @@ -2310,7 +2310,7 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!parser_string_literal.parse(pos, ast_space_name, expected)) return false; - destination_name = ast_space_name->as().value.get(); + destination_name = ast_space_name->as().value.safeGet(); } else if (mode == TTLMode::GROUP_BY) { diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index dbefb0cb966..73fd563faf6 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -517,7 +517,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!parser_string_literal.parse(pos, ast_space_name, expected)) return false; - command->move_destination_name = ast_space_name->as().value.get(); + command->move_destination_name = ast_space_name->as().value.safeGet(); } else if (s_move_partition.ignore(pos, expected)) { @@ -545,7 +545,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!parser_string_literal.parse(pos, ast_space_name, expected)) return false; - command->move_destination_name = ast_space_name->as().value.get(); + command->move_destination_name = ast_space_name->as().value.safeGet(); } } else if (s_add_constraint.ignore(pos, expected)) @@ -638,7 +638,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!parser_string_literal.parse(pos, ast_from, expected)) return false; - command->from = ast_from->as().value.get(); + command->from = ast_from->as().value.safeGet(); command->type = ASTAlterCommand::FETCH_PARTITION; } else if (s_fetch_part.ignore(pos, expected)) @@ -652,7 +652,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ASTPtr ast_from; if (!parser_string_literal.parse(pos, ast_from, expected)) return false; - command->from = ast_from->as().value.get(); + command->from = ast_from->as().value.safeGet(); command->part = true; command->type = ASTAlterCommand::FETCH_PARTITION; } @@ -680,7 +680,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!parser_string_literal.parse(pos, ast_with_name, expected)) return false; - command->with_name = ast_with_name->as().value.get(); + command->with_name = ast_with_name->as().value.safeGet(); } } else if (s_unfreeze.ignore(pos, expected)) @@ -707,7 +707,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!parser_string_literal.parse(pos, ast_with_name, expected)) return false; - command->with_name = ast_with_name->as().value.get(); + command->with_name = ast_with_name->as().value.safeGet(); } else { diff --git a/src/Parsers/ParserCheckQuery.cpp b/src/Parsers/ParserCheckQuery.cpp index 42716ba7f2c..33b6a5a1ac2 100644 --- a/src/Parsers/ParserCheckQuery.cpp +++ b/src/Parsers/ParserCheckQuery.cpp @@ -55,7 +55,7 @@ bool ParserCheckQuery::parseCheckTable(Pos & pos, ASTPtr & node, Expected & expe const auto * ast_literal = ast_part_name->as(); if (!ast_literal || ast_literal->value.getType() != Field::Types::String) return false; - query->part_name = ast_literal->value.get(); + query->part_name = ast_literal->value.safeGet(); } if (query->database) diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index a592975613b..3621a695272 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -922,7 +922,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->is_create_empty = is_create_empty; if (from_path) - query->attach_from_path = from_path->as().value.get(); + query->attach_from_path = from_path->as().value.safeGet(); return true; } @@ -1431,7 +1431,7 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e ASTPtr ast_uuid; if (!uuid_p.parse(pos, ast_uuid, expected)) return false; - uuid = parseFromString(ast_uuid->as()->value.get()); + uuid = parseFromString(ast_uuid->as()->value.safeGet()); } if (s_on.ignore(pos, expected)) diff --git a/src/Parsers/ParserDictionary.cpp b/src/Parsers/ParserDictionary.cpp index 83a006231d9..ce38d1b54d1 100644 --- a/src/Parsers/ParserDictionary.cpp +++ b/src/Parsers/ParserDictionary.cpp @@ -33,7 +33,7 @@ bool ParserDictionaryLifetime::parseImpl(Pos & pos, ASTPtr & node, Expected & ex if (literal.value.getType() != Field::Types::UInt64) return false; - res->max_sec = literal.value.get(); + res->max_sec = literal.value.safeGet(); node = res; return true; } @@ -58,10 +58,10 @@ bool ParserDictionaryLifetime::parseImpl(Pos & pos, ASTPtr & node, Expected & ex return false; if (pair.first == "min") - res->min_sec = literal->value.get(); + res->min_sec = literal->value.safeGet(); else if (pair.first == "max") { - res->max_sec = literal->value.get(); + res->max_sec = literal->value.safeGet(); initialized_max = true; } else diff --git a/src/Parsers/ParserPartition.cpp b/src/Parsers/ParserPartition.cpp index 80a28f4803e..ab97b3d0e3b 100644 --- a/src/Parsers/ParserPartition.cpp +++ b/src/Parsers/ParserPartition.cpp @@ -65,7 +65,7 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (literal_ast->value.getType() == Field::Types::Tuple) { - fields_count = literal_ast->value.get().size(); + fields_count = literal_ast->value.safeGet().size(); } else { diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 0545c3e5568..81b64ab47c6 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -445,7 +445,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & ASTPtr ast; if (!ParserStringLiteral{}.parse(pos, ast, expected)) return false; - String time_str = ast->as().value.get(); + String time_str = ast->as().value.safeGet(); ReadBufferFromString buf(time_str); time_t time; readDateTimeText(time, buf); @@ -467,7 +467,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & return false; } - res->seconds = seconds->as()->value.get(); + res->seconds = seconds->as()->value.safeGet(); break; } case Type::DROP_FILESYSTEM_CACHE: @@ -538,7 +538,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & ASTPtr ast; if (ParserKeyword{Keyword::WITH_NAME}.ignore(pos, expected) && ParserStringLiteral{}.parse(pos, ast, expected)) { - res->backup_name = ast->as().value.get(); + res->backup_name = ast->as().value.safeGet(); } else { @@ -577,7 +577,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & if (!ParserStringLiteral{}.parse(pos, ast, expected)) return false; - custom_name = ast->as().value.get(); + custom_name = ast->as().value.safeGet(); } return true; diff --git a/src/Parsers/ParserUndropQuery.cpp b/src/Parsers/ParserUndropQuery.cpp index 07ca8a3b5fd..57da47df70d 100644 --- a/src/Parsers/ParserUndropQuery.cpp +++ b/src/Parsers/ParserUndropQuery.cpp @@ -41,7 +41,7 @@ bool parseUndropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected) ASTPtr ast_uuid; if (!uuid_p.parse(pos, ast_uuid, expected)) return false; - uuid = parseFromString(ast_uuid->as()->value.get()); + uuid = parseFromString(ast_uuid->as()->value.safeGet()); } if (ParserKeyword{Keyword::ON}.ignore(pos, expected)) { diff --git a/src/Parsers/tests/gtest_dictionary_parser.cpp b/src/Parsers/tests/gtest_dictionary_parser.cpp index a1ba46125a7..af3591750a1 100644 --- a/src/Parsers/tests/gtest_dictionary_parser.cpp +++ b/src/Parsers/tests/gtest_dictionary_parser.cpp @@ -56,21 +56,21 @@ TEST(ParserDictionaryDDL, SimpleDictionary) EXPECT_EQ(create->dictionary->source->name, "clickhouse"); auto children = create->dictionary->source->elements->children; EXPECT_EQ(children[0]->as() -> first, "host"); - EXPECT_EQ(children[0]->as()->second->as()->value.get(), "localhost"); + EXPECT_EQ(children[0]->as()->second->as()->value.safeGet(), "localhost"); EXPECT_EQ(children[1]->as()->first, "port"); - EXPECT_EQ(children[1]->as()->second->as()->value.get(), 9000); + EXPECT_EQ(children[1]->as()->second->as()->value.safeGet(), 9000); EXPECT_EQ(children[2]->as()->first, "user"); - EXPECT_EQ(children[2]->as()->second->as()->value.get(), "default"); + EXPECT_EQ(children[2]->as()->second->as()->value.safeGet(), "default"); EXPECT_EQ(children[3]->as()->first, "password"); - EXPECT_EQ(children[3]->as()->second->as()->value.get(), ""); + EXPECT_EQ(children[3]->as()->second->as()->value.safeGet(), ""); EXPECT_EQ(children[4]->as()->first, "db"); - EXPECT_EQ(children[4]->as()->second->as()->value.get(), "test"); + EXPECT_EQ(children[4]->as()->second->as()->value.safeGet(), "test"); EXPECT_EQ(children[5]->as()->first, "table"); - EXPECT_EQ(children[5]->as()->second->as()->value.get(), "table_for_dict"); + EXPECT_EQ(children[5]->as()->second->as()->value.safeGet(), "table_for_dict"); /// layout test auto * layout = create->dictionary->layout; @@ -102,9 +102,9 @@ TEST(ParserDictionaryDDL, SimpleDictionary) EXPECT_EQ(attributes_children[1]->as()->name, "second_column"); EXPECT_EQ(attributes_children[2]->as()->name, "third_column"); - EXPECT_EQ(attributes_children[0]->as()->default_value->as()->value.get(), 0); - EXPECT_EQ(attributes_children[1]->as()->default_value->as()->value.get(), 1); - EXPECT_EQ(attributes_children[2]->as()->default_value->as()->value.get(), 2); + EXPECT_EQ(attributes_children[0]->as()->default_value->as()->value.safeGet(), 0); + EXPECT_EQ(attributes_children[1]->as()->default_value->as()->value.safeGet(), 1); + EXPECT_EQ(attributes_children[2]->as()->default_value->as()->value.safeGet(), 2); EXPECT_EQ(attributes_children[0]->as()->expression, nullptr); EXPECT_EQ(attributes_children[1]->as()->expression, nullptr); @@ -150,8 +150,8 @@ TEST(ParserDictionaryDDL, AttributesWithMultipleProperties) EXPECT_EQ(attributes_children[2]->as()->name, "third_column"); EXPECT_EQ(attributes_children[0]->as()->default_value, nullptr); - EXPECT_EQ(attributes_children[1]->as()->default_value->as()->value.get(), 1); - EXPECT_EQ(attributes_children[2]->as()->default_value->as()->value.get(), 2); + EXPECT_EQ(attributes_children[1]->as()->default_value->as()->value.safeGet(), 1); + EXPECT_EQ(attributes_children[2]->as()->default_value->as()->value.safeGet(), 2); EXPECT_EQ(attributes_children[0]->as()->expression, nullptr); EXPECT_EQ(attributes_children[1]->as()->expression, nullptr); @@ -195,9 +195,9 @@ TEST(ParserDictionaryDDL, CustomAttributePropertiesOrder) EXPECT_EQ(attributes_children[1]->as()->name, "second_column"); EXPECT_EQ(attributes_children[2]->as()->name, "third_column"); - EXPECT_EQ(attributes_children[0]->as()->default_value->as()->value.get(), 100); - EXPECT_EQ(attributes_children[1]->as()->default_value->as()->value.get(), 1); - EXPECT_EQ(attributes_children[2]->as()->default_value->as()->value.get(), 2); + EXPECT_EQ(attributes_children[0]->as()->default_value->as()->value.safeGet(), 100); + EXPECT_EQ(attributes_children[1]->as()->default_value->as()->value.safeGet(), 1); + EXPECT_EQ(attributes_children[2]->as()->default_value->as()->value.safeGet(), 2); EXPECT_EQ(attributes_children[0]->as()->expression, nullptr); EXPECT_EQ(attributes_children[1]->as()->expression, nullptr); @@ -248,25 +248,25 @@ TEST(ParserDictionaryDDL, NestedSource) auto children = create->dictionary->source->elements->children; EXPECT_EQ(children[0]->as()->first, "host"); - EXPECT_EQ(children[0]->as()->second->as()->value.get(), "localhost"); + EXPECT_EQ(children[0]->as()->second->as()->value.safeGet(), "localhost"); EXPECT_EQ(children[1]->as()->first, "port"); - EXPECT_EQ(children[1]->as()->second->as()->value.get(), 9000); + EXPECT_EQ(children[1]->as()->second->as()->value.safeGet(), 9000); EXPECT_EQ(children[2]->as()->first, "user"); - EXPECT_EQ(children[2]->as()->second->as()->value.get(), "default"); + EXPECT_EQ(children[2]->as()->second->as()->value.safeGet(), "default"); EXPECT_EQ(children[3]->as()->first, "replica"); auto replica = children[3]->as()->second->children; EXPECT_EQ(replica[0]->as()->first, "host"); - EXPECT_EQ(replica[0]->as()->second->as()->value.get(), "127.0.0.1"); + EXPECT_EQ(replica[0]->as()->second->as()->value.safeGet(), "127.0.0.1"); EXPECT_EQ(replica[1]->as()->first, "priority"); - EXPECT_EQ(replica[1]->as()->second->as()->value.get(), 1); + EXPECT_EQ(replica[1]->as()->second->as()->value.safeGet(), 1); EXPECT_EQ(children[4]->as()->first, "password"); - EXPECT_EQ(children[4]->as()->second->as()->value.get(), ""); + EXPECT_EQ(children[4]->as()->second->as()->value.safeGet(), ""); } diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 58bf4c1a2fc..30301b242db 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -185,7 +185,7 @@ namespace DB } else { - auto value = static_cast(column[value_i].get>().getValue()); + auto value = static_cast(column[value_i].safeGet>().getValue()); if (need_rescale) { if (common::mulOverflow(value, rescale_multiplier, value)) diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 06e8668cd7c..566a036d79c 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -208,20 +208,20 @@ private: /// Do not replace empty array and array of NULLs if (literal->value.getType() == Field::Types::Array) { - const Array & array = literal->value.get(); + const Array & array = literal->value.safeGet(); auto not_null = std::find_if_not(array.begin(), array.end(), [](const auto & elem) { return elem.isNull(); }); if (not_null == array.end()) return true; } else if (literal->value.getType() == Field::Types::Map) { - const Map & map = literal->value.get(); + const Map & map = literal->value.safeGet(); if (map.size() % 2) return false; } else if (literal->value.getType() == Field::Types::Tuple) { - const Tuple & tuple = literal->value.get(); + const Tuple & tuple = literal->value.safeGet(); for (const auto & value : tuple) if (value.isNull()) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 649721f28bf..58bec8120f1 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -262,7 +262,7 @@ convertFieldToORCLiteral(const orc::Type & orc_type, const Field & field, DataTy { case orc::BOOLEAN: { /// May throw exception - auto val = field.get(); + auto val = field.safeGet(); return orc::Literal(val != 0); } case orc::BYTE: @@ -275,7 +275,7 @@ convertFieldToORCLiteral(const orc::Type & orc_type, const Field & field, DataTy /// SELECT * FROM file('t.orc', ORC, 'x UInt8') WHERE x > 10 /// We have to reject this, otherwise it would miss values > 127 (because /// they're treated as negative by ORC). - auto val = field.get(); + auto val = field.safeGet(); return orc::Literal(val); } case orc::FLOAT: diff --git a/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp b/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp index 3578401a0f8..b43c195f201 100644 --- a/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp @@ -286,10 +286,10 @@ static void columnMapToContainer(const ColumnMap * col_map, size_t row_num, Cont { Field field; col_map->get(row_num, field); - const auto & map_field = field.get(); + const auto & map_field = field.safeGet(); for (const auto & map_element : map_field) { - const auto & map_entry = map_element.get(); + const auto & map_entry = map_element.safeGet(); String entry_key; String entry_value; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index de34a8aa04f..9839f64b947 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -333,7 +333,7 @@ namespace { const DataTypeTuple & type_tuple = static_cast(data_type); - Tuple & tuple_value = value.get(); + Tuple & tuple_value = value.safeGet(); size_t src_tuple_size = tuple_value.size(); size_t dst_tuple_size = type_tuple.getElements().size(); @@ -360,7 +360,7 @@ namespace if (element_type.isNullable()) return; - Array & array_value = value.get(); + Array & array_value = value.safeGet(); size_t array_value_size = array_value.size(); for (size_t i = 0; i < array_value_size; ++i) @@ -378,12 +378,12 @@ namespace const auto & key_type = *type_map.getKeyType(); const auto & value_type = *type_map.getValueType(); - auto & map = value.get(); + auto & map = value.safeGet(); size_t map_size = map.size(); for (size_t i = 0; i < map_size; ++i) { - auto & map_entry = map[i].get(); + auto & map_entry = map[i].safeGet(); auto & entry_key = map_entry[0]; auto & entry_value = map_entry[1]; diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index e2c6371c44f..80c00f91d82 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -127,14 +127,14 @@ static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, Row right(left.size()); for (size_t col_num : desc.key_col_nums) - right[col_num] = (*raw_columns[col_num])[row_number].template get(); + right[col_num] = (*raw_columns[col_num])[row_number].template safeGet(); for (size_t col_num : desc.val_col_nums) - right[col_num] = (*raw_columns[col_num])[row_number].template get(); + right[col_num] = (*raw_columns[col_num])[row_number].template safeGet(); auto at_ith_column_jth_row = [&](const Row & matrix, size_t i, size_t j) -> const Field & { - return matrix[i].get()[j]; + return matrix[i].safeGet()[j]; }; auto tuple_of_nth_columns_at_jth_row = [&](const Row & matrix, const ColumnNumbers & col_nums, size_t j) -> Array @@ -160,7 +160,7 @@ static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, auto merge = [&](const Row & matrix) { - size_t rows = matrix[desc.key_col_nums[0]].get().size(); + size_t rows = matrix[desc.key_col_nums[0]].safeGet().size(); for (size_t j = 0; j < rows; ++j) { @@ -190,10 +190,10 @@ static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, for (const auto & key_value : merged) { for (size_t col_num_index = 0, size = desc.key_col_nums.size(); col_num_index < size; ++col_num_index) - row[desc.key_col_nums[col_num_index]].get()[row_num] = key_value.first[col_num_index]; + row[desc.key_col_nums[col_num_index]].safeGet()[row_num] = key_value.first[col_num_index]; for (size_t col_num_index = 0, size = desc.val_col_nums.size(); col_num_index < size; ++col_num_index) - row[desc.val_col_nums[col_num_index]].get()[row_num] = key_value.second[col_num_index]; + row[desc.val_col_nums[col_num_index]].safeGet()[row_num] = key_value.second[col_num_index]; ++row_num; } diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 2080e29ceba..596d08845e1 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -119,23 +119,23 @@ using RangesWithStep = std::vector; std::optional steppedRangeFromRange(const Range & r, UInt64 step, UInt64 remainder) { - if ((r.right.get() == 0) && (!r.right_included)) + if ((r.right.safeGet() == 0) && (!r.right_included)) return std::nullopt; - UInt64 begin = (r.left.get() / step) * step; + UInt64 begin = (r.left.safeGet() / step) * step; if (begin > std::numeric_limits::max() - remainder) return std::nullopt; begin += remainder; - while ((r.left_included <= r.left.get()) && (begin <= r.left.get() - r.left_included)) + while ((r.left_included <= r.left.safeGet()) && (begin <= r.left.safeGet() - r.left_included)) { if (std::numeric_limits::max() - step < begin) return std::nullopt; begin += step; } - if ((begin >= r.right_included) && (begin - r.right_included >= r.right.get())) + if ((begin >= r.right_included) && (begin - r.right_included >= r.right.safeGet())) return std::nullopt; - UInt64 right_edge_included = r.right.get() - (1 - r.right_included); + UInt64 right_edge_included = r.right.safeGet() - (1 - r.right_included); return std::optional{RangeWithStep{begin, step, static_cast(right_edge_included - begin) / step + 1}}; } diff --git a/src/Processors/Sources/MySQLSource.cpp b/src/Processors/Sources/MySQLSource.cpp index 5d533a7747e..52be9a6e84a 100644 --- a/src/Processors/Sources/MySQLSource.cpp +++ b/src/Processors/Sources/MySQLSource.cpp @@ -219,11 +219,11 @@ namespace read_bytes_size += 8; break; case ValueType::vtEnum8: - assert_cast(column).insertValue(assert_cast &>(data_type).castToValue(value.data()).get()); + assert_cast(column).insertValue(assert_cast &>(data_type).castToValue(value.data()).safeGet()); read_bytes_size += assert_cast(column).byteSize(); break; case ValueType::vtEnum16: - assert_cast(column).insertValue(assert_cast &>(data_type).castToValue(value.data()).get()); + assert_cast(column).insertValue(assert_cast &>(data_type).castToValue(value.data()).safeGet()); read_bytes_size += assert_cast(column).byteSize(); break; case ValueType::vtString: diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 9601f821cc8..95f4a674ebb 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -62,7 +62,7 @@ static FillColumnDescription::StepFunction getStepFunction( case IntervalKind::Kind::NAME: \ return [step, scale, &date_lut](Field & field) { \ field = Add##NAME##sImpl::execute(static_cast(\ - field.get()), static_cast(step), date_lut, utc_time_zone, scale); }; + field.safeGet()), static_cast(step), date_lut, utc_time_zone, scale); }; FOR_EACH_INTERVAL_KIND(DECLARE_CASE) #undef DECLARE_CASE @@ -139,21 +139,21 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & { if (which.isDate() || which.isDate32()) { - Int64 avg_seconds = descr.fill_step.get() * descr.step_kind->toAvgSeconds(); + Int64 avg_seconds = descr.fill_step.safeGet() * descr.step_kind->toAvgSeconds(); if (std::abs(avg_seconds) < 86400) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Value of step is to low ({} seconds). Must be >= 1 day", std::abs(avg_seconds)); } if (which.isDate()) - descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.get(), DateLUT::instance()); + descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.safeGet(), DateLUT::instance()); else if (which.isDate32()) - descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.get(), DateLUT::instance()); + descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.safeGet(), DateLUT::instance()); else if (const auto * date_time = checkAndGetDataType(type.get())) - descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.get(), date_time->getTimeZone()); + descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.safeGet(), date_time->getTimeZone()); else if (const auto * date_time64 = checkAndGetDataType(type.get())) { - const auto & step_dec = descr.fill_step.get &>(); + const auto & step_dec = descr.fill_step.safeGet &>(); Int64 step = DecimalUtils::convertTo(step_dec.getValue(), step_dec.getScale()); static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC"); @@ -163,7 +163,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & case IntervalKind::Kind::NAME: \ descr.step_func = [step, &time_zone = date_time64->getTimeZone()](Field & field) \ { \ - auto field_decimal = field.get>(); \ + auto field_decimal = field.safeGet>(); \ auto res = Add##NAME##sImpl::execute(field_decimal.getValue(), step, time_zone, utc_time_zone, field_decimal.getScale()); \ field = DecimalField(res, field_decimal.getScale()); \ }; \ diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index f76e2d64368..32066adad0d 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -107,7 +107,7 @@ static int compareValuesWithOffset(const IColumn * _compared_column, using ValueType = typename ColumnType::ValueType; // Note that the storage type of offset returned by get<> is different, so // we need to specify the type explicitly. - const ValueType offset = static_cast(_offset.get()); + const ValueType offset = static_cast(_offset.safeGet()); assert(offset >= 0); const auto compared_value_data = compared_column->getDataAt(compared_row); @@ -162,7 +162,7 @@ static int compareValuesWithOffsetFloat(const IColumn * _compared_column, _compared_column); const auto * reference_column = assert_cast( _reference_column); - const auto offset = _offset.get(); + const auto offset = _offset.safeGet(); chassert(offset >= 0); const auto compared_value_data = compared_column->getDataAt(compared_row); @@ -631,7 +631,7 @@ void WindowTransform::advanceFrameStartRowsOffset() { // Just recalculate it each time by walking blocks. const auto [moved_row, offset_left] = moveRowNumber(current_row, - window_description.frame.begin_offset.get() + window_description.frame.begin_offset.safeGet() * (window_description.frame.begin_preceding ? -1 : 1)); frame_start = moved_row; @@ -870,7 +870,7 @@ void WindowTransform::advanceFrameEndRowsOffset() // Walk the specified offset from the current row. The "+1" is needed // because the frame_end is a past-the-end pointer. const auto [moved_row, offset_left] = moveRowNumber(current_row, - window_description.frame.end_offset.get() + window_description.frame.end_offset.safeGet() * (window_description.frame.end_preceding ? -1 : 1) + 1); @@ -2192,13 +2192,13 @@ namespace throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of 'ntile' function must be a constant"); auto type_id = argument_types[0]->getTypeId(); if (type_id == TypeIndex::UInt8) - buckets = arg_col[transform->current_row.row].get(); + buckets = arg_col[transform->current_row.row].safeGet(); else if (type_id == TypeIndex::UInt16) - buckets = arg_col[transform->current_row.row].get(); + buckets = arg_col[transform->current_row.row].safeGet(); else if (type_id == TypeIndex::UInt32) - buckets = arg_col[transform->current_row.row].get(); + buckets = arg_col[transform->current_row.row].safeGet(); else if (type_id == TypeIndex::UInt64) - buckets = arg_col[transform->current_row.row].get(); + buckets = arg_col[transform->current_row.row].safeGet(); if (!buckets) { @@ -2490,7 +2490,7 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction { offset = (*current_block.input_columns[ workspace.argument_column_indices[1]])[ - transform->current_row.row].get(); + transform->current_row.row].safeGet(); /// Either overflow or really negative value, both is not acceptable. if (offset < 0) @@ -2576,7 +2576,7 @@ struct WindowFunctionNthValue final : public WindowFunction Int64 offset = (*current_block.input_columns[ workspace.argument_column_indices[1]])[ - transform->current_row.row].get(); + transform->current_row.row].safeGet(); /// Either overflow or really negative value, both is not acceptable. if (offset <= 0) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 7891042bb96..4869819d0d6 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -109,7 +109,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ if (ast_col_decl.comment) { const auto & ast_comment = typeid_cast(*ast_col_decl.comment); - command.comment = ast_comment.value.get(); + command.comment = ast_comment.value.safeGet(); } if (ast_col_decl.codec) @@ -167,7 +167,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ if (ast_col_decl.comment) { const auto & ast_comment = ast_col_decl.comment->as(); - command.comment.emplace(ast_comment.value.get()); + command.comment.emplace(ast_comment.value.safeGet()); } if (ast_col_decl.ttl) @@ -210,7 +210,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.type = COMMENT_COLUMN; command.column_name = getIdentifierName(command_ast->column); const auto & ast_comment = command_ast->comment->as(); - command.comment = ast_comment.value.get(); + command.comment = ast_comment.value.safeGet(); command.if_exists = command_ast->if_exists; return command; } @@ -220,7 +220,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.ast = command_ast->clone(); command.type = COMMENT_TABLE; const auto & ast_comment = command_ast->comment->as(); - command.comment = ast_comment.value.get(); + command.comment = ast_comment.value.safeGet(); return command; } else if (command_ast->type == ASTAlterCommand::MODIFY_ORDER_BY) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index da749812167..0d724245b49 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -197,7 +197,7 @@ void ColumnDescription::readText(ReadBuffer & buf) } if (col_ast->comment) - comment = col_ast->comment->as().value.get(); + comment = col_ast->comment->as().value.safeGet(); if (col_ast->codec) codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(col_ast->codec, type, false, true, true, true); diff --git a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp index 7354243732c..d6a8af3238e 100644 --- a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp @@ -268,7 +268,7 @@ bool ApproximateNearestNeighborCondition::tryCastToConstType(const ASTPtr & node if (const_value.getType() == Field::Types::Float64) { out.function = RPNElement::FUNCTION_FLOAT_LITERAL; - out.float_literal.emplace(const_value.get()); + out.float_literal.emplace(const_value.safeGet()); out.func_name = "Float literal"; return true; } @@ -276,7 +276,7 @@ bool ApproximateNearestNeighborCondition::tryCastToConstType(const ASTPtr & node if (const_value.getType() == Field::Types::UInt64) { out.function = RPNElement::FUNCTION_INT_LITERAL; - out.int_literal.emplace(const_value.get()); + out.int_literal.emplace(const_value.safeGet()); out.func_name = "Int literal"; return true; } @@ -284,7 +284,7 @@ bool ApproximateNearestNeighborCondition::tryCastToConstType(const ASTPtr & node if (const_value.getType() == Field::Types::Int64) { out.function = RPNElement::FUNCTION_INT_LITERAL; - out.int_literal.emplace(const_value.get()); + out.int_literal.emplace(const_value.safeGet()); out.func_name = "Int literal"; return true; } @@ -292,7 +292,7 @@ bool ApproximateNearestNeighborCondition::tryCastToConstType(const ASTPtr & node if (const_value.getType() == Field::Types::Tuple) { out.function = RPNElement::FUNCTION_LITERAL_TUPLE; - out.tuple_literal = const_value.get(); + out.tuple_literal = const_value.safeGet(); out.func_name = "Tuple literal"; return true; } @@ -300,7 +300,7 @@ bool ApproximateNearestNeighborCondition::tryCastToConstType(const ASTPtr & node if (const_value.getType() == Field::Types::Array) { out.function = RPNElement::FUNCTION_LITERAL_ARRAY; - out.array_literal = const_value.get(); + out.array_literal = const_value.safeGet(); out.func_name = "Array literal"; return true; } @@ -308,7 +308,7 @@ bool ApproximateNearestNeighborCondition::tryCastToConstType(const ASTPtr & node if (const_value.getType() == Field::Types::String) { out.function = RPNElement::FUNCTION_STRING_LITERAL; - out.func_name = const_value.get(); + out.func_name = const_value.safeGet(); return true; } } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 3a44359b537..0db05373e43 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -428,7 +428,7 @@ std::pair IMergeTreeDataPart::getMinMaxDate() const if (storage.minmax_idx_date_column_pos != -1 && minmax_idx->initialized) { const auto & hyperrectangle = minmax_idx->hyperrectangle[storage.minmax_idx_date_column_pos]; - return {DayNum(hyperrectangle.left.get()), DayNum(hyperrectangle.right.get())}; + return {DayNum(hyperrectangle.left.safeGet()), DayNum(hyperrectangle.right.safeGet())}; } else return {}; @@ -444,15 +444,15 @@ std::pair IMergeTreeDataPart::getMinMaxTime() const if (hyperrectangle.left.getType() == Field::Types::UInt64) { assert(hyperrectangle.right.getType() == Field::Types::UInt64); - return {hyperrectangle.left.get(), hyperrectangle.right.get()}; + return {hyperrectangle.left.safeGet(), hyperrectangle.right.safeGet()}; } /// The case of DateTime64 else if (hyperrectangle.left.getType() == Field::Types::Decimal64) { assert(hyperrectangle.right.getType() == Field::Types::Decimal64); - auto left = hyperrectangle.left.get>(); - auto right = hyperrectangle.right.get>(); + auto left = hyperrectangle.left.safeGet>(); + auto right = hyperrectangle.right.safeGet>(); assert(left.getScale() == right.getScale()); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 69bffac9160..a717c08f0a2 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -349,7 +349,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = extractFixedPrefixFromLikePattern(value.get(), /*requires_perfect_prefix*/ false); + String prefix = extractFixedPrefixFromLikePattern(value.safeGet(), /*requires_perfect_prefix*/ false); if (prefix.empty()) return false; @@ -370,7 +370,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = extractFixedPrefixFromLikePattern(value.get(), /*requires_perfect_prefix*/ true); + String prefix = extractFixedPrefixFromLikePattern(value.safeGet(), /*requires_perfect_prefix*/ true); if (prefix.empty()) return false; @@ -391,7 +391,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = value.get(); + String prefix = value.safeGet(); if (prefix.empty()) return false; @@ -412,7 +412,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - const String & expression = value.get(); + const String & expression = value.safeGet(); /// This optimization can't process alternation - this would require /// a comprehensive parsing of regular expression. @@ -2918,8 +2918,8 @@ BoolMask KeyCondition::checkInHyperrectangle( /// Let's support only the case of 2d, because I'm not confident in other cases. if (num_dimensions == 2) { - UInt64 left = key_range.left.get(); - UInt64 right = key_range.right.get(); + UInt64 left = key_range.left.safeGet(); + UInt64 right = key_range.right.safeGet(); BoolMask mask(false, true); auto hyperrectangle_intersection_callback = [&](std::array, 2> curve_hyperrectangle) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2e10f5a0227..830f6f5d52f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5869,7 +5869,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc if (partition_lit && partition_lit->value.getType() == Field::Types::String) { MergeTreePartInfo::validatePartitionID(partition_ast.value->clone(), format_version); - return partition_lit->value.get(); + return partition_lit->value.safeGet(); } } @@ -5932,7 +5932,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", partition_key_value.getTypeName()); - const Tuple & tuple = partition_key_value.get(); + const Tuple & tuple = partition_key_value.safeGet(); if (tuple.size() != fields_count) throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong number of fields in the partition expression: {}, must be: {}", tuple.size(), fields_count); @@ -6868,7 +6868,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( auto * place = arena.alignedAlloc(size_of_state, align_of_state); func->create(place); if (const AggregateFunctionCount * agg_count = typeid_cast(func.get())) - AggregateFunctionCount::set(place, value.get()); + AggregateFunctionCount::set(place, value.safeGet()); else { auto value_column = func->getArgumentTypes().front()->createColumnConst(1, value)->convertToFullColumnIfConst(); @@ -7510,7 +7510,7 @@ MergeTreeData::MatcherFn MergeTreeData::getPartitionMatcher(const ASTPtr & parti if (const auto * partition_lit = partition_ast->as().value->as()) { id = partition_lit->value.getType() == Field::Types::UInt64 - ? toString(partition_lit->value.get()) + ? toString(partition_lit->value.safeGet()) : partition_lit->value.safeGet(); prefixed = true; } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index ee3ac4207cc..98f1563546e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -441,8 +441,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( String part_name; if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { - DayNum min_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].left.get()); - DayNum max_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].right.get()); + DayNum min_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].left.safeGet()); + DayNum max_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].right.safeGet()); const auto & date_lut = DateLUT::serverTimezoneInstance(); diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 497e86334f3..b68e48eeb3a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -343,12 +343,12 @@ MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) static constexpr auto DEFAULT_DISTANCE_FUNCTION = DISTANCE_FUNCTION_L2; String distance_function = DEFAULT_DISTANCE_FUNCTION; if (!index.arguments.empty()) - distance_function = index.arguments[0].get(); + distance_function = index.arguments[0].safeGet(); static constexpr auto DEFAULT_TREES = 100uz; UInt64 trees = DEFAULT_TREES; if (index.arguments.size() > 1) - trees = index.arguments[1].get(); + trees = index.arguments[1].safeGet(); return std::make_shared(index, trees, distance_function); } @@ -375,7 +375,7 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (!index.arguments.empty()) { - String distance_name = index.arguments[0].get(); + String distance_name = index.arguments[0].safeGet(); if (distance_name != DISTANCE_FUNCTION_L2 && distance_name != DISTANCE_FUNCTION_COSINE) throw Exception(ErrorCodes::INCORRECT_DATA, "Annoy index only supports distance functions '{}' and '{}'", DISTANCE_FUNCTION_L2, DISTANCE_FUNCTION_COSINE); } diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index c6a00751f25..be0ee693e15 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -348,19 +348,19 @@ bool MergeTreeIndexConditionBloomFilter::extractAtomFromTree(const RPNBuilderTre { if (const_value.getType() == Field::Types::UInt64) { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } if (const_value.getType() == Field::Types::Int64) { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } if (const_value.getType() == Field::Types::Float64) { - out.function = const_value.get() != 0.0 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() != 0.0 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } } @@ -692,7 +692,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeEquals( const bool is_nullable = actual_type->isNullable(); auto mutable_column = actual_type->createColumn(); - for (const auto & f : value_field.get()) + for (const auto & f : value_field.safeGet()) { if ((f.isNull() && !is_nullable) || f.isDecimal(f.getType())) /// NOLINT(readability-static-accessed-through-instance) return false; @@ -763,7 +763,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeEquals( if (which.isTuple() && key_node_function_name == "tuple") { - const Tuple & tuple = value_field.get(); + const Tuple & tuple = value_field.safeGet(); const auto * value_tuple_data_type = typeid_cast(value_type.get()); if (tuple.size() != key_node_function_arguments_size) @@ -952,7 +952,7 @@ void bloomFilterIndexValidator(const IndexDescription & index, bool attach) { const auto & argument = index.arguments[0]; - if (!attach && (argument.getType() != Field::Types::Float64 || argument.get() < 0 || argument.get() > 1)) + if (!attach && (argument.getType() != Field::Types::Float64 || argument.safeGet() < 0 || argument.safeGet() > 1)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The BloomFilter false positive must be a double number between 0 and 1."); } } diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp index 5b6813d12e3..857b7903588 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp @@ -341,19 +341,19 @@ bool MergeTreeConditionBloomFilterText::extractAtomFromTree(const RPNBuilderTree if (const_value.getType() == Field::Types::UInt64) { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } if (const_value.getType() == Field::Types::Int64) { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } if (const_value.getType() == Field::Types::Float64) { - out.function = const_value.get() != 0.0 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() != 0.0 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } } @@ -493,7 +493,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); - auto value = const_value.get(); + auto value = const_value.safeGet(); if (is_case_insensitive_scenario) std::ranges::transform(value, value.begin(), [](const auto & c) { return static_cast(std::tolower(c)); }); @@ -509,7 +509,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_HAS; out.bloom_filter = std::make_unique(params); - auto & value = const_value.get(); + auto & value = const_value.safeGet(); token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter); return true; } @@ -519,7 +519,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_HAS; out.bloom_filter = std::make_unique(params); - auto & value = const_value.get(); + auto & value = const_value.safeGet(); token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter); return true; } @@ -529,7 +529,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_NOT_EQUALS; out.bloom_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter); return true; } @@ -538,7 +538,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter); return true; } @@ -547,7 +547,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringLikeToBloomFilter(value.data(), value.size(), *out.bloom_filter); return true; } @@ -556,7 +556,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_NOT_EQUALS; out.bloom_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringLikeToBloomFilter(value.data(), value.size(), *out.bloom_filter); return true; } @@ -565,7 +565,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->substringToBloomFilter(value.data(), value.size(), *out.bloom_filter, true, false); return true; } @@ -574,7 +574,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->substringToBloomFilter(value.data(), value.size(), *out.bloom_filter, false, true); return true; } @@ -589,13 +589,13 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( /// 2d vector is not needed here but is used because already exists for FUNCTION_IN std::vector> bloom_filters; bloom_filters.emplace_back(); - for (const auto & element : const_value.get()) + for (const auto & element : const_value.safeGet()) { if (element.getType() != Field::Types::String) return false; bloom_filters.back().emplace_back(params); - const auto & value = element.get(); + const auto & value = element.safeGet(); if (function_name == "multiSearchAny") { @@ -615,7 +615,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.function = RPNElement::FUNCTION_MATCH; out.bloom_filter = std::make_unique(params); - auto & value = const_value.get(); + auto & value = const_value.safeGet(); String required_substring; bool dummy_is_trivial, dummy_required_substring_is_prefix; std::vector alternatives; @@ -743,11 +743,11 @@ MergeTreeIndexPtr bloomFilterIndexTextCreator( { if (index.type == NgramTokenExtractor::getName()) { - size_t n = index.arguments[0].get(); + size_t n = index.arguments[0].safeGet(); BloomFilterParameters params( - index.arguments[1].get(), - index.arguments[2].get(), - index.arguments[3].get()); + index.arguments[1].safeGet(), + index.arguments[2].safeGet(), + index.arguments[3].safeGet()); auto tokenizer = std::make_unique(n); @@ -756,9 +756,9 @@ MergeTreeIndexPtr bloomFilterIndexTextCreator( else if (index.type == SplitTokenExtractor::getName()) { BloomFilterParameters params( - index.arguments[0].get(), - index.arguments[1].get(), - index.arguments[2].get()); + index.arguments[0].safeGet(), + index.arguments[1].safeGet(), + index.arguments[2].safeGet()); auto tokenizer = std::make_unique(); @@ -815,9 +815,9 @@ void bloomFilterIndexTextValidator(const IndexDescription & index, bool /*attach /// Just validate BloomFilterParameters params( - index.arguments[0].get(), - index.arguments[1].get(), - index.arguments[2].get()); + index.arguments[0].safeGet(), + index.arguments[1].safeGet(), + index.arguments[2].safeGet()); } } diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index cd6af68ebcc..b5c6bb95d37 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -74,7 +74,7 @@ void MergeTreeIndexGranuleFullText::deserializeBinary(ReadBuffer & istr, MergeTr for (auto & gin_filter : gin_filters) { size_serialization->deserializeBinary(field_rows, istr, {}); - size_t filter_size = field_rows.get(); + size_t filter_size = field_rows.safeGet(); gin_filter.getFilter().resize(filter_size); if (filter_size == 0) @@ -379,19 +379,19 @@ bool MergeTreeConditionFullText::traverseAtomAST(const RPNBuilderTreeNode & node /// Check constant like in KeyCondition if (const_value.getType() == Field::Types::UInt64) { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } if (const_value.getType() == Field::Types::Int64) { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } if (const_value.getType() == Field::Types::Float64) { - out.function = const_value.get() != 0.00 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() != 0.00 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } } @@ -530,7 +530,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_HAS; out.gin_filter = std::make_unique(params); - auto & value = const_value.get(); + auto & value = const_value.safeGet(); token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter); return true; } @@ -539,7 +539,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_HAS; out.gin_filter = std::make_unique(params); - auto & value = const_value.get(); + auto & value = const_value.safeGet(); token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter); return true; } @@ -549,7 +549,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_NOT_EQUALS; out.gin_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter); return true; } @@ -558,7 +558,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_EQUALS; out.gin_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter); return true; } @@ -567,7 +567,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_EQUALS; out.gin_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringLikeToGinFilter(value.data(), value.size(), *out.gin_filter); return true; } @@ -576,7 +576,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_NOT_EQUALS; out.gin_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringLikeToGinFilter(value.data(), value.size(), *out.gin_filter); return true; } @@ -585,7 +585,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_EQUALS; out.gin_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter); return true; } @@ -594,7 +594,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_EQUALS; out.gin_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->substringToGinFilter(value.data(), value.size(), *out.gin_filter, true, false); return true; } @@ -603,7 +603,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_EQUALS; out.gin_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->substringToGinFilter(value.data(), value.size(), *out.gin_filter, false, true); return true; } @@ -615,13 +615,13 @@ bool MergeTreeConditionFullText::traverseASTEquals( /// 2d vector is not needed here but is used because already exists for FUNCTION_IN std::vector gin_filters; gin_filters.emplace_back(); - for (const auto & element : const_value.get()) + for (const auto & element : const_value.safeGet()) { if (element.getType() != Field::Types::String) return false; gin_filters.back().emplace_back(params); - const auto & value = element.get(); + const auto & value = element.safeGet(); token_extractor->substringToGinFilter(value.data(), value.size(), gin_filters.back().back(), false, false); } out.set_gin_filters = std::move(gin_filters); @@ -632,7 +632,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_MATCH; - auto & value = const_value.get(); + auto & value = const_value.safeGet(); String required_substring; bool dummy_is_trivial, dummy_required_substring_is_prefix; std::vector alternatives; @@ -776,8 +776,8 @@ MergeTreeIndexConditionPtr MergeTreeIndexFullText::createIndexCondition( MergeTreeIndexPtr fullTextIndexCreator( const IndexDescription & index) { - size_t n = index.arguments.empty() ? 0 : index.arguments[0].get(); - UInt64 max_rows = index.arguments.size() < 2 ? DEFAULT_MAX_ROWS_PER_POSTINGS_LIST : index.arguments[1].get(); + size_t n = index.arguments.empty() ? 0 : index.arguments[0].safeGet(); + UInt64 max_rows = index.arguments.size() < 2 ? DEFAULT_MAX_ROWS_PER_POSTINGS_LIST : index.arguments[1].safeGet(); GinFilterParameters params(n, max_rows); /// Use SplitTokenExtractor when n is 0, otherwise use NgramTokenExtractor @@ -826,12 +826,12 @@ void fullTextIndexValidator(const IndexDescription & index, bool /*attach*/) { if (index.arguments[1].getType() != Field::Types::UInt64) throw Exception(ErrorCodes::INCORRECT_QUERY, "The second full text index argument must be UInt64"); - if (index.arguments[1].get() != UNLIMITED_ROWS_PER_POSTINGS_LIST && index.arguments[1].get() < MIN_ROWS_PER_POSTINGS_LIST) + if (index.arguments[1].safeGet() != UNLIMITED_ROWS_PER_POSTINGS_LIST && index.arguments[1].safeGet() < MIN_ROWS_PER_POSTINGS_LIST) throw Exception(ErrorCodes::INCORRECT_QUERY, "The maximum rows per postings list must be no less than {}", MIN_ROWS_PER_POSTINGS_LIST); } /// Just validate - size_t ngrams = index.arguments.empty() ? 0 : index.arguments[0].get(); - UInt64 max_rows_per_postings_list = index.arguments.size() < 2 ? DEFAULT_MAX_ROWS_PER_POSTINGS_LIST : index.arguments[1].get(); + size_t ngrams = index.arguments.empty() ? 0 : index.arguments[0].safeGet(); + UInt64 max_rows_per_postings_list = index.arguments.size() < 2 ? DEFAULT_MAX_ROWS_PER_POSTINGS_LIST : index.arguments[1].safeGet(); GinFilterParameters params(ngrams, max_rows_per_postings_list); } } diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp index cd8065ecadf..abf3ae56376 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp @@ -37,7 +37,7 @@ void MergeTreeIndexGranuleHypothesis::deserializeBinary(ReadBuffer & istr, Merge Field field_met; const auto & size_type = DataTypePtr(std::make_shared()); size_type->getDefaultSerialization()->deserializeBinary(field_met, istr, {}); - met = field_met.get(); + met = field_met.safeGet(); is_empty = false; } diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index a92df4ac72d..fa242fccbc1 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -97,7 +97,7 @@ void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr, MergeTreeInd Field field_rows; const auto & size_type = DataTypePtr(std::make_shared()); size_type->getDefaultSerialization()->deserializeBinary(field_rows, istr, {}); - size_t rows_to_read = field_rows.get(); + size_t rows_to_read = field_rows.safeGet(); if (rows_to_read == 0) return; @@ -591,7 +591,7 @@ MergeTreeIndexConditionPtr MergeTreeIndexSet::createIndexCondition( MergeTreeIndexPtr setIndexCreator(const IndexDescription & index) { - size_t max_rows = index.arguments[0].get(); + size_t max_rows = index.arguments[0].safeGet(); return std::make_shared(index, max_rows); } diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index 59a4b0fbf9c..efd9bb754e1 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -377,12 +377,12 @@ MergeTreeIndexPtr usearchIndexCreator(const IndexDescription & index) static constexpr auto default_distance_function = DISTANCE_FUNCTION_L2; String distance_function = default_distance_function; if (!index.arguments.empty()) - distance_function = index.arguments[0].get(); + distance_function = index.arguments[0].safeGet(); static constexpr auto default_scalar_kind = unum::usearch::scalar_kind_t::f16_k; auto scalar_kind = default_scalar_kind; if (index.arguments.size() > 1) - scalar_kind = nameToScalarKind.at(index.arguments[1].get()); + scalar_kind = nameToScalarKind.at(index.arguments[1].safeGet()); return std::make_shared(index, distance_function, scalar_kind); } @@ -408,14 +408,14 @@ void usearchIndexValidator(const IndexDescription & index, bool /* attach */) if (!index.arguments.empty()) { - String distance_name = index.arguments[0].get(); + String distance_name = index.arguments[0].safeGet(); if (distance_name != DISTANCE_FUNCTION_L2 && distance_name != DISTANCE_FUNCTION_COSINE) throw Exception(ErrorCodes::INCORRECT_DATA, "USearch index only supports distance functions '{}' and '{}'", DISTANCE_FUNCTION_L2, DISTANCE_FUNCTION_COSINE); } /// Check that a supported kind was passed as a second argument - if (index.arguments.size() > 1 && !nameToScalarKind.contains(index.arguments[1].get())) + if (index.arguments.size() > 1 && !nameToScalarKind.contains(index.arguments[1].safeGet())) { String supported_kinds; for (const auto & [name, kind] : nameToScalarKind) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index b240f80ee13..5b5bc244f92 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -241,7 +241,7 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const if (typeid_cast(partition_key_sample.getByPosition(i).type.get())) result += toString(DateLUT::serverTimezoneInstance().toNumYYYYMMDD(DayNum(value[i].safeGet()))); else if (typeid_cast(partition_key_sample.getByPosition(i).type.get())) - result += toString(value[i].get().toUnderType()); + result += toString(value[i].safeGet().toUnderType()); else result += applyVisitor(to_string_visitor, value[i]); diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 43c40dee77d..f0c26c302e1 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -222,17 +222,17 @@ static bool isConditionGood(const RPNBuilderTreeNode & condition, const NameSet /// check the value with respect to threshold if (type == Field::Types::UInt64) { - const auto value = output_value.get(); + const auto value = output_value.safeGet(); return value > threshold; } else if (type == Field::Types::Int64) { - const auto value = output_value.get(); + const auto value = output_value.safeGet(); return value < -threshold || threshold < value; } else if (type == Field::Types::Float64) { - const auto value = output_value.get(); + const auto value = output_value.safeGet(); return value < -threshold || threshold < value; } diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 3f0603f6900..52857845e99 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -591,7 +591,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (ast->value.getType() != Field::Types::String) throw Exception(ErrorCodes::BAD_ARGUMENTS, format_str, error_msg); - graphite_config_name = ast->value.get(); + graphite_config_name = ast->value.safeGet(); } else throw Exception(ErrorCodes::BAD_ARGUMENTS, format_str, error_msg); diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index ade3326288a..fd9f29b6375 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -32,19 +32,19 @@ std::optional StatisticsUtils::tryConvertToFloat64(const Field & field) switch (field.getType()) { case Field::Types::Int64: - return field.get(); + return field.safeGet(); case Field::Types::UInt64: - return field.get(); + return field.safeGet(); case Field::Types::Float64: - return field.get(); + return field.safeGet(); case Field::Types::Int128: - return field.get(); + return field.safeGet(); case Field::Types::UInt128: - return field.get(); + return field.safeGet(); case Field::Types::Int256: - return field.get(); + return field.safeGet(); case Field::Types::UInt256: - return field.get(); + return field.safeGet(); default: return {}; } @@ -53,7 +53,7 @@ std::optional StatisticsUtils::tryConvertToFloat64(const Field & field) std::optional StatisticsUtils::tryConvertToString(const DB::Field & field) { if (field.getType() == Field::Types::String) - return field.get(); + return field.safeGet(); return {}; } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index e69bbc1515b..a6f2f883e65 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -48,7 +48,7 @@ Float64 StatisticsCountMinSketch::estimateEqual(const Field & val) const return sketch.get_estimate(&val_converted, data_type->getSizeOfValueInMemory()); if (isStringOrFixedString(data_type)) - return sketch.get_estimate(val.get()); + return sketch.get_estimate(val.safeGet()); throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'count_min' does not support estimate data type of {}", data_type->getName()); } diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index 060b271d8f4..b95ccedb093 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -203,7 +203,7 @@ StoragePtr StorageFactory::get( } if (query.comment) - comment = query.comment->as().value.get(); + comment = query.comment->as().value.safeGet(); ASTs empty_engine_args; Arguments arguments{ diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 98cd5c4dfa9..20b02b59a10 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -2203,11 +2203,11 @@ void registerStorageFile(StorageFactory & factory) { auto type = literal->value.getType(); if (type == Field::Types::Int64) - source_fd = static_cast(literal->value.get()); + source_fd = static_cast(literal->value.safeGet()); else if (type == Field::Types::UInt64) - source_fd = static_cast(literal->value.get()); + source_fd = static_cast(literal->value.safeGet()); else if (type == Field::Types::String) - StorageFile::parseFileSource(literal->value.get(), source_path, storage_args.path_to_archive); + StorageFile::parseFileSource(literal->value.safeGet(), source_path, storage_args.path_to_archive); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second argument must be path or file descriptor"); } diff --git a/src/Storages/StorageFuzzJSON.cpp b/src/Storages/StorageFuzzJSON.cpp index 9950d41f1c2..fc73f246d35 100644 --- a/src/Storages/StorageFuzzJSON.cpp +++ b/src/Storages/StorageFuzzJSON.cpp @@ -419,7 +419,7 @@ void fuzzJSONObject( if (val.fixed->getType() == Field::Types::Which::String) { out << fuzzJSONStructure(config, rnd, "\""); - writeText(val.fixed->get(), out); + writeText(val.fixed->safeGet(), out); out << fuzzJSONStructure(config, rnd, "\""); } else diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index a0d6cf11b64..136df425813 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -341,10 +341,10 @@ void registerStorageJoin(StorageFactory & factory) else if (setting.name == "any_join_distinct_right_table_keys") old_any_join = setting.value; else if (setting.name == "disk") - disk_name = setting.value.get(); + disk_name = setting.value.safeGet(); else if (setting.name == "persistent") { - persistent = setting.value.get(); + persistent = setting.value.safeGet(); } else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown setting {} for storage {}", setting.name, args.engine_name); diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index d3214e7ed13..e0a4af68824 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -294,7 +294,7 @@ public: { const auto * array_type = typeid_cast(data_type.get()); const auto & nested = array_type->getNestedType(); - const auto & array = array_field.get(); + const auto & array = array_field.safeGet(); if (!isArray(nested)) { @@ -312,7 +312,7 @@ public: if (!isArray(nested_array_type->getNestedType())) { - parseArrayContent(iter->get(), nested, ostr); + parseArrayContent(iter->safeGet(), nested, ostr); } else { diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 6a7810b97f9..522884b4fef 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -105,8 +105,8 @@ protected: while (rows_count < max_block_size && db_table_num < total_tables) { - const std::string database_name = (*databases)[db_table_num].get(); - const std::string table_name = (*tables)[db_table_num].get(); + const std::string database_name = (*databases)[db_table_num].safeGet(); + const std::string table_name = (*tables)[db_table_num].safeGet(); ++db_table_num; ColumnsDescription columns; @@ -426,7 +426,7 @@ void ReadFromSystemColumns::initializePipeline(QueryPipelineBuilder & pipeline, for (size_t i = 0; i < num_databases; ++i) { - const std::string database_name = (*database_column)[i].get(); + const std::string database_name = (*database_column)[i].safeGet(); if (database_name.empty()) { for (auto & [table_name, table] : external_tables) diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 7ace8ee24aa..c87bdb6d26a 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -138,7 +138,7 @@ StoragesInfoStream::StoragesInfoStream(std::optional filter_by_datab for (size_t i = 0; i < rows; ++i) { - String database_name = (*database_column_for_filter)[i].get(); + String database_name = (*database_column_for_filter)[i].safeGet(); const DatabasePtr database = databases.at(database_name); offsets[i] = i ? offsets[i - 1] : 0; diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 806af4a7bf8..3be73aeda17 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -52,13 +52,13 @@ public: { StoragesInfo info; - info.database = (*database_column)[next_row].get(); - info.table = (*table_column)[next_row].get(); - UUID storage_uuid = (*storage_uuid_column)[next_row].get(); + info.database = (*database_column)[next_row].safeGet(); + info.table = (*table_column)[next_row].safeGet(); + UUID storage_uuid = (*storage_uuid_column)[next_row].safeGet(); auto is_same_table = [&storage_uuid, this] (size_t row) -> bool { - return (*storage_uuid_column)[row].get() == storage_uuid; + return (*storage_uuid_column)[row].safeGet() == storage_uuid; }; /// We may have two rows per table which differ in 'active' value. @@ -66,7 +66,7 @@ public: /// must collect the inactive parts. Remember this fact in StoragesInfo. for (; next_row < rows && is_same_table(next_row); ++next_row) { - const auto active = (*active_column)[next_row].get(); + const auto active = (*active_column)[next_row].safeGet(); if (active == 0) info.need_inactive_parts = true; } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 73b7908b75c..ea837da1e73 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -51,7 +51,7 @@ auto extractSingleValueFromBlock(const Block & block, const String & name) const ColumnWithTypeAndName & data = block.getByName(name); size_t rows = block.rows(); for (size_t i = 0; i < rows; ++i) - res.insert((*data.column)[i].get()); + res.insert((*data.column)[i].safeGet()); return res; } diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 9d23f132759..1408e120bc5 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -102,16 +102,16 @@ ColumnsDescription getStructureOfRemoteTableInShard( { ColumnDescription column; - column.name = (*name)[i].get(); + column.name = (*name)[i].safeGet(); - String data_type_name = (*type)[i].get(); + String data_type_name = (*type)[i].safeGet(); column.type = data_type_factory.get(data_type_name); - String kind_name = (*default_kind)[i].get(); + String kind_name = (*default_kind)[i].safeGet(); if (!kind_name.empty()) { column.default_desc.kind = columnDefaultKindFromString(kind_name); - String expr_str = (*default_expr)[i].get(); + String expr_str = (*default_expr)[i].safeGet(); column.default_desc.expression = parseQuery( expr_parser, expr_str.data(), expr_str.data() + expr_str.size(), "default expression", 0, settings.max_parser_depth, settings.max_parser_backtracks); @@ -207,8 +207,8 @@ ColumnsDescriptionByShardNum getExtendedObjectsOfRemoteTables( size_t size = name_col.size(); for (size_t i = 0; i < size; ++i) { - auto name = name_col[i].get(); - auto type_name = type_col[i].get(); + auto name = name_col[i].safeGet(); + auto type_name = type_col[i].safeGet(); auto storage_column = storage_columns.tryGetPhysical(name); if (storage_column && storage_column->type->hasDynamicSubcolumnsDeprecated()) diff --git a/src/TableFunctions/TableFunctionExplain.cpp b/src/TableFunctions/TableFunctionExplain.cpp index 552b9fde986..69d24c879bd 100644 --- a/src/TableFunctions/TableFunctionExplain.cpp +++ b/src/TableFunctions/TableFunctionExplain.cpp @@ -83,7 +83,7 @@ void TableFunctionExplain::parseArguments(const ASTPtr & ast_function, ContextPt "Table function '{}' requires a String argument for EXPLAIN kind, got '{}'", getName(), queryToString(kind_arg)); - ASTExplainQuery::ExplainKind kind = ASTExplainQuery::fromString(kind_literal->value.get()); + ASTExplainQuery::ExplainKind kind = ASTExplainQuery::fromString(kind_literal->value.safeGet()); auto explain_query = std::make_shared(kind); const auto * settings_arg = function->arguments->children[1]->as(); @@ -92,7 +92,7 @@ void TableFunctionExplain::parseArguments(const ASTPtr & ast_function, ContextPt "Table function '{}' requires a serialized string settings argument, got '{}'", getName(), queryToString(function->arguments->children[1])); - const auto & settings_str = settings_arg->value.get(); + const auto & settings_str = settings_arg->value.safeGet(); if (!settings_str.empty()) { const Settings & settings = context->getSettingsRef(); diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 1b6d86f8fa5..e59ee52fd82 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -47,7 +47,7 @@ void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, const ContextPtr else if (type == Field::Types::Int64 || type == Field::Types::UInt64) { fd = static_cast( - (type == Field::Types::Int64) ? literal->value.get() : literal->value.get()); + (type == Field::Types::Int64) ? literal->value.safeGet() : literal->value.safeGet()); if (fd < 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, "File descriptor must be non-negative"); } diff --git a/src/TableFunctions/TableFunctionMergeTreeIndex.cpp b/src/TableFunctions/TableFunctionMergeTreeIndex.cpp index 06a48f0e25f..27ed50fb711 100644 --- a/src/TableFunctions/TableFunctionMergeTreeIndex.cpp +++ b/src/TableFunctions/TableFunctionMergeTreeIndex.cpp @@ -76,9 +76,9 @@ void TableFunctionMergeTreeIndex::parseArguments(const ASTPtr & ast_function, Co "Table function '{}' expected bool flag for 'with_marks' argument", getName()); if (value.getType() == Field::Types::Bool) - with_marks = value.get(); + with_marks = value.safeGet(); else - with_marks = value.get(); + with_marks = value.safeGet(); } if (!params.empty()) From 18d9bb2ade4e98051df007663a387eb74146c26f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 10 Aug 2024 16:25:32 +0200 Subject: [PATCH 538/644] tests: attempt to fix 01600_parts_states_metrics_long (by forbid parallel run) CI: https://s3.amazonaws.com/clickhouse-test-reports/68134/8d4f822fee64d44440459b733c67dee5e9fb1e02/stateless_tests__tsan__s3_storage__[2_4].html Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/01600_parts_states_metrics_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh index 47b5a4dea13..8062bb0ba5d 100755 --- a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh +++ b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From eeda67042c08bedbd18c3b7a76cb8928e9975348 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 10 Aug 2024 16:28:49 +0200 Subject: [PATCH 539/644] tests: make 01600_parts_states_metrics_long faster Signed-off-by: Azat Khuzhin --- .../01600_parts_states_metrics_long.sh | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh index 8062bb0ba5d..a07dd306b3e 100755 --- a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh +++ b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh @@ -5,6 +5,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +function query() +{ + # NOTE: database_atomic_wait_for_drop_and_detach_synchronously needed only for local env, CI has it ON + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&database_atomic_wait_for_drop_and_detach_synchronously=1" -d "$*" +} + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted verify_sql="SELECT (SELECT sumIf(value, metric = 'PartsActive'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics) @@ -18,13 +24,13 @@ verify() { for i in {1..5000} do - result=$( $CLICKHOUSE_CLIENT --query="$verify_sql" ) + result=$( query "$verify_sql" ) [ "$result" = "1" ] && echo "$result" && break sleep 0.1 if [[ $i -eq 5000 ]] then - $CLICKHOUSE_CLIENT " + query " SELECT sumIf(value, metric = 'PartsActive'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics; SELECT sum(active), sum(NOT active) FROM system.parts; SELECT sum(active), sum(NOT active) FROM system.projection_parts; @@ -34,17 +40,17 @@ verify() done } -$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=1 --query="DROP TABLE IF EXISTS test_table" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test_table (data Date) ENGINE = MergeTree PARTITION BY toYear(data) ORDER BY data;" +query "DROP TABLE IF EXISTS test_table" +query "CREATE TABLE test_table (data Date) ENGINE = MergeTree PARTITION BY toYear(data) ORDER BY data;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test_table VALUES ('1992-01-01')" +query "INSERT INTO test_table VALUES ('1992-01-01')" verify -$CLICKHOUSE_CLIENT --query="INSERT INTO test_table VALUES ('1992-01-02')" +query "INSERT INTO test_table VALUES ('1992-01-02')" verify -$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE test_table FINAL" +query "OPTIMIZE TABLE test_table FINAL" verify -$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=1 --query="DROP TABLE test_table" +query "DROP TABLE test_table" verify From 0a8fb05ece2771439844c03456d43b02eb8f51cd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 10 Aug 2024 16:23:23 +0000 Subject: [PATCH 540/644] fix after merge --- src/Core/Field.h | 7 +++++++ src/Disks/DiskFomAST.cpp | 2 +- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Core/Field.h b/src/Core/Field.h index 689ac38a235..13741183f21 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -869,6 +869,13 @@ constexpr bool isInt64OrUInt64FieldType(Field::Types::Which t) || t == Field::Types::UInt64; } +constexpr bool isInt64OrUInt64orBoolFieldType(Field::Types::Which t) +{ + return t == Field::Types::Int64 + || t == Field::Types::UInt64 + || t == Field::Types::Bool; +} + template auto & Field::safeGet() { diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index b2f1280c507..5329ff8748a 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -132,7 +132,7 @@ std::string DiskFomAST::createCustomDisk(const ASTPtr & disk_function_ast, Conte FlattenDiskConfigurationVisitor::Data data{context, attach}; FlattenDiskConfigurationVisitor{data}.visit(ast); - return assert_cast(*ast).value.get(); + return assert_cast(*ast).value.safeGet(); } void DiskFomAST::ensureDiskIsNotCustom(const std::string & disk_name, ContextPtr context) From 0a536cbf150916bff0ec63c6ef3b77df517868f8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sat, 10 Aug 2024 18:58:28 +0200 Subject: [PATCH 541/644] Add batch size --- docker/test/stateless/run.sh | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index a030be92506..17e39487e3e 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -193,8 +193,8 @@ ORDER BY tuple()" # create minio log webhooks for both audit and server logs # use async inserts to avoid creating too many parts -./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" queue_size=1000000 -./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" queue_size=1000000 +./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" queue_size=1000000 batch_size=500 +./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" queue_size=1000000 batch_size=500 max_retries=100 retry=1 @@ -376,6 +376,8 @@ done # collect minio audit and server logs +# wait for minio to flush its batch if it has any +sleep 1 clickhouse-client -q "SYSTEM FLUSH ASYNC INSERT QUEUE" clickhouse-client -q "SELECT log FROM minio_audit_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_audit_logs.jsonl.zst' FORMAT JSONEachRow" clickhouse-client -q "SELECT log FROM minio_server_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_server_logs.jsonl.zst' FORMAT JSONEachRow" From d53513a81a10b8230a30fdbb386aca1d067cbcfa Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 10 Aug 2024 18:12:34 +0000 Subject: [PATCH 542/644] fix --- tests/queries/0_stateless/01710_projection_vertical_merges.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01710_projection_vertical_merges.sql b/tests/queries/0_stateless/01710_projection_vertical_merges.sql index 0f80d659e92..0d745e44b10 100644 --- a/tests/queries/0_stateless/01710_projection_vertical_merges.sql +++ b/tests/queries/0_stateless/01710_projection_vertical_merges.sql @@ -1,4 +1,5 @@ --- Tags: long +-- Tags: long, no-parallel +-- set no-parallel tag is to prevent timeout of this test drop table if exists t; From 80e926996319b926f3cbebf2050ed4a60666ee71 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 10 Aug 2024 19:39:59 +0000 Subject: [PATCH 543/644] allow UInt64 <-> Int64 conversion --- src/Core/Field.h | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Core/Field.h b/src/Core/Field.h index 13741183f21..ba8c66580ad 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -882,8 +882,11 @@ auto & Field::safeGet() const Types::Which target = TypeToEnum>>::value; /// bool is stored as uint64, will be returned as UInt64 when requested as bool or UInt64, as Int64 when requested as Int64 - if (target != which && !(which == Field::Types::Bool && (target == Field::Types::UInt64 || target == Field::Types::Int64))) - throw Exception(ErrorCodes::BAD_GET, "Bad get: has {}, requested {}", getTypeName(), target); + /// also allow UInt64 <-> Int64 conversion + if (target != which && + !(which == Field::Types::Bool && (target == Field::Types::UInt64 || target == Field::Types::Int64)) && + !(isInt64OrUInt64FieldType(which) && isInt64OrUInt64FieldType(target))) + throw Exception(ErrorCodes::BAD_GET, "Bad get: has {}, requested {}", getTypeName(), target); return get(); } From 556f66987897bd5065426e187bef4e0cba2a975c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Sat, 10 Aug 2024 21:52:55 +0000 Subject: [PATCH 544/644] Separate test into separate file to pass bugfix check --- tests/integration/parallel_skip.json | 4 + tests/integration/test_storage_kafka/test.py | 135 ---------- .../test_produce_http_interface.py | 243 ++++++++++++++++++ 3 files changed, 247 insertions(+), 135 deletions(-) create mode 100644 tests/integration/test_storage_kafka/test_produce_http_interface.py diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 99fa626bd1e..fca2126d824 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -162,9 +162,13 @@ "test_storage_kafka/test.py::test_system_kafka_consumers_rebalance_mv", "test_storage_kafka/test.py::test_formats_errors", "test_storage_kafka/test.py::test_multiple_read_in_materialized_views", + "test_storage_kafka/test.py::test_kafka_null_message", + + "test_storage_kafka/test_produce_http_interface.py::test_kafka_produce_http_interface_row_based_format", "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string", "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_request_new_ticket_after_expiration", "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc", "test_storage_kerberized_kafka/test.py::test_kafka_config_from_sql_named_collection" + ] diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 9b2f465c1b6..4b6c9922d74 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -5524,141 +5524,6 @@ def test_kafka_null_message(kafka_cluster, create_query_generator): ) -def test_kafka_produce_http_interface_row_based_format(kafka_cluster): - # reproduction of #61060 with validating the written messages - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - - topic_prefix = "http_row_" - - # It is important to have: - # - long enough messages - # - enough messages - # I don't know the exact requirement for message sizes, but it doesn't reproduce with short messages - # For the number of messages it seems like at least 3 messages is necessary - expected_key = "01234567890123456789" - expected_value = "aaaaabbbbbccccc" - - insert_query_end = f"(key, value) VALUES ('{expected_key}', '{expected_value}'), ('{expected_key}', '{expected_value}'), ('{expected_key}', '{expected_value}')" - insert_query_template = "INSERT INTO {table_name} " + insert_query_end - - extra_settings = { - "Protobuf": ", kafka_schema = 'string_key_value.proto:StringKeyValuePair'", - "CapnProto": ", kafka_schema='string_key_value:StringKeyValuePair'", - "Template": ", format_template_row='string_key_value.format'", - } - - # Only the formats that can be used both and input and output format are tested - # Reasons to exclude following formats: - # - JSONStrings: not actually an input format - # - ProtobufSingle: I cannot make it work to parse the messages. Probably something is broken, - # because the producer can write multiple rows into a same message, which makes them impossible to parse properly. Should added after #67549 is fixed. - # - ProtobufList: I didn't want to deal with the envelope and stuff - # - Npy: supports only single column - # - LineAsString: supports only single column - # - RawBLOB: supports only single column - formats_to_test = [ - "TabSeparated", - "TabSeparatedRaw", - "TabSeparatedWithNames", - "TabSeparatedWithNamesAndTypes", - "TabSeparatedRawWithNames", - "TabSeparatedRawWithNamesAndTypes", - "Template", - "CSV", - "CSVWithNames", - "CSVWithNamesAndTypes", - "CustomSeparated", - "CustomSeparatedWithNames", - "CustomSeparatedWithNamesAndTypes", - "Values", - "JSON", - "JSONColumns", - "JSONColumnsWithMetadata", - "JSONCompact", - "JSONCompactColumns", - "JSONEachRow", - "JSONStringsEachRow", - "JSONCompactEachRow", - "JSONCompactEachRowWithNames", - "JSONCompactEachRowWithNamesAndTypes", - "JSONCompactStringsEachRow", - "JSONCompactStringsEachRowWithNames", - "JSONCompactStringsEachRowWithNamesAndTypes", - "JSONObjectEachRow", - "BSONEachRow", - "TSKV", - "Protobuf", - "Avro", - "Parquet", - "Arrow", - "ArrowStream", - "ORC", - "RowBinary", - "RowBinaryWithNames", - "RowBinaryWithNamesAndTypes", - "Native", - "CapnProto", - "MsgPack", - ] - for format in formats_to_test: - logging.debug(f"Creating tables and writing messages to {format}") - topic = topic_prefix + format - kafka_create_topic(admin_client, topic) - - extra_setting = extra_settings.get(format, "") - - # kafka_max_rows_per_message is set to 2 to make sure every format produces at least 2 messages, thus increasing the chance of catching a bug - instance.query( - f""" - DROP TABLE IF EXISTS test.view_{topic}; - DROP TABLE IF EXISTS test.consumer_{topic}; - CREATE TABLE test.kafka_writer_{topic} (key String, value String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic}', - kafka_group_name = '{topic}', - kafka_format = '{format}', - kafka_max_rows_per_message = 2 {extra_setting}; - - CREATE TABLE test.kafka_{topic} (key String, value String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic}', - kafka_group_name = '{topic}', - kafka_format = '{format}' {extra_setting}; - - CREATE MATERIALIZED VIEW test.view_{topic} Engine=Log AS - SELECT key, value FROM test.kafka_{topic}; - """ - ) - - instance.http_query( - insert_query_template.format(table_name="test.kafka_writer_" + topic), - method="POST", - ) - - expected = f"""\ -{expected_key}\t{expected_value} -{expected_key}\t{expected_value} -{expected_key}\t{expected_value} -""" - # give some times for the readers to read the messages - for format in formats_to_test: - logging.debug(f"Checking result for {format}") - topic = topic_prefix + format - - result = instance.query_with_retry( - f"SELECT * FROM test.view_{topic}", - check_callback=lambda res: res.count("\n") == 3, - ) - - assert TSV(result) == TSV(expected) - - kafka_delete_topic(admin_client, topic) - - if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_storage_kafka/test_produce_http_interface.py b/tests/integration/test_storage_kafka/test_produce_http_interface.py new file mode 100644 index 00000000000..fc10a07f239 --- /dev/null +++ b/tests/integration/test_storage_kafka/test_produce_http_interface.py @@ -0,0 +1,243 @@ +import time +import logging + +import pytest +from helpers.cluster import ClickHouseCluster, is_arm +from helpers.test_tools import TSV +from kafka import KafkaAdminClient +from kafka.admin import NewTopic + +if is_arm(): + pytestmark = pytest.mark.skip + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "instance", + main_configs=["configs/kafka.xml", "configs/named_collection.xml"], + user_configs=["configs/users.xml"], + with_kafka=True, + with_zookeeper=True, # For Replicated Table + macros={ + "kafka_broker": "kafka1", + "kafka_topic_old": "old", + "kafka_group_name_old": "old", + "kafka_topic_new": "new", + "kafka_group_name_new": "new", + "kafka_client_id": "instance", + "kafka_format_json_each_row": "JSONEachRow", + }, + clickhouse_path_dir="clickhouse_path", +) + + +@pytest.fixture(scope="module") +def kafka_cluster(): + try: + cluster.start() + kafka_id = instance.cluster.kafka_docker_id + print(("kafka_id is {}".format(kafka_id))) + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def kafka_setup_teardown(): + instance.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;") + # logging.debug("kafka is available - running test") + yield # run test + + +def kafka_create_topic( + admin_client, + topic_name, + num_partitions=1, + replication_factor=1, + max_retries=50, + config=None, +): + logging.debug( + f"Kafka create topic={topic_name}, num_partitions={num_partitions}, replication_factor={replication_factor}" + ) + topics_list = [ + NewTopic( + name=topic_name, + num_partitions=num_partitions, + replication_factor=replication_factor, + topic_configs=config, + ) + ] + retries = 0 + while True: + try: + admin_client.create_topics(new_topics=topics_list, validate_only=False) + logging.debug("Admin client succeed") + return + except Exception as e: + retries += 1 + time.sleep(0.5) + if retries < max_retries: + logging.warning(f"Failed to create topic {e}") + else: + raise + + +def kafka_delete_topic(admin_client, topic, max_retries=50): + result = admin_client.delete_topics([topic]) + for topic, e in result.topic_error_codes: + if e == 0: + logging.debug(f"Topic {topic} deleted") + else: + logging.error(f"Failed to delete topic {topic}: {e}") + + retries = 0 + while True: + topics_listed = admin_client.list_topics() + logging.debug(f"TOPICS LISTED: {topics_listed}") + if topic not in topics_listed: + return + else: + retries += 1 + time.sleep(0.5) + if retries > max_retries: + raise Exception(f"Failed to delete topics {topic}, {result}") + + +def test_kafka_produce_http_interface_row_based_format(kafka_cluster): + # reproduction of #61060 with validating the written messages + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + topic_prefix = "http_row_" + + # It is important to have: + # - long enough messages + # - enough messages + # I don't know the exact requirement for message sizes, but it doesn't reproduce with short messages + # For the number of messages it seems like at least 3 messages is necessary + expected_key = "01234567890123456789" + expected_value = "aaaaabbbbbccccc" + + insert_query_end = f"(key, value) VALUES ('{expected_key}', '{expected_value}'), ('{expected_key}', '{expected_value}'), ('{expected_key}', '{expected_value}')" + insert_query_template = "INSERT INTO {table_name} " + insert_query_end + + extra_settings = { + "Protobuf": ", kafka_schema = 'string_key_value.proto:StringKeyValuePair'", + "CapnProto": ", kafka_schema='string_key_value:StringKeyValuePair'", + "Template": ", format_template_row='string_key_value.format'", + } + + # Only the formats that can be used both and input and output format are tested + # Reasons to exclude following formats: + # - JSONStrings: not actually an input format + # - ProtobufSingle: I cannot make it work to parse the messages. Probably something is broken, + # because the producer can write multiple rows into a same message, which makes them impossible to parse properly. Should added after #67549 is fixed. + # - ProtobufList: I didn't want to deal with the envelope and stuff + # - Npy: supports only single column + # - LineAsString: supports only single column + # - RawBLOB: supports only single column + formats_to_test = [ + "TabSeparated", + "TabSeparatedRaw", + "TabSeparatedWithNames", + "TabSeparatedWithNamesAndTypes", + "TabSeparatedRawWithNames", + "TabSeparatedRawWithNamesAndTypes", + "Template", + "CSV", + "CSVWithNames", + "CSVWithNamesAndTypes", + "CustomSeparated", + "CustomSeparatedWithNames", + "CustomSeparatedWithNamesAndTypes", + "Values", + "JSON", + "JSONColumns", + "JSONColumnsWithMetadata", + "JSONCompact", + "JSONCompactColumns", + "JSONEachRow", + "JSONStringsEachRow", + "JSONCompactEachRow", + "JSONCompactEachRowWithNames", + "JSONCompactEachRowWithNamesAndTypes", + "JSONCompactStringsEachRow", + "JSONCompactStringsEachRowWithNames", + "JSONCompactStringsEachRowWithNamesAndTypes", + "JSONObjectEachRow", + "BSONEachRow", + "TSKV", + "Protobuf", + "Avro", + "Parquet", + "Arrow", + "ArrowStream", + "ORC", + "RowBinary", + "RowBinaryWithNames", + "RowBinaryWithNamesAndTypes", + "Native", + "CapnProto", + "MsgPack", + ] + for format in formats_to_test: + logging.debug(f"Creating tables and writing messages to {format}") + topic = topic_prefix + format + kafka_create_topic(admin_client, topic) + + extra_setting = extra_settings.get(format, "") + + # kafka_max_rows_per_message is set to 2 to make sure every format produces at least 2 messages, thus increasing the chance of catching a bug + instance.query( + f""" + DROP TABLE IF EXISTS test.view_{topic}; + DROP TABLE IF EXISTS test.consumer_{topic}; + CREATE TABLE test.kafka_writer_{topic} (key String, value String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic}', + kafka_group_name = '{topic}', + kafka_format = '{format}', + kafka_max_rows_per_message = 2 {extra_setting}; + + CREATE TABLE test.kafka_{topic} (key String, value String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic}', + kafka_group_name = '{topic}', + kafka_format = '{format}' {extra_setting}; + + CREATE MATERIALIZED VIEW test.view_{topic} Engine=Log AS + SELECT key, value FROM test.kafka_{topic}; + """ + ) + instance.http_query( + insert_query_template.format(table_name="test.kafka_writer_" + topic), + method="POST", + ) + + expected = f"""\ +{expected_key}\t{expected_value} +{expected_key}\t{expected_value} +{expected_key}\t{expected_value} +""" + # give some times for the readers to read the messages + for format in formats_to_test: + logging.debug(f"Checking result for {format}") + topic = topic_prefix + format + + result = instance.query_with_retry( + f"SELECT * FROM test.view_{topic}", + check_callback=lambda res: res.count("\n") == 3, + ) + + assert TSV(result) == TSV(expected) + + kafka_delete_topic(admin_client, topic) + + +if __name__ == "__main__": + cluster.start() + input("Cluster created, press any key to destroy...") + cluster.shutdown() From 613ebe367c1f811eea38d7c5e778cedddbfb0ce7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Sat, 10 Aug 2024 22:05:11 +0000 Subject: [PATCH 545/644] Only add extra cell when necessary --- tests/ci/report.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 15b1512896a..6779a6dae96 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -738,7 +738,7 @@ def create_test_html_report( if test_results: rows_part = [] num_fails = 0 - has_test_time = False + has_test_time = any(tr.time is not None for tr in test_results) has_log_urls = False # Display entires with logs at the top (they correspond to failed tests) @@ -770,12 +770,12 @@ def create_test_html_report( row.append(f'{test_result.status}') colspan += 1 - row.append("") - if test_result.time is not None: - has_test_time = True - row.append(str(test_result.time)) - row.append("") - colspan += 1 + if has_test_time: + if test_result.time is not None: + row.append(f"{test_result.time}") + else: + row.append("") + colspan += 1 if test_result.log_urls is not None: has_log_urls = True From e93584e741fab888977413c800df4595220e7552 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 10 Aug 2024 23:02:30 +0000 Subject: [PATCH 546/644] fix Field conversion to IPv4 --- src/Interpreters/convertFieldToType.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 3489852bbd5..738c51baa64 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -304,8 +304,8 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID } if (which_type.isIPv4() && src.getType() == Field::Types::UInt64) { - /// convert to UInt32 which is the underlying type for native IPv4 - return convertNumericType(src, type); + /// convert through UInt32 which is the underlying type for native IPv4 + return static_cast(convertNumericType(src, type).safeGet()); } } else if (which_type.isUUID() && src.getType() == Field::Types::UUID) From 1142305b113e261d0c8910c0b622ba94727fe78d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 10 Aug 2024 09:53:43 +0200 Subject: [PATCH 547/644] tests: fix 01246_buffer_flush flakiness due to slow trace_log flush Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/01246_buffer_flush.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01246_buffer_flush.sh b/tests/queries/0_stateless/01246_buffer_flush.sh index aea91a0bf6b..3c7b9038e1f 100755 --- a/tests/queries/0_stateless/01246_buffer_flush.sh +++ b/tests/queries/0_stateless/01246_buffer_flush.sh @@ -27,7 +27,7 @@ function wait_until() function get_buffer_delay() { local buffer_insert_id=$1 && shift - query "SYSTEM FLUSH LOGS" + $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" query " WITH (SELECT event_time_microseconds FROM system.query_log WHERE current_database = '$CLICKHOUSE_DATABASE' AND type = 'QueryStart' AND query_id = '$buffer_insert_id') AS begin_, From 45028620332d55391e900b6c7e75acb34df1d98c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 11 Aug 2024 08:35:47 +0000 Subject: [PATCH 548/644] Fix no-SSE3 build --- contrib/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index eb3afe0ccdf..b33e7083e32 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -179,7 +179,7 @@ else() message(STATUS "Not using QPL") endif () -if (OS_LINUX AND ARCH_AMD64) +if (OS_LINUX AND ARCH_AMD64 AND NOT NO_SSE3_OR_HIGHER) option (ENABLE_QATLIB "Enable Intel® QuickAssist Technology Library (QATlib)" ${ENABLE_LIBRARIES}) elseif(ENABLE_QATLIB) message (${RECONFIGURE_MESSAGE_LEVEL} "QATLib is only supported on x86_64") From 53bc1b7e3539cde14cb34f26af296bde5c29449e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 11 Aug 2024 13:19:36 +0200 Subject: [PATCH 549/644] Revert "Refactor tests for (experimental) statistics" --- docs/en/development/tests.md | 4 +- .../statements/alter/statistics.md | 16 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- ...2864_statistics_count_min_sketch.reference | 14 ++ .../02864_statistics_count_min_sketch.sql | 70 ++++++ .../02864_statistics_ddl.reference | 37 ++- .../0_stateless/02864_statistics_ddl.sql | 234 ++++-------------- ...delayed_materialization_in_merge.reference | 12 - ...stics_delayed_materialization_in_merge.sql | 36 --- .../02864_statistics_exception.reference | 0 .../02864_statistics_exception.sql | 55 ++++ ..._statistics_materialize_in_merge.reference | 10 + .../02864_statistics_materialize_in_merge.sql | 52 ++++ .../02864_statistics_predicates.reference | 98 -------- .../02864_statistics_predicates.sql | 214 ---------------- .../02864_statistics_uniq.reference | 35 +++ .../0_stateless/02864_statistics_uniq.sql | 73 ++++++ .../02864_statistics_usage.reference | 20 -- .../0_stateless/02864_statistics_usage.sql | 42 ---- 19 files changed, 399 insertions(+), 625 deletions(-) create mode 100644 tests/queries/0_stateless/02864_statistics_count_min_sketch.reference create mode 100644 tests/queries/0_stateless/02864_statistics_count_min_sketch.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql create mode 100644 tests/queries/0_stateless/02864_statistics_exception.reference create mode 100644 tests/queries/0_stateless/02864_statistics_exception.sql create mode 100644 tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference create mode 100644 tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_predicates.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_predicates.sql create mode 100644 tests/queries/0_stateless/02864_statistics_uniq.reference create mode 100644 tests/queries/0_stateless/02864_statistics_uniq.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_usage.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_usage.sql diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 6cb36e2049b..269995a1a96 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -14,7 +14,7 @@ Each functional test sends one or multiple queries to the running ClickHouse ser Tests are located in `queries` directory. There are two subdirectories: `stateless` and `stateful`. Stateless tests run queries without any preloaded test data - they often create small synthetic datasets on the fly, within the test itself. Stateful tests require preloaded test data from ClickHouse and it is available to general public. -Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`. +Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`. :::note A common mistake when testing data types `DateTime` and `DateTime64` is assuming that the server uses a specific time zone (e.g. "UTC"). This is not the case, time zones in CI test runs @@ -38,7 +38,7 @@ For more options, see `tests/clickhouse-test --help`. You can simply run all tes ### Adding a New Test -To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. +To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client --multiquery < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables. diff --git a/docs/en/sql-reference/statements/alter/statistics.md b/docs/en/sql-reference/statements/alter/statistics.md index 7a1774a01b5..6880cef0e5c 100644 --- a/docs/en/sql-reference/statements/alter/statistics.md +++ b/docs/en/sql-reference/statements/alter/statistics.md @@ -8,28 +8,26 @@ sidebar_label: STATISTICS The following operations are available: -- `ALTER TABLE [db].table ADD STATISTICS [IF NOT EXISTS] (column list) TYPE (type list)` - Adds statistic description to tables metadata. +- `ALTER TABLE [db].table ADD STATISTICS (columns list) TYPE (type list)` - Adds statistic description to tables metadata. -- `ALTER TABLE [db].table MODIFY STATISTICS (column list) TYPE (type list)` - Modifies statistic description to tables metadata. +- `ALTER TABLE [db].table MODIFY STATISTICS (columns list) TYPE (type list)` - Modifies statistic description to tables metadata. -- `ALTER TABLE [db].table DROP STATISTICS [IF EXISTS] (column list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns. +- `ALTER TABLE [db].table DROP STATISTICS (columns list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns. -- `ALTER TABLE [db].table CLEAR STATISTICS [IF EXISTS] (column list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`. +- `ALTER TABLE [db].table CLEAR STATISTICS (columns list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`. -- `ALTER TABLE [db.]table MATERIALIZE STATISTICS [IF EXISTS] (column list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +- `ALTER TABLE [db.]table MATERIALIZE STATISTICS (columns list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). The first two commands are lightweight in a sense that they only change metadata or remove files. Also, they are replicated, syncing statistics metadata via ZooKeeper. -## Example: - -Adding two statistics types to two columns: +There is an example adding two statistics types to two columns: ``` ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq; ``` :::note -Statistic are supported only for [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine tables (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). +Statistic manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). ::: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c7101021f02..625b1281c61 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3517,7 +3517,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context const auto & new_column = new_metadata.getColumns().get(command.column_name); if (!old_column.type->equals(*new_column.type)) throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, - "ALTER types of column {} with statistics is not safe " + "ALTER types of column {} with statistics is not not safe " "because it can change the representation of statistics", backQuoteIfNeed(command.column_name)); } diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference b/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference new file mode 100644 index 00000000000..02c41656a36 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference @@ -0,0 +1,14 @@ +CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +Test statistics count_min: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) +Test statistics multi-types: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) +Test LowCardinality and Nullable data type: +tab2 diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql b/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql new file mode 100644 index 00000000000..c730aa7b4a7 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql @@ -0,0 +1,70 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS tab SYNC; + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET allow_suspicious_low_cardinality_types=1; +SET mutations_sync = 2; + +CREATE TABLE tab +( + a String, + b UInt64, + c Int64, + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; + +SHOW CREATE TABLE tab; + +INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), generateUUIDv4() FROM system.numbers LIMIT 10000; + +SELECT 'Test statistics count_min:'; + +ALTER TABLE tab ADD STATISTICS a TYPE count_min; +ALTER TABLE tab ADD STATISTICS b TYPE count_min; +ALTER TABLE tab ADD STATISTICS c TYPE count_min; +ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; + +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +ALTER TABLE tab DROP STATISTICS a, b, c; + + +SELECT 'Test statistics multi-types:'; + +ALTER TABLE tab ADD STATISTICS a TYPE count_min; +ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; +ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; +ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; + +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +ALTER TABLE tab DROP STATISTICS a, b, c; + +DROP TABLE IF EXISTS tab SYNC; + + +SELECT 'Test LowCardinality and Nullable data type:'; +DROP TABLE IF EXISTS tab2 SYNC; +SET allow_suspicious_low_cardinality_types=1; +CREATE TABLE tab2 +( + a LowCardinality(Int64) STATISTICS(count_min), + b Nullable(Int64) STATISTICS(count_min), + c LowCardinality(Nullable(Int64)) STATISTICS(count_min), + pk String, +) Engine = MergeTree() ORDER BY pk; + +select name from system.tables where name = 'tab2' and database = currentDatabase(); + +DROP TABLE IF EXISTS tab2 SYNC; diff --git a/tests/queries/0_stateless/02864_statistics_ddl.reference b/tests/queries/0_stateless/02864_statistics_ddl.reference index 0e453b0ee8a..a7ff5caa0b0 100644 --- a/tests/queries/0_stateless/02864_statistics_ddl.reference +++ b/tests/queries/0_stateless/02864_statistics_ddl.reference @@ -1,6 +1,31 @@ -CREATE TABLE default.tab\n(\n `f64` Float64,\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32,\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.tab\n(\n `f64` Float64,\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32,\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +After insert + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) +10 +0 +After drop statistic + Prewhere info + Prewhere filter + Prewhere filter column: and(less(b, 10), less(a, 10)) (removed) +10 +CREATE TABLE default.tab\n(\n `a` Float64,\n `b` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +After add statistic +CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +After materialize statistic + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) +20 +After merge + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) +20 +CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +After rename + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), less(c, 10)) (removed) +20 diff --git a/tests/queries/0_stateless/02864_statistics_ddl.sql b/tests/queries/0_stateless/02864_statistics_ddl.sql index 32b56a842b7..fe612efe2ac 100644 --- a/tests/queries/0_stateless/02864_statistics_ddl.sql +++ b/tests/queries/0_stateless/02864_statistics_ddl.sql @@ -1,195 +1,59 @@ --- Tags: no-fasttest --- no-fasttest: 'count_min' sketches need a 3rd party library - --- Tests that DDL statements which create / drop / materialize statistics - -SET mutations_sync = 1; +-- Tests that various DDL statements create/drop/materialize statistics DROP TABLE IF EXISTS tab; --- Error case: Can't create statistics when allow_experimental_statistics = 0 -CREATE TABLE tab (col Float64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - SET allow_experimental_statistics = 1; - --- Error case: Unknown statistics types are rejected -CREATE TABLE tab (col Float64 STATISTICS(no_statistics_type)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - --- Error case: The same statistics type can't exist more than once on a column -CREATE TABLE tab (col Float64 STATISTICS(tdigest, tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - -SET allow_suspicious_low_cardinality_types = 1; - --- Statistics can only be created on columns of specific data types (depending on the statistics kind), (*) - --- tdigest requires data_type.isValueRepresentedByInteger --- These types work: -CREATE TABLE tab (col UInt8 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col UInt256 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Float32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Decimal32(3) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Date STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Date32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col DateTime STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col DateTime64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col IPv4 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Nullable(UInt8) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; --- These types don't work: -CREATE TABLE tab (col String STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col FixedString(1) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Array(Float64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col UUID STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col IPv6 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } - --- uniq requires data_type.isValueRepresentedByInteger --- These types work: -CREATE TABLE tab (col UInt8 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col UInt256 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Float32 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Decimal32(3) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Date STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Date32 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col DateTime STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col DateTime64 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col IPv4 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Nullable(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; --- These types don't work: -CREATE TABLE tab (col String STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col FixedString(1) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Array(Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col UUID STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col IPv6 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } - --- count_min requires data_type.isValueRepresentedByInteger or data_type = (Fixed)String --- These types work: -CREATE TABLE tab (col UInt8 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col UInt256 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Float32 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Decimal32(3) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Date STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Date32 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col DateTime STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col DateTime64 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col IPv4 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Nullable(UInt8) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col String STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col FixedString(1) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; --- These types don't work: -CREATE TABLE tab (col Array(Float64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col UUID STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col IPv6 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } - --- CREATE TABLE was easy, ALTER is more fun +SET allow_statistics_optimize = 1; CREATE TABLE tab ( - f64 Float64, - f64_tdigest Float64 STATISTICS(tdigest), - f32 Float32, - s String, - a Array(Float64) -) -Engine = MergeTree() -ORDER BY tuple(); + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; --- Error case: Unknown statistics types are rejected --- (relevant for ADD and MODIFY) -ALTER TABLE tab ADD STATISTICS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } -ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } -ALTER TABLE tab MODIFY STATISTICS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } --- for some reason, ALTER TABLE tab MODIFY STATISTICS IF EXISTS is not supported - --- Error case: The same statistics type can't exist more than once on a column --- (relevant for ADD and MODIFY) --- Create the same statistics object twice -ALTER TABLE tab ADD STATISTICS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } -ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } -ALTER TABLE tab MODIFY STATISTICS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } --- Create an statistics which exists already -ALTER TABLE tab ADD STATISTICS f64_tdigest TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64_tdigest TYPE tdigest; -- no-op -ALTER TABLE tab MODIFY STATISTICS f64_tdigest TYPE tdigest; -- no-op - --- Error case: Column does not exist --- (relevant for ADD, MODIFY, DROP, CLEAR, and MATERIALIZE) --- Note that the results are unfortunately quite inconsistent ... -ALTER TABLE tab ADD STATISTICS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab ADD STATISTICS IF NOT EXISTS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MODIFY STATISTICS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS IF EXISTS no_such_column; -- no-op -ALTER TABLE tab CLEAR STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab CLEAR STATISTICS IF EXISTS no_such_column; -- no-op -ALTER TABLE tab MATERIALIZE STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MATERIALIZE STATISTICS IF EXISTS no_such_column; -- { serverError ILLEGAL_STATISTICS } - --- Error case: Column exists but has no statistics --- (relevant for MODIFY, DROP, CLEAR, and MATERIALIZE) --- Note that the results are unfortunately quite inconsistent ... -ALTER TABLE tab MODIFY STATISTICS s TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS s; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS IF EXISTS s; -- no-op -ALTER TABLE tab CLEAR STATISTICS s; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab CLEAR STATISTICS IF EXISTS s; -- no-op -ALTER TABLE tab MATERIALIZE STATISTICS s; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MATERIALIZE STATISTICS IF EXISTS s; -- { serverError ILLEGAL_STATISTICS } - --- We don't check systematically that that statistics can only be created via ALTER ADD STATISTICS on columns of specific data types (the --- internal type validation code is tested already above, (*)). Only do a rudimentary check for each statistics type with a data type that --- works and one that doesn't work. --- tdigest --- Works: -ALTER TABLE tab ADD STATISTICS f64 TYPE tdigest; ALTER TABLE tab DROP STATISTICS f64; -ALTER TABLE tab MODIFY STATISTICS f64 TYPE tdigest; ALTER TABLE tab DROP STATISTICS f64; --- Doesn't work: -ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MODIFY STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } --- uniq --- Works: -ALTER TABLE tab ADD STATISTICS f64 TYPE uniq; ALTER TABLE tab DROP STATISTICS f64; -ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; --- Doesn't work: -ALTER TABLE tab ADD STATISTICS a TYPE uniq; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MODIFY STATISTICS a TYPE uniq; -- { serverError ILLEGAL_STATISTICS } --- count_min --- Works: -ALTER TABLE tab ADD STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; -ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; --- Doesn't work: -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MODIFY STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS } - --- Any data type changes on columns with statistics are disallowed, for simplicity even if the new data type is compatible with all existing --- statistics objects (e.g. tdigest can be created on Float64 and UInt64) -ALTER TABLE tab MODIFY COLUMN f64_tdigest UInt64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } - --- Finally, do a full-circle test of a good case. Print table definition after each step. --- Intentionally specifying _two_ columns and _two_ statistics types to have that also tested. -SHOW CREATE TABLE tab; -ALTER TABLE tab ADD STATISTICS f64, f32 TYPE tdigest, uniq; -SHOW CREATE TABLE tab; -ALTER TABLE tab MODIFY STATISTICS f64, f32 TYPE tdigest, uniq; -SHOW CREATE TABLE tab; -ALTER TABLE tab CLEAR STATISTICS f64, f32; -SHOW CREATE TABLE tab; -ALTER TABLE tab MATERIALIZE STATISTICS f64, f32; -SHOW CREATE TABLE tab; -ALTER TABLE tab DROP STATISTICS f64, f32; SHOW CREATE TABLE tab; -DROP TABLE tab; +INSERT INTO tab select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; + +SELECT 'After insert'; +SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT count(*) FROM tab WHERE b < 10 and a < 10; +SELECT count(*) FROM tab WHERE b < NULL and a < '10'; + +ALTER TABLE tab DROP STATISTICS a, b; + +SELECT 'After drop statistic'; +SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT count(*) FROM tab WHERE b < 10 and a < 10; + +SHOW CREATE TABLE tab; + +ALTER TABLE tab ADD STATISTICS a, b TYPE tdigest; + +SELECT 'After add statistic'; + +SHOW CREATE TABLE tab; + +ALTER TABLE tab MATERIALIZE STATISTICS a, b; +INSERT INTO tab select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; + +SELECT 'After materialize statistic'; +SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT count(*) FROM tab WHERE b < 10 and a < 10; + +OPTIMIZE TABLE tab FINAL; + +SELECT 'After merge'; +SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT count(*) FROM tab WHERE b < 10 and a < 10; + +ALTER TABLE tab RENAME COLUMN b TO c; +SHOW CREATE TABLE tab; + +SELECT 'After rename'; +SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT count(*) FROM tab WHERE c < 10 and a < 10; + +DROP TABLE IF EXISTS tab; diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference deleted file mode 100644 index eb5e685597c..00000000000 --- a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference +++ /dev/null @@ -1,12 +0,0 @@ -After insert - Prewhere info - Prewhere filter - Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) -After merge - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) -After truncate, insert, and materialize - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql deleted file mode 100644 index d469a4c2036..00000000000 --- a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql +++ /dev/null @@ -1,36 +0,0 @@ --- Tests delayed materialization of statistics in merge instead of during insert (setting 'materialize_statistics_on_insert = 0'). --- (The concrete statistics type, column data type and predicate type don't matter) - --- Checks by the predicate evaluation order in EXPLAIN. This is quite fragile, a better approach would be helpful (maybe 'send_logs_level'?) - -DROP TABLE IF EXISTS tab; - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; -SET enable_analyzer = 1; - -SET materialize_statistics_on_insert = 0; - -CREATE TABLE tab -( - a Int64 STATISTICS(tdigest), - b Int16 STATISTICS(tdigest), -) ENGINE = MergeTree() ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics. - -INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; -SELECT 'After insert'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks b first, then a (statistics not used) - -OPTIMIZE TABLE tab FINAL; -SELECT 'After merge'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) - -TRUNCATE TABLE tab; -SET mutations_sync = 2; -INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; -ALTER TABLE tab MATERIALIZE STATISTICS a, b; -SELECT 'After truncate, insert, and materialize'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) - -DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_exception.reference b/tests/queries/0_stateless/02864_statistics_exception.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02864_statistics_exception.sql b/tests/queries/0_stateless/02864_statistics_exception.sql new file mode 100644 index 00000000000..289ffee6600 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_exception.sql @@ -0,0 +1,55 @@ +-- Tests creating/dropping/materializing statistics produces the right exceptions. + +DROP TABLE IF EXISTS tab; + +-- Can't create statistics when allow_experimental_statistics = 0 +CREATE TABLE tab +( + a Float64 STATISTICS(tdigest) +) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + +SET allow_experimental_statistics = 1; + +-- The same type of statistics can't exist more than once on a column +CREATE TABLE tab +( + a Float64 STATISTICS(tdigest, tdigest) +) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + +-- Unknown statistics types are rejected +CREATE TABLE tab +( + a Float64 STATISTICS(no_statistics_type) +) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + +-- tDigest statistics can only be created on numeric columns +CREATE TABLE tab +( + a String STATISTICS(tdigest), +) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } + +CREATE TABLE tab +( + a Float64, + b String +) Engine = MergeTree() ORDER BY tuple(); + +ALTER TABLE tab ADD STATISTICS a TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab ADD STATISTICS a TYPE tdigest; +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS a TYPE tdigest; +ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS a TYPE tdigest; +-- Statistics can be created only on integer columns +ALTER TABLE tab ADD STATISTICS b TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS b; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS a; +ALTER TABLE tab DROP STATISTICS IF EXISTS a; +ALTER TABLE tab CLEAR STATISTICS a; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab CLEAR STATISTICS IF EXISTS a; +ALTER TABLE tab MATERIALIZE STATISTICS b; -- { serverError ILLEGAL_STATISTICS } + +ALTER TABLE tab ADD STATISTICS a TYPE tdigest; +ALTER TABLE tab MODIFY COLUMN a Float64 TTL toDateTime(b) + INTERVAL 1 MONTH; +ALTER TABLE tab MODIFY COLUMN a Int64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference new file mode 100644 index 00000000000..5e969cf41cb --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference @@ -0,0 +1,10 @@ +10 +10 +10 +statistics not used Condition less(b, 10_UInt8) moved to PREWHERE +statistics not used Condition less(a, 10_UInt8) moved to PREWHERE +statistics used after merge Condition less(a, 10_UInt8) moved to PREWHERE +statistics used after merge Condition less(b, 10_UInt8) moved to PREWHERE +statistics used after materialize Condition less(a, 10_UInt8) moved to PREWHERE +statistics used after materialize Condition less(b, 10_UInt8) moved to PREWHERE +2 0 diff --git a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql new file mode 100644 index 00000000000..6606cff263f --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql @@ -0,0 +1,52 @@ +-- Tests delayed materialization of statistics in merge instead of during insert (setting 'materialize_statistics_on_insert = 0'). + +DROP TABLE IF EXISTS tab; + +SET enable_analyzer = 1; +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; + +SET materialize_statistics_on_insert = 0; + +CREATE TABLE tab +( + a Int64 STATISTICS(tdigest), + b Int16 STATISTICS(tdigest), +) ENGINE = MergeTree() ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics. + +INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; + +SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics not used'; + +OPTIMIZE TABLE tab FINAL; + +SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after merge'; + +TRUNCATE TABLE tab; +SET mutations_sync = 2; + +INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; +ALTER TABLE tab MATERIALIZE STATISTICS a, b; + +SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after materialize'; + +DROP TABLE tab; + +SYSTEM FLUSH LOGS; + +SELECT log_comment, message FROM system.text_log JOIN +( + SELECT Settings['log_comment'] AS log_comment, query_id FROM system.query_log + WHERE current_database = currentDatabase() + AND query LIKE 'SELECT count(*) FROM tab%' + AND type = 'QueryFinish' +) AS query_log USING (query_id) +WHERE message LIKE '%moved to PREWHERE%' +ORDER BY event_time_microseconds; + +SELECT count(), sum(ProfileEvents['MergeTreeDataWriterStatisticsCalculationMicroseconds']) +FROM system.query_log +WHERE current_database = currentDatabase() + AND query LIKE 'INSERT INTO tab SELECT%' + AND type = 'QueryFinish'; diff --git a/tests/queries/0_stateless/02864_statistics_predicates.reference b/tests/queries/0_stateless/02864_statistics_predicates.reference deleted file mode 100644 index ffbd7269e05..00000000000 --- a/tests/queries/0_stateless/02864_statistics_predicates.reference +++ /dev/null @@ -1,98 +0,0 @@ -u64 and = -10 -10 -10 -10 -0 -0 -0 -0 -10 -10 -10 -10 -u64 and < -70 -70 -70 -70 -80 -80 -80 -80 -70 -70 -70 -70 -f64 and = -10 -10 -10 -10 -0 -0 -0 -0 -10 -10 -10 -10 -0 -0 -0 -0 -f64 and < -70 -70 -70 -70 -80 -80 -80 -80 -70 -70 -70 -70 -80 -80 -80 -80 -dt and = -0 -0 -0 -0 -10 -10 -10 -10 -dt and < -10000 -10000 -10000 -10000 -70 -70 -70 -70 -b and = -5000 -5000 -5000 -5000 -5000 -5000 -5000 -5000 -5000 -5000 -5000 -5000 -0 -0 -0 -0 -s and = -10 -10 diff --git a/tests/queries/0_stateless/02864_statistics_predicates.sql b/tests/queries/0_stateless/02864_statistics_predicates.sql deleted file mode 100644 index 779116cf19a..00000000000 --- a/tests/queries/0_stateless/02864_statistics_predicates.sql +++ /dev/null @@ -1,214 +0,0 @@ --- Tags: no-fasttest --- no-fasttest: 'count_min' sketches need a 3rd party library - --- Tests the cross product of all predicates with all right-hand sides on all data types and all statistics types. - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; - -DROP TABLE IF EXISTS tab; - -CREATE TABLE tab -( - u64 UInt64, - u64_tdigest UInt64 STATISTICS(tdigest), - u64_count_min UInt64 STATISTICS(count_min), - u64_uniq UInt64 STATISTICS(uniq), - f64 Float64, - f64_tdigest Float64 STATISTICS(tdigest), - f64_count_min Float64 STATISTICS(count_min), - f64_uniq Float64 STATISTICS(uniq), - dt DateTime, - dt_tdigest DateTime STATISTICS(tdigest), - dt_count_min DateTime STATISTICS(count_min), - dt_uniq DateTime STATISTICS(uniq), - b Bool, - b_tdigest Bool STATISTICS(tdigest), - b_count_min Bool STATISTICS(count_min), - b_uniq Bool STATISTICS(uniq), - s String, - -- s_tdigest String STATISTICS(tdigest), -- not supported by tdigest - s_count_min String STATISTICS(count_min) - -- s_uniq String STATISTICS(uniq), -- not supported by uniq -) Engine = MergeTree() ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0; - -INSERT INTO tab --- SELECT number % 10000, number % 1000, -(number % 100) FROM system.numbers LIMIT 10000; -SELECT number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 2, - number % 2, - number % 2, - number % 2, - toString(number % 1000), - toString(number % 1000) -FROM system.numbers LIMIT 10000; - --- u64 ---------------------------------------------------- - -SELECT 'u64 and ='; - -SELECT count(*) FROM tab WHERE u64 = 7; -SELECT count(*) FROM tab WHERE u64_tdigest = 7; -SELECT count(*) FROM tab WHERE u64_count_min = 7; -SELECT count(*) FROM tab WHERE u64_uniq = 7; - -SELECT count(*) FROM tab WHERE u64 = 7.7; -SELECT count(*) FROM tab WHERE u64_tdigest = 7.7; -SELECT count(*) FROM tab WHERE u64_count_min = 7.7; -SELECT count(*) FROM tab WHERE u64_uniq = 7.7; - -SELECT count(*) FROM tab WHERE u64 = '7'; -SELECT count(*) FROM tab WHERE u64_tdigest = '7'; -SELECT count(*) FROM tab WHERE u64_count_min = '7'; -SELECT count(*) FROM tab WHERE u64_uniq = '7'; - -SELECT count(*) FROM tab WHERE u64 = '7.7'; -- { serverError TYPE_MISMATCH } -SELECT count(*) FROM tab WHERE u64_tdigest = '7.7'; -- { serverError TYPE_MISMATCH } -SELECT count(*) FROM tab WHERE u64_count_min = '7.7'; -- { serverError TYPE_MISMATCH } -SELECT count(*) FROM tab WHERE u64_uniq = '7.7'; -- { serverError TYPE_MISMATCH } - -SELECT 'u64 and <'; - -SELECT count(*) FROM tab WHERE u64 < 7; -SELECT count(*) FROM tab WHERE u64_tdigest < 7; -SELECT count(*) FROM tab WHERE u64_count_min < 7; -SELECT count(*) FROM tab WHERE u64_uniq < 7; - -SELECT count(*) FROM tab WHERE u64 < 7.7; -SELECT count(*) FROM tab WHERE u64_tdigest < 7.7; -SELECT count(*) FROM tab WHERE u64_count_min < 7.7; -SELECT count(*) FROM tab WHERE u64_uniq < 7.7; - -SELECT count(*) FROM tab WHERE u64 < '7'; -SELECT count(*) FROM tab WHERE u64_tdigest < '7'; -SELECT count(*) FROM tab WHERE u64_count_min < '7'; -SELECT count(*) FROM tab WHERE u64_uniq < '7'; - -SELECT count(*) FROM tab WHERE u64 < '7.7'; -- { serverError TYPE_MISMATCH } -SELECT count(*) FROM tab WHERE u64_tdigest < '7.7'; -- { serverError TYPE_MISMATCH } -SELECT count(*) FROM tab WHERE u64_count_min < '7.7'; -- { serverError TYPE_MISMATCH } -SELECT count(*) FROM tab WHERE u64_uniq < '7.7'; -- { serverError TYPE_MISMATCH } - --- f64 ---------------------------------------------------- - -SELECT 'f64 and ='; - -SELECT count(*) FROM tab WHERE f64 = 7; -SELECT count(*) FROM tab WHERE f64_tdigest = 7; -SELECT count(*) FROM tab WHERE f64_count_min = 7; -SELECT count(*) FROM tab WHERE f64_uniq = 7; - -SELECT count(*) FROM tab WHERE f64 = 7.7; -SELECT count(*) FROM tab WHERE f64_tdigest = 7.7; -SELECT count(*) FROM tab WHERE f64_count_min = 7.7; -SELECT count(*) FROM tab WHERE f64_uniq = 7.7; - -SELECT count(*) FROM tab WHERE f64 = '7'; -SELECT count(*) FROM tab WHERE f64_tdigest = '7'; -SELECT count(*) FROM tab WHERE f64_count_min = '7'; -SELECT count(*) FROM tab WHERE f64_uniq = '7'; - -SELECT count(*) FROM tab WHERE f64 = '7.7'; -SELECT count(*) FROM tab WHERE f64_tdigest = '7.7'; -SELECT count(*) FROM tab WHERE f64_count_min = '7.7'; -SELECT count(*) FROM tab WHERE f64_uniq = '7.7'; - -SELECT 'f64 and <'; - -SELECT count(*) FROM tab WHERE f64 < 7; -SELECT count(*) FROM tab WHERE f64_tdigest < 7; -SELECT count(*) FROM tab WHERE f64_count_min < 7; -SELECT count(*) FROM tab WHERE f64_uniq < 7; - -SELECT count(*) FROM tab WHERE f64 < 7.7; -SELECT count(*) FROM tab WHERE f64_tdigest < 7.7; -SELECT count(*) FROM tab WHERE f64_count_min < 7.7; -SELECT count(*) FROM tab WHERE f64_uniq < 7.7; - -SELECT count(*) FROM tab WHERE f64 < '7'; -SELECT count(*) FROM tab WHERE f64_tdigest < '7'; -SELECT count(*) FROM tab WHERE f64_count_min < '7'; -SELECT count(*) FROM tab WHERE f64_uniq < '7'; - -SELECT count(*) FROM tab WHERE f64 < '7.7'; -SELECT count(*) FROM tab WHERE f64_tdigest < '7.7'; -SELECT count(*) FROM tab WHERE f64_count_min < '7.7'; -SELECT count(*) FROM tab WHERE f64_uniq < '7.7'; - --- dt ---------------------------------------------------- - -SELECT 'dt and ='; - -SELECT count(*) FROM tab WHERE dt = '2024-08-08 11:12:13'; -SELECT count(*) FROM tab WHERE dt_tdigest = '2024-08-08 11:12:13'; -SELECT count(*) FROM tab WHERE dt_count_min = '2024-08-08 11:12:13'; -SELECT count(*) FROM tab WHERE dt_uniq = '2024-08-08 11:12:13'; - -SELECT count(*) FROM tab WHERE dt = 7; -SELECT count(*) FROM tab WHERE dt_tdigest = 7; -SELECT count(*) FROM tab WHERE dt_count_min = 7; -SELECT count(*) FROM tab WHERE dt_uniq = 7; - -SELECT 'dt and <'; - -SELECT count(*) FROM tab WHERE dt < '2024-08-08 11:12:13'; -SELECT count(*) FROM tab WHERE dt_tdigest < '2024-08-08 11:12:13'; -SELECT count(*) FROM tab WHERE dt_count_min < '2024-08-08 11:12:13'; -SELECT count(*) FROM tab WHERE dt_uniq < '2024-08-08 11:12:13'; - -SELECT count(*) FROM tab WHERE dt < 7; -SELECT count(*) FROM tab WHERE dt_tdigest < 7; -SELECT count(*) FROM tab WHERE dt_count_min < 7; -SELECT count(*) FROM tab WHERE dt_uniq < 7; - --- b ---------------------------------------------------- - -SELECT 'b and ='; - -SELECT count(*) FROM tab WHERE b = true; -SELECT count(*) FROM tab WHERE b_tdigest = true; -SELECT count(*) FROM tab WHERE b_count_min = true; -SELECT count(*) FROM tab WHERE b_uniq = true; - -SELECT count(*) FROM tab WHERE b = 'true'; -SELECT count(*) FROM tab WHERE b_tdigest = 'true'; -SELECT count(*) FROM tab WHERE b_count_min = 'true'; -SELECT count(*) FROM tab WHERE b_uniq = 'true'; - -SELECT count(*) FROM tab WHERE b = 1; -SELECT count(*) FROM tab WHERE b_tdigest = 1; -SELECT count(*) FROM tab WHERE b_count_min = 1; -SELECT count(*) FROM tab WHERE b_uniq = 1; - -SELECT count(*) FROM tab WHERE b = 1.1; -SELECT count(*) FROM tab WHERE b_tdigest = 1.1; -SELECT count(*) FROM tab WHERE b_count_min = 1.1; -SELECT count(*) FROM tab WHERE b_uniq = 1.1; - --- s ---------------------------------------------------- - -SELECT 's and ='; - -SELECT count(*) FROM tab WHERE s = 7; -- { serverError NO_COMMON_TYPE } --- SELECT count(*) FROM tab WHERE s_tdigest = 7; -- not supported -SELECT count(*) FROM tab WHERE s_count_min = 7; -- { serverError NO_COMMON_TYPE } --- SELECT count(*) FROM tab WHERE s_uniq = 7; -- not supported - -SELECT count(*) FROM tab WHERE s = '7'; --- SELECT count(*) FROM tab WHERE s_tdigest = '7'; -- not supported -SELECT count(*) FROM tab WHERE s_count_min = '7'; --- SELECT count(*) FROM tab WHERE s_uniq = '7'; -- not supported - -DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_uniq.reference b/tests/queries/0_stateless/02864_statistics_uniq.reference new file mode 100644 index 00000000000..77786dbdd8c --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_uniq.reference @@ -0,0 +1,35 @@ +CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest, uniq),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +After insert + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) +After merge + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) +After modify TDigest + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(c, -1), less(a, 10), less(b, 10)) (removed) +After drop + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), less(c, -1), less(b, 10)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_uniq.sql b/tests/queries/0_stateless/02864_statistics_uniq.sql new file mode 100644 index 00000000000..0f5f353c045 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_uniq.sql @@ -0,0 +1,73 @@ +DROP TABLE IF EXISTS t1; + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET mutations_sync = 1; + +CREATE TABLE t1 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + c Int64 STATISTICS(tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; + +SHOW CREATE TABLE t1; + +INSERT INTO t1 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; +INSERT INTO t1 select 0, 0, 11, generateUUIDv4(); + +SELECT 'After insert'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +OPTIMIZE TABLE t1 FINAL; + +SELECT 'After merge'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +SELECT 'After modify TDigest'; +ALTER TABLE t1 MODIFY STATISTICS c TYPE TDigest; +ALTER TABLE t1 MATERIALIZE STATISTICS c; + +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + + +ALTER TABLE t1 DROP STATISTICS c; + +SELECT 'After drop'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +SET allow_suspicious_low_cardinality_types=1; +CREATE TABLE t2 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + c LowCardinality(Int64) STATISTICS(tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO t2 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; + +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; + +CREATE TABLE t3 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + c Nullable(Int64) STATISTICS(tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO t3 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; + +DROP TABLE IF EXISTS t3; + diff --git a/tests/queries/0_stateless/02864_statistics_usage.reference b/tests/queries/0_stateless/02864_statistics_usage.reference deleted file mode 100644 index a9f669b88c1..00000000000 --- a/tests/queries/0_stateless/02864_statistics_usage.reference +++ /dev/null @@ -1,20 +0,0 @@ -After insert - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) -After drop statistic - Prewhere info - Prewhere filter - Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) -After add and materialize statistic - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) -After merge - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) -After rename - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(c, 10_UInt8)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_usage.sql b/tests/queries/0_stateless/02864_statistics_usage.sql deleted file mode 100644 index 4956bd27e87..00000000000 --- a/tests/queries/0_stateless/02864_statistics_usage.sql +++ /dev/null @@ -1,42 +0,0 @@ --- Test that the optimizer picks up column statistics --- (The concrete statistics type, column data type and predicate type don't matter) - --- Checks by the predicate evaluation order in EXPLAIN. This is quite fragile, a better approach would be helpful (maybe 'send_logs_level'?) - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; -SET mutations_sync = 1; -SET enable_analyzer = 1; - -DROP TABLE IF EXISTS tab; - -CREATE TABLE tab -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest) -) Engine = MergeTree() ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0; - -INSERT INTO tab select number, -number FROM system.numbers LIMIT 10000; -SELECT 'After insert'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) - -ALTER TABLE tab DROP STATISTICS a, b; -SELECT 'After drop statistic'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks b first, then a (statistics not used) - -ALTER TABLE tab ADD STATISTICS a, b TYPE tdigest; -ALTER TABLE tab MATERIALIZE STATISTICS a, b; -INSERT INTO tab select number, -number FROM system.numbers LIMIT 10000; -SELECT 'After add and materialize statistic'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) - -OPTIMIZE TABLE tab FINAL; -SELECT 'After merge'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) - -ALTER TABLE tab RENAME COLUMN b TO c; -SELECT 'After rename'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then c (statistics used) - -DROP TABLE IF EXISTS tab; From 29afd2de785450f2e7f5faec1dc6b35e166cefb4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 11 Aug 2024 13:26:45 +0200 Subject: [PATCH 550/644] Remove "Processing configuration file" message from clickhouse-local Make the behaviour identical to the clickhouse-client Signed-off-by: Azat Khuzhin --- programs/local/LocalServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 6b0b8fc5b50..200beea7b63 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -143,7 +143,7 @@ void LocalServer::initialize(Poco::Util::Application & self) if (fs::exists(config_path)) { - ConfigProcessor config_processor(config_path, false, true); + ConfigProcessor config_processor(config_path); ConfigProcessor::setConfigPath(fs::path(config_path).parent_path()); auto loaded_config = config_processor.loadConfig(); getClientConfiguration().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false); From 4fec61da55c1032f274da87198af59c78cd0d87e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 11 Aug 2024 12:35:27 +0000 Subject: [PATCH 551/644] fix wrong datatype in system.kafka_consumers --- src/Storages/System/StorageSystemKafkaConsumers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 86713632339..5e790587716 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -79,7 +79,7 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context auto & num_rebalance_revocations = assert_cast(*res_columns[index++]); auto & num_rebalance_assigments = assert_cast(*res_columns[index++]); auto & is_currently_used = assert_cast(*res_columns[index++]); - auto & last_used = assert_cast(*res_columns[index++]); + auto & last_used = assert_cast(*res_columns[index++]); auto & rdkafka_stat = assert_cast(*res_columns[index++]); const auto access = context->getAccess(); From 4ef3fe416debecefcea4d7336aac7c679092cf0c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 11 Aug 2024 13:08:53 +0000 Subject: [PATCH 552/644] Fix and simplify test --- .../02496_remove_redundant_sorting.reference | 68 ++++++++---------- .../02496_remove_redundant_sorting.sh | 43 ++++++----- ...emove_redundant_sorting_analyzer.reference | 71 ++++++++----------- 3 files changed, 82 insertions(+), 100 deletions(-) diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference index a0a1fd60812..4d004f2f78f 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference @@ -465,6 +465,37 @@ Expression ((Projection + Before ORDER BY)) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 +-- presence of an inner OFFSET retains the ORDER BY +-- query +WITH + t1 AS ( + SELECT a, b + FROM + VALUES ( + 'b UInt32, a Int32', + (1, 1), + (2, 0) + ) + ) +SELECT + SUM(a) +FROM ( + SELECT a, b + FROM t1 + ORDER BY 1 DESC, 2 + OFFSET 1 +) t2 +-- explain +Expression ((Projection + Before ORDER BY)) + Aggregating + Expression (Before GROUP BY) + Offset + Expression (Projection) + Sorting (Sorting for ORDER BY) + Expression ((Before ORDER BY + (Projection + Before ORDER BY))) + ReadFromStorage (Values) +-- execute +0 -- disable common optimization to avoid functions to be lifted up (liftUpFunctions optimization), needed for testing with stateful function -- neighbor() as stateful function prevents removing inner ORDER BY since its result depends on order -- query @@ -527,40 +558,3 @@ Expression (Projection) 2 4 1 3 0 2 --- presence of an inner OFFSET retains the ORDER BY --- query -WITH - t1 AS ( - SELECT SUM(a) AS a, b - FROM - VALUES ( - 'b UInt32, a Int32', - (1, 1), - (2, 0) - ) - GROUP BY 2 - ) -SELECT - SUM(a) -FROM ( - SELECT a, b - FROM t1 - ORDER BY 1 DESC, 2 - OFFSET 1 -) t2 --- explain -Expression (Projection) - Expression (Before ORDER BY) - Aggregating - Expression (Before GROUP BY) - Offset - Expression (Projection) - Sorting (Sorting for ORDER BY) - Expression (Before ORDER BY) - Expression (Projection) - Expression (Before ORDER BY) - Aggregating - Expression (Before GROUP BY) - ReadFromStorage (Values) --- execute -0 diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh index d59b4387101..c9bd242e429 100755 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh @@ -302,6 +302,27 @@ FROM )" run_query "$query" +echo "-- presence of an inner OFFSET retains the ORDER BY" +query="WITH + t1 AS ( + SELECT a, b + FROM + VALUES ( + 'b UInt32, a Int32', + (1, 1), + (2, 0) + ) + ) +SELECT + SUM(a) +FROM ( + SELECT a, b + FROM t1 + ORDER BY 1 DESC, 2 + OFFSET 1 +) t2" +run_query "$query" + echo "-- disable common optimization to avoid functions to be lifted up (liftUpFunctions optimization), needed for testing with stateful function" ENABLE_OPTIMIZATION="SET query_plan_enable_optimizations=0;$ENABLE_OPTIMIZATION" echo "-- neighbor() as stateful function prevents removing inner ORDER BY since its result depends on order" @@ -329,25 +350,3 @@ FROM ORDER BY number DESC )" run_query "$query" - -echo "-- presence of an inner OFFSET retains the ORDER BY" -query="WITH - t1 AS ( - SELECT SUM(a) AS a, b - FROM - VALUES ( - 'b UInt32, a Int32', - (1, 1), - (2, 0) - ) - GROUP BY 2 - ) -SELECT - SUM(a) -FROM ( - SELECT a, b - FROM t1 - ORDER BY 1 DESC, 2 - OFFSET 1 -) t2" -run_query "$query" diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference index 58441de5f22..dd5ac7bf706 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference @@ -464,6 +464,36 @@ Expression ((Project names + Projection)) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 +-- presence of an inner OFFSET retains the ORDER BY +-- query +WITH + t1 AS ( + SELECT a, b + FROM + VALUES ( + 'b UInt32, a Int32', + (1, 1), + (2, 0) + ) + ) +SELECT + SUM(a) +FROM ( + SELECT a, b + FROM t1 + ORDER BY 1 DESC, 2 + OFFSET 1 +) t2 +-- explain +Expression ((Project names + Projection)) + Aggregating + Expression ((Before GROUP BY + (Change column names to column identifiers + Project names))) + Offset + Sorting (Sorting for ORDER BY) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))) + ReadFromStorage (Values) +-- execute +0 -- disable common optimization to avoid functions to be lifted up (liftUpFunctions optimization), needed for testing with stateful function -- neighbor() as stateful function prevents removing inner ORDER BY since its result depends on order -- query @@ -533,44 +563,3 @@ Expression (Project names) 2 4 1 3 0 2 --- presence of an inner OFFSET retains the ORDER BY --- query -WITH - t1 AS ( - SELECT SUM(a) AS a, b - FROM - VALUES ( - 'b UInt32, a Int32', - (1, 1), - (2, 0) - ) - GROUP BY 2 - ) -SELECT - SUM(a) -FROM ( - SELECT a, b - FROM t1 - ORDER BY 1 DESC, 2 - OFFSET 1 -) t2 --- explain -Expression (Project names) - Expression (Projection) - Aggregating - Expression (Before GROUP BY) - Expression (Change column names to column identifiers) - Expression (Project names) - Offset - Sorting (Sorting for ORDER BY) - Expression (Before ORDER BY) - Expression (Projection) - Expression (Change column names to column identifiers) - Expression (Project names) - Expression (Projection) - Aggregating - Expression (Before GROUP BY) - Expression (Change column names to column identifiers) - ReadFromStorage (Values) --- execute -0 From 8e706265e6df3653de76224bfc050b4f52e49282 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 11 Aug 2024 16:29:35 +0000 Subject: [PATCH 553/644] fix --- src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp index 6a0522b0676..3ca7e8183f1 100644 --- a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp +++ b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp @@ -125,7 +125,7 @@ void OptimizeShardingKeyRewriteInMatcher::visit(ASTFunction & function, Data & d { Tuple new_tuple; - for (const auto & child : tuple) + for (auto & child : tuple) if (shardContains(child, name, data)) new_tuple.emplace_back(std::move(child)); From d314e5aa45fb8ac91324721ab278185b09437a40 Mon Sep 17 00:00:00 2001 From: Vladimir Varankin Date: Sun, 11 Aug 2024 18:37:29 +0200 Subject: [PATCH 554/644] typos in prometheus.md --- docs/en/interfaces/prometheus.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/interfaces/prometheus.md b/docs/en/interfaces/prometheus.md index bf541901b34..8e7023cc51f 100644 --- a/docs/en/interfaces/prometheus.md +++ b/docs/en/interfaces/prometheus.md @@ -75,7 +75,7 @@ Data are received by this protocol and written to a [TimeSeries](/en/engines/tab /write - remote_writeremote_write db_name time_series_table
@@ -105,7 +105,7 @@ Data are read from a [TimeSeries](/en/engines/table-engines/special/time_series) /read - remote_readremote_read
db_name time_series_table
@@ -144,14 +144,14 @@ Multiple protocols can be specified together in one place: /write - remote_writeremote_write
db_name.time_series_table
/read - remote_readremote_read
db_name.time_series_table
From e384e2c38e405b1b4758adaa44cd321e6d7f41b3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 11 Aug 2024 18:34:33 +0200 Subject: [PATCH 555/644] tests: fix 02122_join_group_by_timeout flakiness CI found [1] failure of the test: 2024-08-11 21:06:07 /usr/share/clickhouse-test/queries/0_stateless/02122_join_group_by_timeout.sh: line 51: 52614 Killed timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT -q "SELECT a.name as n And the problem is not the server, but the client, since query executed for ~1 second: 2024.08.11 21:06:02.284318 [ 49232 ] {ba989ee2-f615-49ca-bcd8-31b3916aeb2c} executeQuery: (from [::1]:54144) (comment: 02122_join_group_by_timeout.sh) SELECT a.name as n FROM ( SELECT 'Name' as name, number FROM system.numbers LIMIT 2000000 ) AS a, ( SELECT 'Name' as name2, number FROM system.numbers LIMIT 2000000 ) as b FORMAT Null SETTINGS max_execution_time = 1, timeout_overflow_mode = 'break' (stage: Complete) 2024.08.11 21:06:03.331249 [ 49232 ] {ba989ee2-f615-49ca-bcd8-31b3916aeb2c} executeQuery: Read 517104 rows, 3.95 MiB in 1.072023 sec., 482362.78512681165 rows/sec., 3.68 MiB/sec. [1]: https://s3.amazonaws.com/clickhouse-test-reports/67134/18da3f0ab63da1eef9396627d0dfd56cf5356f65/stateless_tests__msan__[1_4].html So instead of using timeout, let's use time from the system.query_log instead. Signed-off-by: Azat Khuzhin --- .../02122_join_group_by_timeout.reference | 6 +- .../02122_join_group_by_timeout.sh | 70 ++++++++++--------- 2 files changed, 41 insertions(+), 35 deletions(-) diff --git a/tests/queries/0_stateless/02122_join_group_by_timeout.reference b/tests/queries/0_stateless/02122_join_group_by_timeout.reference index f314e22e519..6500560e8fc 100644 --- a/tests/queries/0_stateless/02122_join_group_by_timeout.reference +++ b/tests/queries/0_stateless/02122_join_group_by_timeout.reference @@ -1,4 +1,6 @@ -Code: 159 -0 +Code: 159 +query_duration 1 +0 +query_duration 1 Code: 159 0 diff --git a/tests/queries/0_stateless/02122_join_group_by_timeout.sh b/tests/queries/0_stateless/02122_join_group_by_timeout.sh index 8380c5dbd0c..b4644878544 100755 --- a/tests/queries/0_stateless/02122_join_group_by_timeout.sh +++ b/tests/queries/0_stateless/02122_join_group_by_timeout.sh @@ -1,27 +1,23 @@ #!/usr/bin/env bash -# Tags: no-debug - -# no-debug: Query is canceled by timeout after max_execution_time, -# but sending an exception to the client may hang -# for more than MAX_PROCESS_WAIT seconds in a slow debug build, -# and test will fail. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -MAX_PROCESS_WAIT=5 - -IS_SANITIZER=$($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.warnings WHERE message like '%built with sanitizer%'") -if [ "$IS_SANITIZER" -gt 0 ]; then - # Query may hang for more than 5 seconds, especially in tsan build - MAX_PROCESS_WAIT=15 +TIMEOUT=5 +IS_SANITIZER_OR_DEBUG=$($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.warnings WHERE message like '%built with sanitizer%' or message like '%built in debug mode%'") +if [ "$IS_SANITIZER_OR_DEBUG" -gt 0 ]; then + # Increase the timeout due to in debug/sanitizers build: + # - client is slow + # - stacktrace resolving is slow + TIMEOUT=15 fi # TCP CLIENT: As of today (02/12/21) uses PullingAsyncPipelineExecutor ### Should be cancelled after 1 second and return a 159 exception (timeout) -timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT --max_execution_time 1 -q \ - "SELECT * FROM +query_id=$(random_str 12) +$CLICKHOUSE_CLIENT --query_id "$query_id" --max_execution_time 1 -q " + SELECT * FROM ( SELECT a.name as n FROM @@ -34,28 +30,35 @@ timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT --max_execution_time 1 -q \ GROUP BY n ) LIMIT 20 - FORMAT Null" 2>&1 | grep -o "Code: 159" | sort | uniq + FORMAT Null +" 2>&1 | grep -m1 -o "Code: 159" +$CLICKHOUSE_CLIENT -q "system flush logs" +${CLICKHOUSE_CURL} -q -sS "$CLICKHOUSE_URL" -d "select 'query_duration', round(query_duration_ms/1000) from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and query_id = '$query_id' and type != 'QueryStart'" + ### Should stop pulling data and return what has been generated already (return code 0) -timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT -q \ - "SELECT a.name as n - FROM - ( - SELECT 'Name' as name, number FROM system.numbers LIMIT 2000000 - ) AS a, - ( - SELECT 'Name' as name2, number FROM system.numbers LIMIT 2000000 - ) as b - FORMAT Null - SETTINGS max_execution_time = 1, timeout_overflow_mode = 'break' - " +query_id=$(random_str 12) +$CLICKHOUSE_CLIENT --query_id "$query_id" -q " + SELECT a.name as n + FROM + ( + SELECT 'Name' as name, number FROM system.numbers LIMIT 2000000 + ) AS a, + ( + SELECT 'Name' as name2, number FROM system.numbers LIMIT 2000000 + ) as b + FORMAT Null + SETTINGS max_execution_time = 1, timeout_overflow_mode = 'break' +" echo $? +$CLICKHOUSE_CLIENT -q "system flush logs" +${CLICKHOUSE_CURL} -q -sS "$CLICKHOUSE_URL" -d "select 'query_duration', round(query_duration_ms/1000) from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and query_id = '$query_id' and type != 'QueryStart'" # HTTP CLIENT: As of today (02/12/21) uses PullingPipelineExecutor ### Should be cancelled after 1 second and return a 159 exception (timeout) -${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL&max_execution_time=1" -d \ - "SELECT * FROM +${CLICKHOUSE_CURL} -q --max-time $TIMEOUT -sS "$CLICKHOUSE_URL&max_execution_time=1" -d " + SELECT * FROM ( SELECT a.name as n FROM @@ -68,12 +71,13 @@ ${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL&max_exec GROUP BY n ) LIMIT 20 - FORMAT Null" 2>&1 | grep -o "Code: 159" | sort | uniq + FORMAT Null +" 2>&1 | grep -o "Code: 159" | sort | uniq ### Should stop pulling data and return what has been generated already (return code 0) -${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL" -d \ - "SELECT a.name as n +${CLICKHOUSE_CURL} -q --max-time $TIMEOUT -sS "$CLICKHOUSE_URL" -d " + SELECT a.name as n FROM ( SELECT 'Name' as name, number FROM system.numbers LIMIT 2000000 @@ -83,5 +87,5 @@ ${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL" -d \ ) as b FORMAT Null SETTINGS max_execution_time = 1, timeout_overflow_mode = 'break' - " +" echo $? From 8a48b3334433fe5e77c23ff6df10e454db2d3f82 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 11 Aug 2024 21:27:08 +0200 Subject: [PATCH 556/644] Fix settings/current_database in system.processes for async BACKUP/RESTORE Signed-off-by: Azat Khuzhin --- src/Backups/BackupsWorker.cpp | 4 ++++ src/Interpreters/ProcessList.h | 3 +++ 2 files changed, 7 insertions(+) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 0b93ae6d547..8b45c816817 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -490,6 +490,8 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context /// process_list_element_holder is used to make an element in ProcessList live while BACKUP is working asynchronously. auto process_list_element = context_in_use->getProcessListElement(); + /// Update context to preserve query information in processlist (settings, current_database) + process_list_element->updateContext(context_in_use); thread_pool.scheduleOrThrowOnError( [this, @@ -853,6 +855,8 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt /// process_list_element_holder is used to make an element in ProcessList live while RESTORE is working asynchronously. auto process_list_element = context_in_use->getProcessListElement(); + /// Update context to preserve query information in processlist (settings, current_database) + process_list_element->updateContext(context_in_use); thread_pool.scheduleOrThrowOnError( [this, diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index accb73e12df..248ba947bc1 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -244,6 +244,9 @@ public: /// Same as checkTimeLimit but it never throws [[nodiscard]] bool checkTimeLimitSoft(); + /// Use it in case of the query left in background to execute asynchronously + void updateContext(ContextWeakPtr weak_context) { context = std::move(weak_context); } + /// Get the reference for the start of the query. Used to synchronize with other Stopwatches UInt64 getQueryCPUStartTime() { return watch.getStart(); } }; From 5c8665c66069256f4e34fb32068fab2fcb90cc65 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 11 Aug 2024 20:40:55 +0000 Subject: [PATCH 557/644] fix system.kafka_consumers and doc, fix tidy --- docs/en/operations/system-tables/kafka_consumers.md | 1 + src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp | 2 +- src/Storages/System/StorageSystemKafkaConsumers.cpp | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/system-tables/kafka_consumers.md b/docs/en/operations/system-tables/kafka_consumers.md index 7e28a251e26..d58c9f754fd 100644 --- a/docs/en/operations/system-tables/kafka_consumers.md +++ b/docs/en/operations/system-tables/kafka_consumers.md @@ -24,6 +24,7 @@ Columns: - `num_rebalance_revocations`, (UInt64) - number of times the consumer was revoked its partitions - `num_rebalance_assignments`, (UInt64) - number of times the consumer was assigned to Kafka cluster - `is_currently_used`, (UInt8) - consumer is in use +- `last_used`, (UInt64) - last time this consumer was in use, unix time in microseconds - `rdkafka_stat` (String) - library internal statistic. See https://github.com/ClickHouse/librdkafka/blob/master/STATISTICS.md . Set `statistics_interval_ms` to 0 disable, default is 3000 (once in three seconds). Example: diff --git a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp index 3ca7e8183f1..86cec8659f5 100644 --- a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp +++ b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp @@ -129,7 +129,7 @@ void OptimizeShardingKeyRewriteInMatcher::visit(ASTFunction & function, Data & d if (shardContains(child, name, data)) new_tuple.emplace_back(std::move(child)); - if (new_tuple.size() == 0) + if (new_tuple.empty()) new_tuple.emplace_back(std::move(tuple.back())); tuple_literal->value = std::move(new_tuple); diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 5e790587716..db6804d3ad7 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -42,7 +42,7 @@ ColumnsDescription StorageSystemKafkaConsumers::getColumnsDescription() {"num_rebalance_revocations", std::make_shared(), "Number of times the consumer was revoked its partitions."}, {"num_rebalance_assignments", std::make_shared(), "Number of times the consumer was assigned to Kafka cluster."}, {"is_currently_used", std::make_shared(), "The flag which shows whether the consumer is in use."}, - {"last_used", std::make_shared(6), "The last time this consumer was in use."}, + {"last_used", std::make_shared(), "The last time this consumer was in use, unix time in microseconds."}, {"rdkafka_stat", std::make_shared(), "Library internal statistic. Set statistics_interval_ms to 0 disable, default is 3000 (once in three seconds)."}, }; } From 957a0b6ea4c3e262a5c1fa664d81ab31d7e0d757 Mon Sep 17 00:00:00 2001 From: sakulali Date: Sun, 11 Aug 2024 00:12:36 +0800 Subject: [PATCH 558/644] Add a setting query_cache_tag --- docs/en/operations/query-cache.md | 10 ++++++ docs/en/operations/settings/settings.md | 11 +++++++ .../operations/system-tables/query_cache.md | 2 ++ src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/Interpreters/Cache/QueryCache.cpp | 16 +++++++--- src/Interpreters/Cache/QueryCache.h | 13 ++++++-- src/Interpreters/executeQuery.cpp | 5 +-- .../System/StorageSystemQueryCache.cpp | 5 ++- .../02494_query_cache_tag.reference | 14 ++++++++ .../0_stateless/02494_query_cache_tag.sql | 32 +++++++++++++++++++ 11 files changed, 100 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02494_query_cache_tag.reference create mode 100644 tests/queries/0_stateless/02494_query_cache_tag.sql diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index 7a920671fc2..a6c4d74f4ac 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -143,6 +143,16 @@ value can be specified at session, profile or query level using setting [query_c Entries in the query cache are compressed by default. This reduces the overall memory consumption at the cost of slower writes into / reads from the query cache. To disable compression, use setting [query_cache_compress_entries](settings/settings.md#query-cache-compress-entries). +Entries in the query cache can separate by tag, using setting [query_cache_tag](settings/settings.md#query-cache-tag). Queries with different tags are considered different entries. For example, the result of query + +``` sql +SELECT 1 SETTINGS use_query_cache = true; +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one'; +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one diff'; +``` + +have different entries in the query cache, find the specified tag in system table [system.query_cache](system-tables/query_cache.md) + ClickHouse reads table data in blocks of [max_block_size](settings/settings.md#setting-max_block_size) rows. Due to filtering, aggregation, etc., result blocks are typically much smaller than 'max_block_size' but there are also cases where they are much bigger. Setting [query_cache_squash_partial_results](settings/settings.md#query-cache-squash-partial-results) (enabled by default) controls if result blocks diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e432f4e038f..7b855665efb 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1800,6 +1800,17 @@ Possible values: Default value: `0`. +## query_cache_tag {#query-cache-tag} + +An arbitrary string to separate entries in the [query cache](../query-cache.md). +Queries with different values of this setting are considered different. + +Possible values: + +- string: name of query cache tag + +Default value: `''`. + ## query_cache_max_size_in_bytes {#query-cache-max-size-in-bytes} The maximum amount of memory (in bytes) the current user may allocate in the [query cache](../query-cache.md). 0 means unlimited. diff --git a/docs/en/operations/system-tables/query_cache.md b/docs/en/operations/system-tables/query_cache.md index a9f86f5fc2b..393b37d3616 100644 --- a/docs/en/operations/system-tables/query_cache.md +++ b/docs/en/operations/system-tables/query_cache.md @@ -14,6 +14,7 @@ Columns: - `compressed` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is compressed. - `expires_at` ([DateTime](../../sql-reference/data-types/datetime.md)) — When the query cache entry becomes stale. - `key_hash` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — A hash of the query string, used as a key to find query cache entries. +- `tag` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — An arbitrary string to separate entries in the query cache. **Example** @@ -31,6 +32,7 @@ shared: 0 compressed: 1 expires_at: 2023-10-13 13:35:45 key_hash: 12188185624808016954 +tag: 1 row in set. Elapsed: 0.004 sec. ``` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4559cc67b35..ed58f8041d0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -676,6 +676,7 @@ class IColumn; M(Bool, query_cache_squash_partial_results, true, "Squash partial result blocks to blocks of size 'max_block_size'. Reduces performance of inserts into the query cache but improves the compressability of cache entries.", 0) \ M(Seconds, query_cache_ttl, 60, "After this time in seconds entries in the query cache become stale", 0) \ M(Bool, query_cache_share_between_users, false, "Allow other users to read entry in the query cache", 0) \ + M(String, query_cache_tag, "", "An arbitrary string to separate entries in the query cache. Queries with different values of this setting are considered different.", 0) \ M(Bool, enable_sharing_sets_for_mutations, true, "Allow sharing set objects build for IN subqueries between different tasks of the same mutation. This reduces memory usage and CPU consumption", 0) \ \ M(Bool, optimize_rewrite_sum_if_to_count_if, true, "Rewrite sumIf() and sum(if()) function countIf() function when logically equivalent", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index c6392044f72..49a325b07b1 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -84,6 +84,7 @@ static std::initializer_list user_id_, const std::vector & current_user_roles_, bool is_shared_, std::chrono::time_point expires_at_, - bool is_compressed_) - : ast_hash(calculateAstHash(ast_, current_database, settings)) + bool is_compressed_, + const String & tag_) + : ast_hash(calculateAstHash(ast_, current_database, settings, tag_)) , header(header_) , user_id(user_id_) , current_user_roles(current_user_roles_) @@ -242,11 +247,12 @@ QueryCache::Key::Key( , expires_at(expires_at_) , is_compressed(is_compressed_) , query_string(queryStringFromAST(ast_)) + , tag(tag_) { } -QueryCache::Key::Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional user_id_, const std::vector & current_user_roles_) - : QueryCache::Key(ast_, current_database, settings, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST, current database, user name/roles +QueryCache::Key::Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional user_id_, const std::vector & current_user_roles_, const String & tag_) + : QueryCache::Key(ast_, current_database, settings, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false, tag_) /// dummy values for everything != AST, current database, user name/roles { } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 461197cac32..54de5edb145 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -88,6 +88,10 @@ public: /// SYSTEM.QUERY_CACHE. const String query_string; + /// An arbitrary string to separate entries in the query cache. + /// Queries with different values of this setting are considered different. + const String tag; + /// Ctor to construct a Key for writing into query cache. Key(ASTPtr ast_, const String & current_database, @@ -96,10 +100,15 @@ public: std::optional user_id_, const std::vector & current_user_roles_, bool is_shared_, std::chrono::time_point expires_at_, - bool is_compressed); + bool is_compressed, + const String & tag_); /// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name). - Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional user_id_, const std::vector & current_user_roles_); + Key(ASTPtr ast_, + const String & current_database, + const Settings & settings, + std::optional user_id_, const std::vector & current_user_roles_, + const String & tag_); bool operator==(const Key & other) const; }; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index fe87eed5570..6422d3128fa 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1129,7 +1129,7 @@ static std::tuple executeQueryImpl( { if (can_use_query_cache && settings.enable_reads_from_query_cache) { - QueryCache::Key key(ast, context->getCurrentDatabase(), *settings_copy, context->getUserID(), context->getCurrentRoles()); + QueryCache::Key key(ast, context->getCurrentDatabase(), *settings_copy, context->getUserID(), context->getCurrentRoles(), settings.query_cache_tag); QueryCache::Reader reader = query_cache->createReader(key); if (reader.hasCacheEntryForKey()) { @@ -1258,7 +1258,8 @@ static std::tuple executeQueryImpl( context->getUserID(), context->getCurrentRoles(), settings.query_cache_share_between_users, std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl), - settings.query_cache_compress_entries); + settings.query_cache_compress_entries, + settings.query_cache_tag); const size_t num_query_runs = settings.query_cache_min_query_runs ? query_cache->recordQueryRun(key) : 1; /// try to avoid locking a mutex in recordQueryRun() if (num_query_runs <= settings.query_cache_min_query_runs) diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 4c54d4ae16f..f81d50e8806 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -1,6 +1,7 @@ #include "StorageSystemQueryCache.h" #include #include +#include #include #include #include @@ -19,7 +20,8 @@ ColumnsDescription StorageSystemQueryCache::getColumnsDescription() {"shared", std::make_shared(), "If the query cache entry is shared between multiple users."}, {"compressed", std::make_shared(), "If the query cache entry is compressed."}, {"expires_at", std::make_shared(), "When the query cache entry becomes stale."}, - {"key_hash", std::make_shared(), "A hash of the query string, used as a key to find query cache entries."} + {"key_hash", std::make_shared(), "A hash of the query string, used as a key to find query cache entries."}, + {"tag", std::make_shared(std::make_shared()), "An arbitrary string to separate entries in the query cache."} }; } @@ -56,6 +58,7 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr res_columns[4]->insert(key.is_compressed); res_columns[5]->insert(std::chrono::system_clock::to_time_t(key.expires_at)); res_columns[6]->insert(key.ast_hash.low64); /// query cache considers aliases (issue #56258) + res_columns[7]->insert(key.tag); } } diff --git a/tests/queries/0_stateless/02494_query_cache_tag.reference b/tests/queries/0_stateless/02494_query_cache_tag.reference new file mode 100644 index 00000000000..055d3d4c5bb --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_tag.reference @@ -0,0 +1,14 @@ +1 +1 +--- +1 +1 +1 +2 +--- +1 +1 +1 +2 +1 +3 diff --git a/tests/queries/0_stateless/02494_query_cache_tag.sql b/tests/queries/0_stateless/02494_query_cache_tag.sql new file mode 100644 index 00000000000..054607058e8 --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_tag.sql @@ -0,0 +1,32 @@ +-- Tags: no-parallel +-- Tag no-parallel: Messes with internal cache + +SYSTEM DROP QUERY CACHE; + +-- Cache the query after the query invocation +SELECT 1 SETTINGS use_query_cache = true; +SELECT COUNT(*) FROM system.query_cache; + +SELECT '---'; + +SYSTEM DROP QUERY CACHE; + +-- Queries with tag value of this setting or not are considered different cache entries. +SELECT 1 SETTINGS use_query_cache = true; +SELECT COUNT(*) FROM system.query_cache; +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one'; +SELECT COUNT(*) FROM system.query_cache; + +SELECT '---'; + +SYSTEM DROP QUERY CACHE; + +-- Queries with different tags values of this setting are considered different cache entries. +SELECT 1 SETTINGS use_query_cache = true; +SELECT COUNT(*) FROM system.query_cache; +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one'; +SELECT COUNT(*) FROM system.query_cache; +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one diff'; +SELECT COUNT(*) FROM system.query_cache; + +SYSTEM DROP QUERY CACHE; From 5acf9f6f8160ee6de2845b13bed07d63832ca3fa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 07:01:52 +0200 Subject: [PATCH 559/644] Fix `test_cluster_all_replicas` --- tests/integration/test_cluster_all_replicas/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_cluster_all_replicas/test.py b/tests/integration/test_cluster_all_replicas/test.py index d8bad180e1b..9797db7c498 100644 --- a/tests/integration/test_cluster_all_replicas/test.py +++ b/tests/integration/test_cluster_all_replicas/test.py @@ -21,14 +21,14 @@ def start_cluster(): def test_cluster(start_cluster): assert ( node1.query( - "SELECT hostName() FROM clusterAllReplicas('one_shard_two_nodes', system.one)" + "SELECT hostName() FROM clusterAllReplicas('one_shard_two_nodes', system.one) ORDER BY ALL" ) == "node1\nnode2\n" ) assert set( node1.query( - """SELECT hostName(), * FROM clusterAllReplicas("one_shard_two_nodes", system.one) ORDER BY dummy""" + """SELECT hostName(), * FROM clusterAllReplicas("one_shard_two_nodes", system.one) ORDER BY ALL""" ).splitlines() ) == {"node1\t0", "node2\t0"} @@ -48,7 +48,7 @@ def test_global_in(start_cluster): assert set( node1.query( - """SELECT hostName(), * FROM clusterAllReplicas("one_shard_two_nodes", system.one) where dummy GLOBAL IN u""" + """SELECT hostName(), * FROM clusterAllReplicas("one_shard_two_nodes", system.one) where dummy GLOBAL IN u ORDER BY ALL""" ).splitlines() ) == {"node1\t0", "node2\t0"} @@ -63,7 +63,7 @@ def test_global_in(start_cluster): def test_skip_unavailable_replica(start_cluster, cluster): assert ( node1.query( - f"SELECT hostName() FROM clusterAllReplicas('{cluster}', system.one) settings skip_unavailable_shards=1" + f"SELECT hostName() FROM clusterAllReplicas('{cluster}', system.one) ORDER BY ALL settings skip_unavailable_shards=1" ) == "node1\nnode2\n" ) @@ -81,5 +81,5 @@ def test_error_on_unavailable_replica(start_cluster, cluster): # so when skip_unavailable_shards=0 - any unavailable replica should lead to an error with pytest.raises(QueryRuntimeException): node1.query( - f"SELECT hostName() FROM clusterAllReplicas('{cluster}', system.one) settings skip_unavailable_shards=0" + f"SELECT hostName() FROM clusterAllReplicas('{cluster}', system.one) ORDER BY ALL settings skip_unavailable_shards=0" ) From b3504def35ba6baf7a7cf1f9b84a72e8f70f95f3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 07:15:08 +0200 Subject: [PATCH 560/644] Fix leftovers --- tests/config/config.d/transactions.xml | 4 ++-- .../test_distributed_type_object/configs/remote_servers.xml | 2 +- .../configs/host_regexp.xml | 4 ++-- .../configs/listen_host.xml | 2 +- .../test_jbod_ha/configs/config.d/storage_configuration.xml | 2 +- .../integration/test_server_reload/configs/default_passwd.xml | 2 +- .../test_server_reload/configs/overrides_from_zk.xml | 2 +- tests/integration/test_version_update/configs/log_conf.xml | 2 +- 8 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/config/config.d/transactions.xml b/tests/config/config.d/transactions.xml index 9948b1f1865..64e166b81b5 100644 --- a/tests/config/config.d/transactions.xml +++ b/tests/config/config.d/transactions.xml @@ -1,4 +1,4 @@ - + 42 @@ -18,4 +18,4 @@ 0.01 - + diff --git a/tests/integration/test_distributed_type_object/configs/remote_servers.xml b/tests/integration/test_distributed_type_object/configs/remote_servers.xml index ebce4697529..0ea61f0d5fc 100644 --- a/tests/integration/test_distributed_type_object/configs/remote_servers.xml +++ b/tests/integration/test_distributed_type_object/configs/remote_servers.xml @@ -1,4 +1,4 @@ - + diff --git a/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml b/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml index 7a2141e6c7e..0bf7fad9a70 100644 --- a/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml +++ b/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml @@ -1,4 +1,4 @@ - + @@ -8,4 +8,4 @@ default - \ No newline at end of file + diff --git a/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml b/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml index 58ef55cd3f3..4f0841ab8b6 100644 --- a/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml +++ b/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml @@ -1,4 +1,4 @@ - + :: 0.0.0.0 1 diff --git a/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml b/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml index b5c351d105b..b2c4645644a 100644 --- a/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml @@ -1,4 +1,4 @@ - + 1000 diff --git a/tests/integration/test_server_reload/configs/default_passwd.xml b/tests/integration/test_server_reload/configs/default_passwd.xml index f79149e7e23..45ae005bd19 100644 --- a/tests/integration/test_server_reload/configs/default_passwd.xml +++ b/tests/integration/test_server_reload/configs/default_passwd.xml @@ -1,4 +1,4 @@ - + diff --git a/tests/integration/test_server_reload/configs/overrides_from_zk.xml b/tests/integration/test_server_reload/configs/overrides_from_zk.xml index d420faa88a2..b17c5c9fa99 100644 --- a/tests/integration/test_server_reload/configs/overrides_from_zk.xml +++ b/tests/integration/test_server_reload/configs/overrides_from_zk.xml @@ -1,4 +1,4 @@ - + diff --git a/tests/integration/test_version_update/configs/log_conf.xml b/tests/integration/test_version_update/configs/log_conf.xml index f9d15e572aa..17215c1759d 100644 --- a/tests/integration/test_version_update/configs/log_conf.xml +++ b/tests/integration/test_version_update/configs/log_conf.xml @@ -1,4 +1,4 @@ - + trace /var/log/clickhouse-server/log.log From b92a8f0fbcec640fbac35fca1b3fb23914d44990 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 07:16:25 +0200 Subject: [PATCH 561/644] Fix leftovers --- .../external-authenticators/kerberos.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/zh/operations/external-authenticators/kerberos.md b/docs/zh/operations/external-authenticators/kerberos.md index 649a0b9bd48..d1a39bbc952 100644 --- a/docs/zh/operations/external-authenticators/kerberos.md +++ b/docs/zh/operations/external-authenticators/kerberos.md @@ -23,30 +23,30 @@ slug: /zh/operations/external-authenticators/kerberos 示例 (进入 `config.xml`): ```xml - + - + ``` 主体规范: ```xml - + HTTP/clickhouse.example.com@EXAMPLE.COM - + ``` 按领域过滤: ```xml - + EXAMPLE.COM - + ``` !!! warning "注意" @@ -74,7 +74,7 @@ Kerberos主体名称格式通常遵循以下模式: 示例 (进入 `users.xml`): ``` - + @@ -85,7 +85,7 @@ Kerberos主体名称格式通常遵循以下模式: - + ``` !!! warning "警告" From 6f189e9eb7ded15df4ddbe7f90dafb28feaab2e3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 07:17:17 +0200 Subject: [PATCH 562/644] Fix leftovers --- .../test_distributed_type_object/configs/remote_servers.xml | 2 +- .../configs/host_regexp.xml | 2 +- .../configs/listen_host.xml | 2 +- .../test_jbod_ha/configs/config.d/storage_configuration.xml | 2 +- tests/integration/test_server_reload/configs/default_passwd.xml | 2 +- .../test_server_reload/configs/overrides_from_zk.xml | 2 +- tests/integration/test_version_update/configs/log_conf.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_distributed_type_object/configs/remote_servers.xml b/tests/integration/test_distributed_type_object/configs/remote_servers.xml index 0ea61f0d5fc..68b420f36b4 100644 --- a/tests/integration/test_distributed_type_object/configs/remote_servers.xml +++ b/tests/integration/test_distributed_type_object/configs/remote_servers.xml @@ -15,4 +15,4 @@ - + diff --git a/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml b/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml index 0bf7fad9a70..9329c8dbde2 100644 --- a/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml +++ b/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml @@ -8,4 +8,4 @@ default - + diff --git a/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml b/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml index 4f0841ab8b6..9c27c612f63 100644 --- a/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml +++ b/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml @@ -2,4 +2,4 @@ :: 0.0.0.0 1 - + diff --git a/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml b/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml index b2c4645644a..fb9acc58ad6 100644 --- a/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml @@ -27,4 +27,4 @@ - + diff --git a/tests/integration/test_server_reload/configs/default_passwd.xml b/tests/integration/test_server_reload/configs/default_passwd.xml index 45ae005bd19..9d664cbf9c4 100644 --- a/tests/integration/test_server_reload/configs/default_passwd.xml +++ b/tests/integration/test_server_reload/configs/default_passwd.xml @@ -9,4 +9,4 @@ 123 - + diff --git a/tests/integration/test_server_reload/configs/overrides_from_zk.xml b/tests/integration/test_server_reload/configs/overrides_from_zk.xml index b17c5c9fa99..aa6105f6ebe 100644 --- a/tests/integration/test_server_reload/configs/overrides_from_zk.xml +++ b/tests/integration/test_server_reload/configs/overrides_from_zk.xml @@ -7,4 +7,4 @@ - + diff --git a/tests/integration/test_version_update/configs/log_conf.xml b/tests/integration/test_version_update/configs/log_conf.xml index 17215c1759d..27c7107ce5e 100644 --- a/tests/integration/test_version_update/configs/log_conf.xml +++ b/tests/integration/test_version_update/configs/log_conf.xml @@ -8,4 +8,4 @@ /var/log/clickhouse-server/stderr.log /var/log/clickhouse-server/stdout.log - + From 6016dc96aae57f38ef3ace1ed687b82bcc437425 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 07:19:54 +0200 Subject: [PATCH 563/644] Fix test `01172_transaction_counters` --- programs/server/config.d/transactions.xml | 1 + .../01172_transaction_counters.reference | 44 +++++++++---------- .../01172_transaction_counters.sql | 1 - 3 files changed, 23 insertions(+), 23 deletions(-) create mode 120000 programs/server/config.d/transactions.xml diff --git a/programs/server/config.d/transactions.xml b/programs/server/config.d/transactions.xml new file mode 120000 index 00000000000..be9de46b607 --- /dev/null +++ b/programs/server/config.d/transactions.xml @@ -0,0 +1 @@ +../../../tests/config/config.d/transactions.xml \ No newline at end of file diff --git a/tests/queries/0_stateless/01172_transaction_counters.reference b/tests/queries/0_stateless/01172_transaction_counters.reference index 24083d7d40b..0fd73c7bcec 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.reference +++ b/tests/queries/0_stateless/01172_transaction_counters.reference @@ -16,25 +16,25 @@ 7 all_3_3_0 (0,0,'00000000-0000-0000-0000-000000000000') 0 7 all_4_4_0 (0,0,'00000000-0000-0000-0000-000000000000') 0 8 1 -1 1 AddPart 1 1 1 1 all_1_1_0 -2 1 Begin 1 1 1 1 -2 1 AddPart 1 1 1 1 all_2_2_0 -2 1 Rollback 1 1 1 1 -3 1 Begin 1 1 1 1 -3 1 AddPart 1 1 1 1 all_3_3_0 -3 1 Commit 1 1 1 0 -1 1 LockPart 1 1 1 1 all_2_2_0 -4 1 Begin 1 1 1 1 -4 1 AddPart 1 1 1 1 all_4_4_0 -4 1 Commit 1 1 1 0 -5 1 Begin 1 1 1 1 -5 1 AddPart 1 1 1 1 all_5_5_0 -5 1 LockPart 1 1 1 1 all_1_1_0 -5 1 LockPart 1 1 1 1 all_3_3_0 -5 1 LockPart 1 1 1 1 all_4_4_0 -5 1 LockPart 1 1 1 1 all_5_5_0 -5 1 UnlockPart 1 1 1 1 all_1_1_0 -5 1 UnlockPart 1 1 1 1 all_3_3_0 -5 1 UnlockPart 1 1 1 1 all_4_4_0 -5 1 UnlockPart 1 1 1 1 all_5_5_0 -5 1 Rollback 1 1 1 1 +1 AddPart 1 1 1 1 all_1_1_0 +2 Begin 1 1 1 1 +2 AddPart 1 1 1 1 all_2_2_0 +2 Rollback 1 1 1 1 +3 Begin 1 1 1 1 +3 AddPart 1 1 1 1 all_3_3_0 +3 Commit 1 1 1 0 +1 LockPart 1 1 1 1 all_2_2_0 +4 Begin 1 1 1 1 +4 AddPart 1 1 1 1 all_4_4_0 +4 Commit 1 1 1 0 +5 Begin 1 1 1 1 +5 AddPart 1 1 1 1 all_5_5_0 +5 LockPart 1 1 1 1 all_1_1_0 +5 LockPart 1 1 1 1 all_3_3_0 +5 LockPart 1 1 1 1 all_4_4_0 +5 LockPart 1 1 1 1 all_5_5_0 +5 UnlockPart 1 1 1 1 all_1_1_0 +5 UnlockPart 1 1 1 1 all_3_3_0 +5 UnlockPart 1 1 1 1 all_4_4_0 +5 UnlockPart 1 1 1 1 all_5_5_0 +5 Rollback 1 1 1 1 diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index a809e4196e9..581b45cd15c 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -42,7 +42,6 @@ rollback; system flush logs; select indexOf((select arraySort(groupUniqArray(tid)) from system.transactions_info_log where database=currentDatabase() and table='txn_counters'), tid), - (toDecimal64(now64(6), 6) - toDecimal64(event_time, 6)) < 100, type, thread_id!=0, length(query_id)=length(queryID()) or type='Commit' and query_id='', -- ignore fault injection after commit From 574c445be9368ee481e8d213251106233d417f69 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Aug 2024 16:29:07 +0000 Subject: [PATCH 564/644] Refactor tests for (experimental) statistics --- docs/en/development/tests.md | 4 +- .../statements/alter/statistics.md | 16 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- ...2864_statistics_count_min_sketch.reference | 14 -- .../02864_statistics_count_min_sketch.sql | 70 ------ .../02864_statistics_ddl.reference | 37 +-- .../0_stateless/02864_statistics_ddl.sql | 234 ++++++++++++++---- ...delayed_materialization_in_merge.reference | 12 + ...stics_delayed_materialization_in_merge.sql | 36 +++ .../02864_statistics_exception.reference | 0 .../02864_statistics_exception.sql | 55 ---- ..._statistics_materialize_in_merge.reference | 10 - .../02864_statistics_materialize_in_merge.sql | 52 ---- .../02864_statistics_predicates.reference | 98 ++++++++ .../02864_statistics_predicates.sql | 214 ++++++++++++++++ .../02864_statistics_uniq.reference | 35 --- .../0_stateless/02864_statistics_uniq.sql | 73 ------ .../02864_statistics_usage.reference | 20 ++ .../0_stateless/02864_statistics_usage.sql | 42 ++++ 19 files changed, 625 insertions(+), 399 deletions(-) delete mode 100644 tests/queries/0_stateless/02864_statistics_count_min_sketch.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_count_min_sketch.sql create mode 100644 tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference create mode 100644 tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_exception.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_exception.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql create mode 100644 tests/queries/0_stateless/02864_statistics_predicates.reference create mode 100644 tests/queries/0_stateless/02864_statistics_predicates.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_uniq.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_uniq.sql create mode 100644 tests/queries/0_stateless/02864_statistics_usage.reference create mode 100644 tests/queries/0_stateless/02864_statistics_usage.sql diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 269995a1a96..6cb36e2049b 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -14,7 +14,7 @@ Each functional test sends one or multiple queries to the running ClickHouse ser Tests are located in `queries` directory. There are two subdirectories: `stateless` and `stateful`. Stateless tests run queries without any preloaded test data - they often create small synthetic datasets on the fly, within the test itself. Stateful tests require preloaded test data from ClickHouse and it is available to general public. -Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`. +Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`. :::note A common mistake when testing data types `DateTime` and `DateTime64` is assuming that the server uses a specific time zone (e.g. "UTC"). This is not the case, time zones in CI test runs @@ -38,7 +38,7 @@ For more options, see `tests/clickhouse-test --help`. You can simply run all tes ### Adding a New Test -To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client --multiquery < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. +To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables. diff --git a/docs/en/sql-reference/statements/alter/statistics.md b/docs/en/sql-reference/statements/alter/statistics.md index 6880cef0e5c..7a1774a01b5 100644 --- a/docs/en/sql-reference/statements/alter/statistics.md +++ b/docs/en/sql-reference/statements/alter/statistics.md @@ -8,26 +8,28 @@ sidebar_label: STATISTICS The following operations are available: -- `ALTER TABLE [db].table ADD STATISTICS (columns list) TYPE (type list)` - Adds statistic description to tables metadata. +- `ALTER TABLE [db].table ADD STATISTICS [IF NOT EXISTS] (column list) TYPE (type list)` - Adds statistic description to tables metadata. -- `ALTER TABLE [db].table MODIFY STATISTICS (columns list) TYPE (type list)` - Modifies statistic description to tables metadata. +- `ALTER TABLE [db].table MODIFY STATISTICS (column list) TYPE (type list)` - Modifies statistic description to tables metadata. -- `ALTER TABLE [db].table DROP STATISTICS (columns list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns. +- `ALTER TABLE [db].table DROP STATISTICS [IF EXISTS] (column list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns. -- `ALTER TABLE [db].table CLEAR STATISTICS (columns list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`. +- `ALTER TABLE [db].table CLEAR STATISTICS [IF EXISTS] (column list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`. -- `ALTER TABLE [db.]table MATERIALIZE STATISTICS (columns list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +- `ALTER TABLE [db.]table MATERIALIZE STATISTICS [IF EXISTS] (column list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). The first two commands are lightweight in a sense that they only change metadata or remove files. Also, they are replicated, syncing statistics metadata via ZooKeeper. -There is an example adding two statistics types to two columns: +## Example: + +Adding two statistics types to two columns: ``` ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq; ``` :::note -Statistic manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). +Statistic are supported only for [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine tables (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). ::: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 625b1281c61..c7101021f02 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3517,7 +3517,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context const auto & new_column = new_metadata.getColumns().get(command.column_name); if (!old_column.type->equals(*new_column.type)) throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, - "ALTER types of column {} with statistics is not not safe " + "ALTER types of column {} with statistics is not safe " "because it can change the representation of statistics", backQuoteIfNeed(command.column_name)); } diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference b/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference deleted file mode 100644 index 02c41656a36..00000000000 --- a/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference +++ /dev/null @@ -1,14 +0,0 @@ -CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -Test statistics count_min: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) -Test statistics multi-types: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) -Test LowCardinality and Nullable data type: -tab2 diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql b/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql deleted file mode 100644 index c730aa7b4a7..00000000000 --- a/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql +++ /dev/null @@ -1,70 +0,0 @@ --- Tags: no-fasttest - -DROP TABLE IF EXISTS tab SYNC; - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; -SET allow_suspicious_low_cardinality_types=1; -SET mutations_sync = 2; - -CREATE TABLE tab -( - a String, - b UInt64, - c Int64, - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; - -SHOW CREATE TABLE tab; - -INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT 'Test statistics count_min:'; - -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -ALTER TABLE tab ADD STATISTICS b TYPE count_min; -ALTER TABLE tab ADD STATISTICS c TYPE count_min; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab DROP STATISTICS a, b, c; - - -SELECT 'Test statistics multi-types:'; - -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; -ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab DROP STATISTICS a, b, c; - -DROP TABLE IF EXISTS tab SYNC; - - -SELECT 'Test LowCardinality and Nullable data type:'; -DROP TABLE IF EXISTS tab2 SYNC; -SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE tab2 -( - a LowCardinality(Int64) STATISTICS(count_min), - b Nullable(Int64) STATISTICS(count_min), - c LowCardinality(Nullable(Int64)) STATISTICS(count_min), - pk String, -) Engine = MergeTree() ORDER BY pk; - -select name from system.tables where name = 'tab2' and database = currentDatabase(); - -DROP TABLE IF EXISTS tab2 SYNC; diff --git a/tests/queries/0_stateless/02864_statistics_ddl.reference b/tests/queries/0_stateless/02864_statistics_ddl.reference index a7ff5caa0b0..0e453b0ee8a 100644 --- a/tests/queries/0_stateless/02864_statistics_ddl.reference +++ b/tests/queries/0_stateless/02864_statistics_ddl.reference @@ -1,31 +1,6 @@ -CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After insert - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) -10 -0 -After drop statistic - Prewhere info - Prewhere filter - Prewhere filter column: and(less(b, 10), less(a, 10)) (removed) -10 -CREATE TABLE default.tab\n(\n `a` Float64,\n `b` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After add statistic -CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After materialize statistic - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) -20 -After merge - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) -20 -CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After rename - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(c, 10)) (removed) -20 +CREATE TABLE default.tab\n(\n `f64` Float64,\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32,\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64,\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32,\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02864_statistics_ddl.sql b/tests/queries/0_stateless/02864_statistics_ddl.sql index fe612efe2ac..32b56a842b7 100644 --- a/tests/queries/0_stateless/02864_statistics_ddl.sql +++ b/tests/queries/0_stateless/02864_statistics_ddl.sql @@ -1,59 +1,195 @@ --- Tests that various DDL statements create/drop/materialize statistics +-- Tags: no-fasttest +-- no-fasttest: 'count_min' sketches need a 3rd party library + +-- Tests that DDL statements which create / drop / materialize statistics + +SET mutations_sync = 1; DROP TABLE IF EXISTS tab; +-- Error case: Can't create statistics when allow_experimental_statistics = 0 +CREATE TABLE tab (col Float64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; + +-- Error case: Unknown statistics types are rejected +CREATE TABLE tab (col Float64 STATISTICS(no_statistics_type)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + +-- Error case: The same statistics type can't exist more than once on a column +CREATE TABLE tab (col Float64 STATISTICS(tdigest, tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + +SET allow_suspicious_low_cardinality_types = 1; + +-- Statistics can only be created on columns of specific data types (depending on the statistics kind), (*) + +-- tdigest requires data_type.isValueRepresentedByInteger +-- These types work: +CREATE TABLE tab (col UInt8 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col UInt256 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Float32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Decimal32(3) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col IPv4 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Nullable(UInt8) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +-- These types don't work: +CREATE TABLE tab (col String STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col FixedString(1) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Array(Float64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col UUID STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col IPv6 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } + +-- uniq requires data_type.isValueRepresentedByInteger +-- These types work: +CREATE TABLE tab (col UInt8 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col UInt256 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Float32 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Decimal32(3) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date32 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime64 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col IPv4 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Nullable(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +-- These types don't work: +CREATE TABLE tab (col String STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col FixedString(1) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Array(Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col UUID STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col IPv6 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } + +-- count_min requires data_type.isValueRepresentedByInteger or data_type = (Fixed)String +-- These types work: +CREATE TABLE tab (col UInt8 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col UInt256 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Float32 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Decimal32(3) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date32 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime64 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col IPv4 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Nullable(UInt8) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col String STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col FixedString(1) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +-- These types don't work: +CREATE TABLE tab (col Array(Float64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col UUID STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col IPv6 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } + +-- CREATE TABLE was easy, ALTER is more fun CREATE TABLE tab ( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; + f64 Float64, + f64_tdigest Float64 STATISTICS(tdigest), + f32 Float32, + s String, + a Array(Float64) +) +Engine = MergeTree() +ORDER BY tuple(); +-- Error case: Unknown statistics types are rejected +-- (relevant for ADD and MODIFY) +ALTER TABLE tab ADD STATISTICS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab MODIFY STATISTICS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } +-- for some reason, ALTER TABLE tab MODIFY STATISTICS IF EXISTS is not supported + +-- Error case: The same statistics type can't exist more than once on a column +-- (relevant for ADD and MODIFY) +-- Create the same statistics object twice +ALTER TABLE tab ADD STATISTICS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab MODIFY STATISTICS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } +-- Create an statistics which exists already +ALTER TABLE tab ADD STATISTICS f64_tdigest TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64_tdigest TYPE tdigest; -- no-op +ALTER TABLE tab MODIFY STATISTICS f64_tdigest TYPE tdigest; -- no-op + +-- Error case: Column does not exist +-- (relevant for ADD, MODIFY, DROP, CLEAR, and MATERIALIZE) +-- Note that the results are unfortunately quite inconsistent ... +ALTER TABLE tab ADD STATISTICS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS IF EXISTS no_such_column; -- no-op +ALTER TABLE tab CLEAR STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab CLEAR STATISTICS IF EXISTS no_such_column; -- no-op +ALTER TABLE tab MATERIALIZE STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MATERIALIZE STATISTICS IF EXISTS no_such_column; -- { serverError ILLEGAL_STATISTICS } + +-- Error case: Column exists but has no statistics +-- (relevant for MODIFY, DROP, CLEAR, and MATERIALIZE) +-- Note that the results are unfortunately quite inconsistent ... +ALTER TABLE tab MODIFY STATISTICS s TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS s; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS IF EXISTS s; -- no-op +ALTER TABLE tab CLEAR STATISTICS s; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab CLEAR STATISTICS IF EXISTS s; -- no-op +ALTER TABLE tab MATERIALIZE STATISTICS s; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MATERIALIZE STATISTICS IF EXISTS s; -- { serverError ILLEGAL_STATISTICS } + +-- We don't check systematically that that statistics can only be created via ALTER ADD STATISTICS on columns of specific data types (the +-- internal type validation code is tested already above, (*)). Only do a rudimentary check for each statistics type with a data type that +-- works and one that doesn't work. +-- tdigest +-- Works: +ALTER TABLE tab ADD STATISTICS f64 TYPE tdigest; ALTER TABLE tab DROP STATISTICS f64; +ALTER TABLE tab MODIFY STATISTICS f64 TYPE tdigest; ALTER TABLE tab DROP STATISTICS f64; +-- Doesn't work: +ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +-- uniq +-- Works: +ALTER TABLE tab ADD STATISTICS f64 TYPE uniq; ALTER TABLE tab DROP STATISTICS f64; +ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; +-- Doesn't work: +ALTER TABLE tab ADD STATISTICS a TYPE uniq; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS a TYPE uniq; -- { serverError ILLEGAL_STATISTICS } +-- count_min +-- Works: +ALTER TABLE tab ADD STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; +ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; +-- Doesn't work: +ALTER TABLE tab ADD STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS } + +-- Any data type changes on columns with statistics are disallowed, for simplicity even if the new data type is compatible with all existing +-- statistics objects (e.g. tdigest can be created on Float64 and UInt64) +ALTER TABLE tab MODIFY COLUMN f64_tdigest UInt64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } + +-- Finally, do a full-circle test of a good case. Print table definition after each step. +-- Intentionally specifying _two_ columns and _two_ statistics types to have that also tested. +SHOW CREATE TABLE tab; +ALTER TABLE tab ADD STATISTICS f64, f32 TYPE tdigest, uniq; +SHOW CREATE TABLE tab; +ALTER TABLE tab MODIFY STATISTICS f64, f32 TYPE tdigest, uniq; +SHOW CREATE TABLE tab; +ALTER TABLE tab CLEAR STATISTICS f64, f32; +SHOW CREATE TABLE tab; +ALTER TABLE tab MATERIALIZE STATISTICS f64, f32; +SHOW CREATE TABLE tab; +ALTER TABLE tab DROP STATISTICS f64, f32; SHOW CREATE TABLE tab; -INSERT INTO tab select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT 'After insert'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; -SELECT count(*) FROM tab WHERE b < NULL and a < '10'; - -ALTER TABLE tab DROP STATISTICS a, b; - -SELECT 'After drop statistic'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; - -SHOW CREATE TABLE tab; - -ALTER TABLE tab ADD STATISTICS a, b TYPE tdigest; - -SELECT 'After add statistic'; - -SHOW CREATE TABLE tab; - -ALTER TABLE tab MATERIALIZE STATISTICS a, b; -INSERT INTO tab select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT 'After materialize statistic'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; - -OPTIMIZE TABLE tab FINAL; - -SELECT 'After merge'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; - -ALTER TABLE tab RENAME COLUMN b TO c; -SHOW CREATE TABLE tab; - -SELECT 'After rename'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE c < 10 and a < 10; - -DROP TABLE IF EXISTS tab; +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference new file mode 100644 index 00000000000..eb5e685597c --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference @@ -0,0 +1,12 @@ +After insert + Prewhere info + Prewhere filter + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) +After merge + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After truncate, insert, and materialize + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql new file mode 100644 index 00000000000..d469a4c2036 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql @@ -0,0 +1,36 @@ +-- Tests delayed materialization of statistics in merge instead of during insert (setting 'materialize_statistics_on_insert = 0'). +-- (The concrete statistics type, column data type and predicate type don't matter) + +-- Checks by the predicate evaluation order in EXPLAIN. This is quite fragile, a better approach would be helpful (maybe 'send_logs_level'?) + +DROP TABLE IF EXISTS tab; + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET enable_analyzer = 1; + +SET materialize_statistics_on_insert = 0; + +CREATE TABLE tab +( + a Int64 STATISTICS(tdigest), + b Int16 STATISTICS(tdigest), +) ENGINE = MergeTree() ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics. + +INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; +SELECT 'After insert'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks b first, then a (statistics not used) + +OPTIMIZE TABLE tab FINAL; +SELECT 'After merge'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +TRUNCATE TABLE tab; +SET mutations_sync = 2; +INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; +ALTER TABLE tab MATERIALIZE STATISTICS a, b; +SELECT 'After truncate, insert, and materialize'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_exception.reference b/tests/queries/0_stateless/02864_statistics_exception.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02864_statistics_exception.sql b/tests/queries/0_stateless/02864_statistics_exception.sql deleted file mode 100644 index 289ffee6600..00000000000 --- a/tests/queries/0_stateless/02864_statistics_exception.sql +++ /dev/null @@ -1,55 +0,0 @@ --- Tests creating/dropping/materializing statistics produces the right exceptions. - -DROP TABLE IF EXISTS tab; - --- Can't create statistics when allow_experimental_statistics = 0 -CREATE TABLE tab -( - a Float64 STATISTICS(tdigest) -) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - -SET allow_experimental_statistics = 1; - --- The same type of statistics can't exist more than once on a column -CREATE TABLE tab -( - a Float64 STATISTICS(tdigest, tdigest) -) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - --- Unknown statistics types are rejected -CREATE TABLE tab -( - a Float64 STATISTICS(no_statistics_type) -) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - --- tDigest statistics can only be created on numeric columns -CREATE TABLE tab -( - a String STATISTICS(tdigest), -) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } - -CREATE TABLE tab -( - a Float64, - b String -) Engine = MergeTree() ORDER BY tuple(); - -ALTER TABLE tab ADD STATISTICS a TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } -ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -ALTER TABLE tab ADD STATISTICS IF NOT EXISTS a TYPE tdigest; -ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MODIFY STATISTICS a TYPE tdigest; --- Statistics can be created only on integer columns -ALTER TABLE tab ADD STATISTICS b TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS b; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS a; -ALTER TABLE tab DROP STATISTICS IF EXISTS a; -ALTER TABLE tab CLEAR STATISTICS a; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab CLEAR STATISTICS IF EXISTS a; -ALTER TABLE tab MATERIALIZE STATISTICS b; -- { serverError ILLEGAL_STATISTICS } - -ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -ALTER TABLE tab MODIFY COLUMN a Float64 TTL toDateTime(b) + INTERVAL 1 MONTH; -ALTER TABLE tab MODIFY COLUMN a Int64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } - -DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference deleted file mode 100644 index 5e969cf41cb..00000000000 --- a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference +++ /dev/null @@ -1,10 +0,0 @@ -10 -10 -10 -statistics not used Condition less(b, 10_UInt8) moved to PREWHERE -statistics not used Condition less(a, 10_UInt8) moved to PREWHERE -statistics used after merge Condition less(a, 10_UInt8) moved to PREWHERE -statistics used after merge Condition less(b, 10_UInt8) moved to PREWHERE -statistics used after materialize Condition less(a, 10_UInt8) moved to PREWHERE -statistics used after materialize Condition less(b, 10_UInt8) moved to PREWHERE -2 0 diff --git a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql deleted file mode 100644 index 6606cff263f..00000000000 --- a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql +++ /dev/null @@ -1,52 +0,0 @@ --- Tests delayed materialization of statistics in merge instead of during insert (setting 'materialize_statistics_on_insert = 0'). - -DROP TABLE IF EXISTS tab; - -SET enable_analyzer = 1; -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; - -SET materialize_statistics_on_insert = 0; - -CREATE TABLE tab -( - a Int64 STATISTICS(tdigest), - b Int16 STATISTICS(tdigest), -) ENGINE = MergeTree() ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics. - -INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; - -SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics not used'; - -OPTIMIZE TABLE tab FINAL; - -SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after merge'; - -TRUNCATE TABLE tab; -SET mutations_sync = 2; - -INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; -ALTER TABLE tab MATERIALIZE STATISTICS a, b; - -SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after materialize'; - -DROP TABLE tab; - -SYSTEM FLUSH LOGS; - -SELECT log_comment, message FROM system.text_log JOIN -( - SELECT Settings['log_comment'] AS log_comment, query_id FROM system.query_log - WHERE current_database = currentDatabase() - AND query LIKE 'SELECT count(*) FROM tab%' - AND type = 'QueryFinish' -) AS query_log USING (query_id) -WHERE message LIKE '%moved to PREWHERE%' -ORDER BY event_time_microseconds; - -SELECT count(), sum(ProfileEvents['MergeTreeDataWriterStatisticsCalculationMicroseconds']) -FROM system.query_log -WHERE current_database = currentDatabase() - AND query LIKE 'INSERT INTO tab SELECT%' - AND type = 'QueryFinish'; diff --git a/tests/queries/0_stateless/02864_statistics_predicates.reference b/tests/queries/0_stateless/02864_statistics_predicates.reference new file mode 100644 index 00000000000..ffbd7269e05 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_predicates.reference @@ -0,0 +1,98 @@ +u64 and = +10 +10 +10 +10 +0 +0 +0 +0 +10 +10 +10 +10 +u64 and < +70 +70 +70 +70 +80 +80 +80 +80 +70 +70 +70 +70 +f64 and = +10 +10 +10 +10 +0 +0 +0 +0 +10 +10 +10 +10 +0 +0 +0 +0 +f64 and < +70 +70 +70 +70 +80 +80 +80 +80 +70 +70 +70 +70 +80 +80 +80 +80 +dt and = +0 +0 +0 +0 +10 +10 +10 +10 +dt and < +10000 +10000 +10000 +10000 +70 +70 +70 +70 +b and = +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +0 +0 +0 +0 +s and = +10 +10 diff --git a/tests/queries/0_stateless/02864_statistics_predicates.sql b/tests/queries/0_stateless/02864_statistics_predicates.sql new file mode 100644 index 00000000000..779116cf19a --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_predicates.sql @@ -0,0 +1,214 @@ +-- Tags: no-fasttest +-- no-fasttest: 'count_min' sketches need a 3rd party library + +-- Tests the cross product of all predicates with all right-hand sides on all data types and all statistics types. + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab +( + u64 UInt64, + u64_tdigest UInt64 STATISTICS(tdigest), + u64_count_min UInt64 STATISTICS(count_min), + u64_uniq UInt64 STATISTICS(uniq), + f64 Float64, + f64_tdigest Float64 STATISTICS(tdigest), + f64_count_min Float64 STATISTICS(count_min), + f64_uniq Float64 STATISTICS(uniq), + dt DateTime, + dt_tdigest DateTime STATISTICS(tdigest), + dt_count_min DateTime STATISTICS(count_min), + dt_uniq DateTime STATISTICS(uniq), + b Bool, + b_tdigest Bool STATISTICS(tdigest), + b_count_min Bool STATISTICS(count_min), + b_uniq Bool STATISTICS(uniq), + s String, + -- s_tdigest String STATISTICS(tdigest), -- not supported by tdigest + s_count_min String STATISTICS(count_min) + -- s_uniq String STATISTICS(uniq), -- not supported by uniq +) Engine = MergeTree() ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO tab +-- SELECT number % 10000, number % 1000, -(number % 100) FROM system.numbers LIMIT 10000; +SELECT number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 2, + number % 2, + number % 2, + number % 2, + toString(number % 1000), + toString(number % 1000) +FROM system.numbers LIMIT 10000; + +-- u64 ---------------------------------------------------- + +SELECT 'u64 and ='; + +SELECT count(*) FROM tab WHERE u64 = 7; +SELECT count(*) FROM tab WHERE u64_tdigest = 7; +SELECT count(*) FROM tab WHERE u64_count_min = 7; +SELECT count(*) FROM tab WHERE u64_uniq = 7; + +SELECT count(*) FROM tab WHERE u64 = 7.7; +SELECT count(*) FROM tab WHERE u64_tdigest = 7.7; +SELECT count(*) FROM tab WHERE u64_count_min = 7.7; +SELECT count(*) FROM tab WHERE u64_uniq = 7.7; + +SELECT count(*) FROM tab WHERE u64 = '7'; +SELECT count(*) FROM tab WHERE u64_tdigest = '7'; +SELECT count(*) FROM tab WHERE u64_count_min = '7'; +SELECT count(*) FROM tab WHERE u64_uniq = '7'; + +SELECT count(*) FROM tab WHERE u64 = '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_tdigest = '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_count_min = '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_uniq = '7.7'; -- { serverError TYPE_MISMATCH } + +SELECT 'u64 and <'; + +SELECT count(*) FROM tab WHERE u64 < 7; +SELECT count(*) FROM tab WHERE u64_tdigest < 7; +SELECT count(*) FROM tab WHERE u64_count_min < 7; +SELECT count(*) FROM tab WHERE u64_uniq < 7; + +SELECT count(*) FROM tab WHERE u64 < 7.7; +SELECT count(*) FROM tab WHERE u64_tdigest < 7.7; +SELECT count(*) FROM tab WHERE u64_count_min < 7.7; +SELECT count(*) FROM tab WHERE u64_uniq < 7.7; + +SELECT count(*) FROM tab WHERE u64 < '7'; +SELECT count(*) FROM tab WHERE u64_tdigest < '7'; +SELECT count(*) FROM tab WHERE u64_count_min < '7'; +SELECT count(*) FROM tab WHERE u64_uniq < '7'; + +SELECT count(*) FROM tab WHERE u64 < '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_tdigest < '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_count_min < '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_uniq < '7.7'; -- { serverError TYPE_MISMATCH } + +-- f64 ---------------------------------------------------- + +SELECT 'f64 and ='; + +SELECT count(*) FROM tab WHERE f64 = 7; +SELECT count(*) FROM tab WHERE f64_tdigest = 7; +SELECT count(*) FROM tab WHERE f64_count_min = 7; +SELECT count(*) FROM tab WHERE f64_uniq = 7; + +SELECT count(*) FROM tab WHERE f64 = 7.7; +SELECT count(*) FROM tab WHERE f64_tdigest = 7.7; +SELECT count(*) FROM tab WHERE f64_count_min = 7.7; +SELECT count(*) FROM tab WHERE f64_uniq = 7.7; + +SELECT count(*) FROM tab WHERE f64 = '7'; +SELECT count(*) FROM tab WHERE f64_tdigest = '7'; +SELECT count(*) FROM tab WHERE f64_count_min = '7'; +SELECT count(*) FROM tab WHERE f64_uniq = '7'; + +SELECT count(*) FROM tab WHERE f64 = '7.7'; +SELECT count(*) FROM tab WHERE f64_tdigest = '7.7'; +SELECT count(*) FROM tab WHERE f64_count_min = '7.7'; +SELECT count(*) FROM tab WHERE f64_uniq = '7.7'; + +SELECT 'f64 and <'; + +SELECT count(*) FROM tab WHERE f64 < 7; +SELECT count(*) FROM tab WHERE f64_tdigest < 7; +SELECT count(*) FROM tab WHERE f64_count_min < 7; +SELECT count(*) FROM tab WHERE f64_uniq < 7; + +SELECT count(*) FROM tab WHERE f64 < 7.7; +SELECT count(*) FROM tab WHERE f64_tdigest < 7.7; +SELECT count(*) FROM tab WHERE f64_count_min < 7.7; +SELECT count(*) FROM tab WHERE f64_uniq < 7.7; + +SELECT count(*) FROM tab WHERE f64 < '7'; +SELECT count(*) FROM tab WHERE f64_tdigest < '7'; +SELECT count(*) FROM tab WHERE f64_count_min < '7'; +SELECT count(*) FROM tab WHERE f64_uniq < '7'; + +SELECT count(*) FROM tab WHERE f64 < '7.7'; +SELECT count(*) FROM tab WHERE f64_tdigest < '7.7'; +SELECT count(*) FROM tab WHERE f64_count_min < '7.7'; +SELECT count(*) FROM tab WHERE f64_uniq < '7.7'; + +-- dt ---------------------------------------------------- + +SELECT 'dt and ='; + +SELECT count(*) FROM tab WHERE dt = '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_tdigest = '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_count_min = '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_uniq = '2024-08-08 11:12:13'; + +SELECT count(*) FROM tab WHERE dt = 7; +SELECT count(*) FROM tab WHERE dt_tdigest = 7; +SELECT count(*) FROM tab WHERE dt_count_min = 7; +SELECT count(*) FROM tab WHERE dt_uniq = 7; + +SELECT 'dt and <'; + +SELECT count(*) FROM tab WHERE dt < '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_tdigest < '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_count_min < '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_uniq < '2024-08-08 11:12:13'; + +SELECT count(*) FROM tab WHERE dt < 7; +SELECT count(*) FROM tab WHERE dt_tdigest < 7; +SELECT count(*) FROM tab WHERE dt_count_min < 7; +SELECT count(*) FROM tab WHERE dt_uniq < 7; + +-- b ---------------------------------------------------- + +SELECT 'b and ='; + +SELECT count(*) FROM tab WHERE b = true; +SELECT count(*) FROM tab WHERE b_tdigest = true; +SELECT count(*) FROM tab WHERE b_count_min = true; +SELECT count(*) FROM tab WHERE b_uniq = true; + +SELECT count(*) FROM tab WHERE b = 'true'; +SELECT count(*) FROM tab WHERE b_tdigest = 'true'; +SELECT count(*) FROM tab WHERE b_count_min = 'true'; +SELECT count(*) FROM tab WHERE b_uniq = 'true'; + +SELECT count(*) FROM tab WHERE b = 1; +SELECT count(*) FROM tab WHERE b_tdigest = 1; +SELECT count(*) FROM tab WHERE b_count_min = 1; +SELECT count(*) FROM tab WHERE b_uniq = 1; + +SELECT count(*) FROM tab WHERE b = 1.1; +SELECT count(*) FROM tab WHERE b_tdigest = 1.1; +SELECT count(*) FROM tab WHERE b_count_min = 1.1; +SELECT count(*) FROM tab WHERE b_uniq = 1.1; + +-- s ---------------------------------------------------- + +SELECT 's and ='; + +SELECT count(*) FROM tab WHERE s = 7; -- { serverError NO_COMMON_TYPE } +-- SELECT count(*) FROM tab WHERE s_tdigest = 7; -- not supported +SELECT count(*) FROM tab WHERE s_count_min = 7; -- { serverError NO_COMMON_TYPE } +-- SELECT count(*) FROM tab WHERE s_uniq = 7; -- not supported + +SELECT count(*) FROM tab WHERE s = '7'; +-- SELECT count(*) FROM tab WHERE s_tdigest = '7'; -- not supported +SELECT count(*) FROM tab WHERE s_count_min = '7'; +-- SELECT count(*) FROM tab WHERE s_uniq = '7'; -- not supported + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_uniq.reference b/tests/queries/0_stateless/02864_statistics_uniq.reference deleted file mode 100644 index 77786dbdd8c..00000000000 --- a/tests/queries/0_stateless/02864_statistics_uniq.reference +++ /dev/null @@ -1,35 +0,0 @@ -CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest, uniq),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After insert - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) -After merge - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) -After modify TDigest - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(c, -1), less(a, 10), less(b, 10)) (removed) -After drop - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(c, -1), less(b, 10)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_uniq.sql b/tests/queries/0_stateless/02864_statistics_uniq.sql deleted file mode 100644 index 0f5f353c045..00000000000 --- a/tests/queries/0_stateless/02864_statistics_uniq.sql +++ /dev/null @@ -1,73 +0,0 @@ -DROP TABLE IF EXISTS t1; - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; -SET mutations_sync = 1; - -CREATE TABLE t1 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c Int64 STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; - -SHOW CREATE TABLE t1; - -INSERT INTO t1 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; -INSERT INTO t1 select 0, 0, 11, generateUUIDv4(); - -SELECT 'After insert'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -OPTIMIZE TABLE t1 FINAL; - -SELECT 'After merge'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -SELECT 'After modify TDigest'; -ALTER TABLE t1 MODIFY STATISTICS c TYPE TDigest; -ALTER TABLE t1 MATERIALIZE STATISTICS c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - - -ALTER TABLE t1 DROP STATISTICS c; - -SELECT 'After drop'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -DROP TABLE IF EXISTS t1; -DROP TABLE IF EXISTS t2; -SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE t2 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c LowCardinality(Int64) STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO t2 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; - -DROP TABLE IF EXISTS t2; -DROP TABLE IF EXISTS t3; - -CREATE TABLE t3 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c Nullable(Int64) STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO t3 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; - -DROP TABLE IF EXISTS t3; - diff --git a/tests/queries/0_stateless/02864_statistics_usage.reference b/tests/queries/0_stateless/02864_statistics_usage.reference new file mode 100644 index 00000000000..a9f669b88c1 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_usage.reference @@ -0,0 +1,20 @@ +After insert + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After drop statistic + Prewhere info + Prewhere filter + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) +After add and materialize statistic + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After merge + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After rename + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(c, 10_UInt8)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_usage.sql b/tests/queries/0_stateless/02864_statistics_usage.sql new file mode 100644 index 00000000000..4956bd27e87 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_usage.sql @@ -0,0 +1,42 @@ +-- Test that the optimizer picks up column statistics +-- (The concrete statistics type, column data type and predicate type don't matter) + +-- Checks by the predicate evaluation order in EXPLAIN. This is quite fragile, a better approach would be helpful (maybe 'send_logs_level'?) + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET mutations_sync = 1; +SET enable_analyzer = 1; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest) +) Engine = MergeTree() ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO tab select number, -number FROM system.numbers LIMIT 10000; +SELECT 'After insert'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +ALTER TABLE tab DROP STATISTICS a, b; +SELECT 'After drop statistic'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks b first, then a (statistics not used) + +ALTER TABLE tab ADD STATISTICS a, b TYPE tdigest; +ALTER TABLE tab MATERIALIZE STATISTICS a, b; +INSERT INTO tab select number, -number FROM system.numbers LIMIT 10000; +SELECT 'After add and materialize statistic'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +OPTIMIZE TABLE tab FINAL; +SELECT 'After merge'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +ALTER TABLE tab RENAME COLUMN b TO c; +SELECT 'After rename'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then c (statistics used) + +DROP TABLE IF EXISTS tab; From ee433684ddc4614a5bb93dbfd1e3b481a2f343d6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Aug 2024 10:55:16 +0000 Subject: [PATCH 565/644] Follow-up to ClickHouse#63898, pt. III --- .../02421_simple_queries_for_opentelemetry.sh | 10 +++---- .../02458_insert_select_progress_tcp.sh | 2 +- .../02476_analyzer_identifier_hints.sh | 8 +++--- .../0_stateless/02480_tets_show_full.sh | 2 +- .../0_stateless/02482_load_parts_refcounts.sh | 2 +- .../02496_remove_redundant_sorting.sh | 6 ++--- .../02497_storage_file_reader_selection.sh | 2 +- .../02500_remove_redundant_distinct.sh | 6 ++--- ...atabase_replicated_no_arguments_for_rmt.sh | 4 +-- .../0_stateless/02532_send_logs_level_test.sh | 2 +- .../02555_davengers_rename_chain.sh | 8 +++--- ...02572_query_views_log_background_thread.sh | 8 +++--- .../02703_max_local_read_bandwidth.sh | 4 +-- .../02703_max_local_write_bandwidth.sh | 4 +-- .../0_stateless/02704_max_backup_bandwidth.sh | 4 +-- .../0_stateless/02724_limit_num_mutations.sh | 8 +++--- .../02725_async_insert_table_setting.sh | 4 +-- .../queries/0_stateless/02841_local_assert.sh | 6 ++--- ...rge_across_partitions_final_with_lonely.sh | 6 ++--- .../02871_clickhouse_client_restart_pager.sh | 2 +- .../02875_clickhouse_local_multiquery.sh | 4 +-- .../02875_merge_engine_set_index.sh | 4 +-- ...tion_table_with_explicit_insert_columns.sh | 2 +- ...ture_from_insertion_table_with_defaults.sh | 2 +- .../02883_named_collections_override.sh | 2 +- .../02884_async_insert_native_protocol_1.sh | 4 +-- .../02884_async_insert_native_protocol_2.sh | 4 +-- .../02884_async_insert_native_protocol_3.sh | 4 +-- .../02884_async_insert_native_protocol_4.sh | 4 +-- .../02885_ephemeral_columns_from_file.sh | 2 +- .../0_stateless/02895_npy_output_format.sh | 2 +- ...ak_memory_usage_http_headers_regression.sh | 2 +- .../02903_empty_order_by_throws_error.sh | 4 +-- .../02903_rmt_retriable_merge_exception.sh | 6 ++--- ...904_empty_order_by_with_setting_enabled.sh | 8 +++--- .../02907_backup_mv_with_no_inner_table.sh | 8 +++--- .../02907_backup_mv_with_no_source_table.sh | 12 ++++----- .../02907_backup_restore_default_nullable.sh | 4 +-- .../02907_backup_restore_flatten_nested.sh | 8 +++--- .../02907_clickhouse_dictionary_bug.sh | 2 +- .../02907_system_backups_profile_events.sh | 6 ++--- .../0_stateless/02908_Npy_files_caching.sh | 4 +-- .../0_stateless/02908_table_ttl_dependency.sh | 6 ++--- .../02909_settings_in_json_schema_cache.sh | 2 +- .../02915_input_table_function_in_subquery.sh | 2 +- .../02915_lazy_loading_of_base_backups.sh | 26 +++++++++---------- .../0_stateless/02916_dictionary_access.sh | 8 +++--- .../0_stateless/02916_joinget_dependency.sh | 6 ++--- .../02930_client_file_log_comment.sh | 2 +- ...lumn_use_structure_from_insertion_table.sh | 2 +- .../02940_system_stacktrace_optimizations.sh | 6 ++--- ..._alter_metadata_merge_checksum_mismatch.sh | 8 +++--- .../02947_merge_tree_index_table_3.sh | 4 +-- ...2950_dictionary_ssd_cache_short_circuit.sh | 2 +- .../02950_distributed_initial_query_event.sh | 2 +- .../02974_backup_query_format_null.sh | 4 +-- ...ert_select_resize_to_max_insert_threads.sh | 2 +- .../03008_deduplication_random_setttings.sh | 6 ++--- .../03008_local_plain_rewritable.sh | 14 +++++----- .../03031_clickhouse_local_input.sh | 8 +++--- .../0_stateless/03032_async_backup_restore.sh | 4 +-- .../03096_http_interface_role_query_param.sh | 4 +-- ...03140_client_subsequent_external_tables.sh | 2 +- .../03143_prewhere_profile_events.sh | 12 ++++----- .../03145_non_loaded_projection_backup.sh | 12 ++++----- .../03155_test_move_to_prewhere.sh | 4 +-- .../03156_default_multiquery_split.sh | 2 +- .../0_stateless/03169_time_virtual_column.sh | 2 +- .../03173_parallel_replicas_join_bug.sh | 2 +- .../03198_settings_in_csv_tsv_schema_cache.sh | 8 +++--- .../03198_unload_primary_key_outdated.sh | 4 +-- .../03199_dictionary_table_access.sh | 8 +++--- ...s_to_read_for_schema_inference_in_cache.sh | 2 +- 73 files changed, 186 insertions(+), 186 deletions(-) diff --git a/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh b/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh index 98b571c5968..91e85eabcb8 100755 --- a/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh +++ b/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # $2 - query function execute_query() { - ${CLICKHOUSE_CLIENT} --opentelemetry_start_trace_probability=1 --query_id $1 -nq " + ${CLICKHOUSE_CLIENT} --opentelemetry_start_trace_probability=1 --query_id $1 -q " ${2} " } @@ -18,7 +18,7 @@ function execute_query() # so we only to check the db.statement only function check_query_span_query_only() { -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} -q " SYSTEM FLUSH LOGS; SELECT attribute['db.statement'] as query FROM system.opentelemetry_span_log @@ -31,7 +31,7 @@ ${CLICKHOUSE_CLIENT} -nq " function check_query_span() { -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} -q " SYSTEM FLUSH LOGS; SELECT attribute['db.statement'] as query, attribute['clickhouse.read_rows'] as read_rows, @@ -47,7 +47,7 @@ ${CLICKHOUSE_CLIENT} -nq " # # Set up # -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} -q " DROP TABLE IF EXISTS ${CLICKHOUSE_DATABASE}.opentelemetry_test; CREATE TABLE ${CLICKHOUSE_DATABASE}.opentelemetry_test (id UInt64) Engine=MergeTree Order By id; " @@ -79,4 +79,4 @@ check_query_span $query_id # ${CLICKHOUSE_CLIENT} -q " DROP TABLE IF EXISTS ${CLICKHOUSE_DATABASE}.opentelemetry_test; -" \ No newline at end of file +" diff --git a/tests/queries/0_stateless/02458_insert_select_progress_tcp.sh b/tests/queries/0_stateless/02458_insert_select_progress_tcp.sh index ae3ea017fbb..178da822d41 100755 --- a/tests/queries/0_stateless/02458_insert_select_progress_tcp.sh +++ b/tests/queries/0_stateless/02458_insert_select_progress_tcp.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists insert_select_progress_tcp; create table insert_select_progress_tcp(s UInt16) engine = MergeTree order by s; " diff --git a/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh b/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh index 4c850a6ec9e..92f519a9f8a 100755 --- a/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh +++ b/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS test_table; CREATE TABLE test_table ( @@ -74,7 +74,7 @@ $CLICKHOUSE_CLIENT -q "SELECT 1 AS constant_value, arrayMap(lambda_argument -> l $CLICKHOUSE_CLIENT -q "WITH 1 AS constant_value SELECT (SELECT constant_valu) SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['constant_value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS test_table_compound; CREATE TABLE test_table_compound ( @@ -142,7 +142,7 @@ $CLICKHOUSE_CLIENT -q "SELECT cast(tuple(1), 'Tuple(value_1 String)') AS constan $CLICKHOUSE_CLIENT -q "WITH cast(tuple(1), 'Tuple(value_1 String)') AS constant_value SELECT (SELECT constant_value.value_) SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['constant_value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS test_table_1; CREATE TABLE test_table_1 ( @@ -185,7 +185,7 @@ $CLICKHOUSE_CLIENT -q "SELECT ((1))::Tuple(a Tuple(b UInt32)) AS t, t.a.c SETTIN $CLICKHOUSE_CLIENT -q "SELECT 1"; -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE test_table; DROP TABLE test_table_compound; DROP TABLE test_table_1; diff --git a/tests/queries/0_stateless/02480_tets_show_full.sh b/tests/queries/0_stateless/02480_tets_show_full.sh index 5f5040ba128..50184857a1f 100755 --- a/tests/queries/0_stateless/02480_tets_show_full.sh +++ b/tests/queries/0_stateless/02480_tets_show_full.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) database=$($CLICKHOUSE_CLIENT -q 'SELECT currentDatabase()') -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS test_02480_table; DROP VIEW IF EXISTS test_02480_view; CREATE TABLE test_02480_table (id Int64) ENGINE=MergeTree ORDER BY id; diff --git a/tests/queries/0_stateless/02482_load_parts_refcounts.sh b/tests/queries/0_stateless/02482_load_parts_refcounts.sh index 5303824d97c..4dc7a7fd99b 100755 --- a/tests/queries/0_stateless/02482_load_parts_refcounts.sh +++ b/tests/queries/0_stateless/02482_load_parts_refcounts.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE IF EXISTS load_parts_refcounts SYNC; CREATE TABLE load_parts_refcounts (id UInt32) diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh index c9bd242e429..6e132c55628 100755 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh @@ -26,15 +26,15 @@ FROM ORDER BY number DESC ) ORDER BY number ASC" -$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;EXPLAIN $query" +$CLICKHOUSE_CLIENT -q "$DISABLE_OPTIMIZATION;EXPLAIN $query" function run_query { echo "-- query" echo "$1" echo "-- explain" - $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;EXPLAIN $1" + $CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;EXPLAIN $1" echo "-- execute" - $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;$1" + $CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;$1" } echo "-- Enabled query_plan_remove_redundant_sorting" diff --git a/tests/queries/0_stateless/02497_storage_file_reader_selection.sh b/tests/queries/0_stateless/02497_storage_file_reader_selection.sh index aa43e81f131..27243dd47fa 100755 --- a/tests/queries/0_stateless/02497_storage_file_reader_selection.sh +++ b/tests/queries/0_stateless/02497_storage_file_reader_selection.sh @@ -13,6 +13,6 @@ $CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SEL $CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferMMap" && echo 'Fail' || echo 0 $CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferOrdinary" && echo 1 || echo 'Fail' -$CLICKHOUSE_CLIENT --storage_file_read_method=mmap -nq "SELECT * FROM file('/dev/null', 'LineAsString') FORMAT Null -- { serverError BAD_ARGUMENTS }" +$CLICKHOUSE_CLIENT --storage_file_read_method=mmap -q "SELECT * FROM file('/dev/null', 'LineAsString') FORMAT Null -- { serverError BAD_ARGUMENTS }" rm $DATA_FILE diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct.sh b/tests/queries/0_stateless/02500_remove_redundant_distinct.sh index 3c06119e8d2..6fd42fa940a 100755 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct.sh +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct.sh @@ -24,15 +24,15 @@ FROM ) )" -$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;EXPLAIN $query" +$CLICKHOUSE_CLIENT -q "$DISABLE_OPTIMIZATION;EXPLAIN $query" function run_query { echo "-- query" echo "$1" echo "-- explain" - $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;EXPLAIN $1" + $CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;EXPLAIN $1" echo "-- execute" - $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;$1" + $CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;$1" } echo "-- Enabled $OPTIMIZATION_SETTING" diff --git a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh index a0f228e6af4..c1aa24943c1 100755 --- a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh +++ b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh @@ -14,8 +14,8 @@ ${CLICKHOUSE_CLIENT} -q "CREATE USER user_${CLICKHOUSE_DATABASE} settings databa ${CLICKHOUSE_CLIENT} -q "GRANT CREATE TABLE ON ${CLICKHOUSE_DATABASE}_db.* TO user_${CLICKHOUSE_DATABASE}" ${CLICKHOUSE_CLIENT} -q "GRANT TABLE ENGINE ON ReplicatedMergeTree TO user_${CLICKHOUSE_DATABASE}" ${CLICKHOUSE_CLIENT} -q "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_ok (x UInt32) engine = ReplicatedMergeTree order by x;" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_fail (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/root/{shard}', '{replica}') order by x; -- { serverError 80 }" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_ok (x UInt32) engine = ReplicatedMergeTree order by x;" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_fail (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/root/{shard}', '{replica}') order by x; -- { serverError 80 }" ${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db" ${CLICKHOUSE_CLIENT} -q "DROP USER user_${CLICKHOUSE_DATABASE}" diff --git a/tests/queries/0_stateless/02532_send_logs_level_test.sh b/tests/queries/0_stateless/02532_send_logs_level_test.sh index 506ac2331f2..a50539311cb 100755 --- a/tests/queries/0_stateless/02532_send_logs_level_test.sh +++ b/tests/queries/0_stateless/02532_send_logs_level_test.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists data; create table data (key Int) engine=MergeTree order by tuple() settings min_bytes_for_wide_part = '1G', compress_marks = 1; insert into data values (1); diff --git a/tests/queries/0_stateless/02555_davengers_rename_chain.sh b/tests/queries/0_stateless/02555_davengers_rename_chain.sh index 660a95846c4..196507dc72e 100755 --- a/tests/queries/0_stateless/02555_davengers_rename_chain.sh +++ b/tests/queries/0_stateless/02555_davengers_rename_chain.sh @@ -46,7 +46,7 @@ tables["wrong_metadata_compact"]="min_bytes_for_wide_part = 10000000" for table in "${!tables[@]}"; do settings="${tables[$table]}" - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS $table; CREATE TABLE $table( @@ -69,7 +69,7 @@ for table in "${!tables[@]}"; do wait_column "$table" "\`a1\` UInt64" || exit 2 - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" -- { echoOn } SELECT 'ECHO_ALIGNMENT_FIX' FORMAT Null; @@ -82,7 +82,7 @@ for table in "${!tables[@]}"; do wait_mutation_loaded "$table" "b1 TO a" || exit 2 - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" -- { echoOn } SELECT 'ECHO_ALIGNMENT_FIX' FORMAT Null; @@ -94,7 +94,7 @@ for table in "${!tables[@]}"; do wait_for_all_mutations "$table" - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" -- { echoOn } SELECT 'ECHO_ALIGNMENT_FIX' FORMAT Null; diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.sh b/tests/queries/0_stateless/02572_query_views_log_background_thread.sh index 509cd03f6c2..22b94e09b58 100755 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.sh +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "drop table if exists buffer_02572; +${CLICKHOUSE_CLIENT} --ignore-error --query "drop table if exists buffer_02572; drop table if exists data_02572; drop table if exists copy_02572; drop table if exists mv_02572;" ${CLICKHOUSE_CLIENT} --query="create table copy_02572 (key Int) engine=Memory();" @@ -21,7 +21,7 @@ ${CLICKHOUSE_CLIENT} --query="insert into buffer_02572 values (1);" if [ $(( $(date +%s) - start )) -gt 6 ]; then # clickhouse test cluster is overloaded, will skip # ensure that the flush was not direct - ${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "select * from data_02572; select * from copy_02572;" + ${CLICKHOUSE_CLIENT} --ignore-error --query "select * from data_02572; select * from copy_02572;" fi # we cannot use OPTIMIZE, this will attach query context, so let's wait @@ -31,11 +31,11 @@ for _ in {1..100}; do done -${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "select * from data_02572; select * from copy_02572;" +${CLICKHOUSE_CLIENT} --ignore-error --query "select * from data_02572; select * from copy_02572;" ${CLICKHOUSE_CLIENT} --query="system flush logs;" ${CLICKHOUSE_CLIENT} --query="select count() > 0, lower(status::String), errorCodeToName(exception_code) from system.query_views_log where view_name = concatWithSeparator('.', currentDatabase(), 'mv_02572') and view_target = concatWithSeparator('.', currentDatabase(), 'copy_02572') - group by 2, 3;" \ No newline at end of file + group by 2, 3;" diff --git a/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh b/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh index 03e0f363d71..79253648475 100755 --- a/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh +++ b/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists data; create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9; " @@ -26,7 +26,7 @@ read_methods=( for read_method in "${read_methods[@]}"; do query_id=$(random_str 10) $CLICKHOUSE_CLIENT --query_id "$query_id" -q "select * from data format Null settings max_local_read_bandwidth='1M', local_filesystem_read_method='$read_method'" - $CLICKHOUSE_CLIENT -nm -q " + $CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT '$read_method', diff --git a/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh b/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh index 4f6a300c5b3..c5776134673 100755 --- a/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh +++ b/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists data; create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9; " @@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -nm -q " query_id=$(random_str 10) # writes 1e6*8 bytes with 1M bandwith it should take (8-1)/1=7 seconds $CLICKHOUSE_CLIENT --query_id "$query_id" -q "insert into data select * from numbers(1e6) settings max_local_write_bandwidth='1M'" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT query_duration_ms >= 7e3, diff --git a/tests/queries/0_stateless/02704_max_backup_bandwidth.sh b/tests/queries/0_stateless/02704_max_backup_bandwidth.sh index 8cb03a93a7a..7e914c4c539 100755 --- a/tests/queries/0_stateless/02704_max_backup_bandwidth.sh +++ b/tests/queries/0_stateless/02704_max_backup_bandwidth.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists data; create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9; " @@ -15,7 +15,7 @@ $CLICKHOUSE_CLIENT -q "insert into data select * from numbers(1e6)" query_id=$(random_str 10) $CLICKHOUSE_CLIENT --query_id "$query_id" -q "backup table data to Disk('backups', '$CLICKHOUSE_DATABASE/data/backup1')" --max_backup_bandwidth=1M > /dev/null -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT query_duration_ms >= 7e3, diff --git a/tests/queries/0_stateless/02724_limit_num_mutations.sh b/tests/queries/0_stateless/02724_limit_num_mutations.sh index 60888db0e2e..604cc9ff08e 100755 --- a/tests/queries/0_stateless/02724_limit_num_mutations.sh +++ b/tests/queries/0_stateless/02724_limit_num_mutations.sh @@ -23,7 +23,7 @@ function wait_for_alter() done } -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " DROP TABLE IF EXISTS t_limit_mutations SYNC; CREATE TABLE t_limit_mutations (id UInt64, v UInt64) @@ -48,14 +48,14 @@ SELECT count() FROM system.mutations WHERE database = currentDatabase() AND tabl SHOW CREATE TABLE t_limit_mutations; " -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " ALTER TABLE t_limit_mutations UPDATE v = 6 WHERE 1 SETTINGS number_of_mutations_to_throw = 100; ALTER TABLE t_limit_mutations MODIFY COLUMN v String SETTINGS number_of_mutations_to_throw = 100, alter_sync = 0; " wait_for_alter "String" -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " SELECT * FROM t_limit_mutations ORDER BY id; SELECT count() FROM system.mutations WHERE database = currentDatabase() AND table = 't_limit_mutations' AND NOT is_done; SHOW CREATE TABLE t_limit_mutations; @@ -65,7 +65,7 @@ ${CLICKHOUSE_CLIENT} --query "SYSTEM START MERGES t_limit_mutations" wait_for_mutation "t_limit_mutations" "0000000003" -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " SELECT * FROM t_limit_mutations ORDER BY id; SELECT count() FROM system.mutations WHERE database = currentDatabase() AND table = 't_limit_mutations' AND NOT is_done; SHOW CREATE TABLE t_limit_mutations; diff --git a/tests/queries/0_stateless/02725_async_insert_table_setting.sh b/tests/queries/0_stateless/02725_async_insert_table_setting.sh index 13911e8d677..14c2d335275 100755 --- a/tests/queries/0_stateless/02725_async_insert_table_setting.sh +++ b/tests/queries/0_stateless/02725_async_insert_table_setting.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " DROP TABLE IF EXISTS t_mt_async_insert; DROP TABLE IF EXISTS t_mt_sync_insert; @@ -19,7 +19,7 @@ url="${CLICKHOUSE_URL}&async_insert=0&wait_for_async_insert=1" ${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO t_mt_async_insert VALUES (1, 'aa'), (2, 'bb')" ${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO t_mt_sync_insert VALUES (1, 'aa'), (2, 'bb')" -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " SELECT count() FROM t_mt_async_insert; SELECT count() FROM t_mt_sync_insert; diff --git a/tests/queries/0_stateless/02841_local_assert.sh b/tests/queries/0_stateless/02841_local_assert.sh index a167c09da1f..dc49007b0f6 100755 --- a/tests/queries/0_stateless/02841_local_assert.sh +++ b/tests/queries/0_stateless/02841_local_assert.sh @@ -7,12 +7,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh echo "create table test (x UInt64) engine=Memory; -insert into test from infile 'data'; -- {clientError BAD_ARGUMENTS}" | $CLICKHOUSE_LOCAL -nm +insert into test from infile 'data'; -- {clientError BAD_ARGUMENTS}" | $CLICKHOUSE_LOCAL -m echo "create table test (x UInt64) engine=Memory; -insert into test from infile 'data';" | $CLICKHOUSE_LOCAL -nm --ignore-error +insert into test from infile 'data';" | $CLICKHOUSE_LOCAL -m --ignore-error echo "create table test (x UInt64) engine=Memory; insert into test from infile 'data'; -- {clientError BAD_ARGUMENTS} -select 1" | $CLICKHOUSE_LOCAL -nm +select 1" | $CLICKHOUSE_LOCAL -m diff --git a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh index 4bc29ce4233..be0ef4e2648 100755 --- a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh +++ b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nm -q """ +${CLICKHOUSE_CLIENT} -m -q """ DROP TABLE IF EXISTS with_lonely; CREATE TABLE with_lonely @@ -23,7 +23,7 @@ ORDER BY (id); """ create_optimize_partition() { - ${CLICKHOUSE_CLIENT} -nm -q """ + ${CLICKHOUSE_CLIENT} -m -q """ INSERT INTO with_lonely SELECT number, '$1', number*10, 0 FROM numbers(10); INSERT INTO with_lonely SELECT number+500000, '$1', number*10, 1 FROM numbers(10); """ @@ -39,7 +39,7 @@ create_optimize_partition "2022-10-29" create_optimize_partition "2022-10-30" create_optimize_partition "2022-10-31" -${CLICKHOUSE_CLIENT} -nm -q """ +${CLICKHOUSE_CLIENT} -m -q """ SYSTEM STOP MERGES with_lonely; INSERT INTO with_lonely SELECT number, '2022-11-01', number*10, 0 FROM numbers(10); diff --git a/tests/queries/0_stateless/02871_clickhouse_client_restart_pager.sh b/tests/queries/0_stateless/02871_clickhouse_client_restart_pager.sh index cc4ce9b122e..418e439e44b 100755 --- a/tests/queries/0_stateless/02871_clickhouse_client_restart_pager.sh +++ b/tests/queries/0_stateless/02871_clickhouse_client_restart_pager.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # head by default print 10 rows, but it is not enough to query 11 rows, since # we need to overflow the default pipe size, hence just 1 million of rows (it # should be around 6 MiB in text representation, should be definitelly enough). -$CLICKHOUSE_CLIENT --ignore-error -nm --pager head -q " +$CLICKHOUSE_CLIENT --ignore-error -m --pager head -q " select * from numbers(1e6); -- { clientError CANNOT_WRITE_TO_FILE_DESCRIPTOR } select * from numbers(1e6); -- { clientError CANNOT_WRITE_TO_FILE_DESCRIPTOR } " diff --git a/tests/queries/0_stateless/02875_clickhouse_local_multiquery.sh b/tests/queries/0_stateless/02875_clickhouse_local_multiquery.sh index 3f2b732e71b..3a7d861262e 100755 --- a/tests/queries/0_stateless/02875_clickhouse_local_multiquery.sh +++ b/tests/queries/0_stateless/02875_clickhouse_local_multiquery.sh @@ -9,7 +9,7 @@ $CLICKHOUSE_CLIENT -q "select 1; select 2;" $CLICKHOUSE_LOCAL -q "select 1; select 2;" # -n is a no-op -$CLICKHOUSE_CLIENT -n -q "select 1; select 2;" -$CLICKHOUSE_LOCAL -n -q "select 1; select 2;" +$CLICKHOUSE_CLIENT -q "select 1; select 2;" +$CLICKHOUSE_LOCAL -q "select 1; select 2;" exit 0 diff --git a/tests/queries/0_stateless/02875_merge_engine_set_index.sh b/tests/queries/0_stateless/02875_merge_engine_set_index.sh index 355d83167a6..f40696c31a9 100755 --- a/tests/queries/0_stateless/02875_merge_engine_set_index.sh +++ b/tests/queries/0_stateless/02875_merge_engine_set_index.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nq " +$CLICKHOUSE_CLIENT -q " CREATE TABLE t1 ( a UInt32, @@ -57,7 +57,7 @@ ORDER BY b DESC FORMAT Null;" -$CLICKHOUSE_CLIENT -nq " +$CLICKHOUSE_CLIENT -q " SYSTEM FLUSH LOGS; SELECT ProfileEvents['SelectedMarks'] diff --git a/tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.sh b/tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.sh index 8bdaa47c111..dd08724456b 100755 --- a/tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.sh +++ b/tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_LOCAL -q "select 42 as x format Native" > $CLICKHOUSE_TEST_UNIQUE_NAME.native -$CLICKHOUSE_LOCAL -n -q " +$CLICKHOUSE_LOCAL -q " create table test (x UInt64, y UInt64) engine=Memory; insert into test (x) select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.native'); insert into test (y) select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.native'); diff --git a/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh b/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh index 315bbcd544f..c7270b65e19 100755 --- a/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh +++ b/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_LOCAL -q "select 1 as x format Native" > $CLICKHOUSE_TEST_UNIQUE_NAME.native -$CLICKHOUSE_LOCAL -n -q " +$CLICKHOUSE_LOCAL -q " create table test (x UInt64, y UInt64 default 42) engine=Memory; insert into test select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.native'); select * from test; diff --git a/tests/queries/0_stateless/02883_named_collections_override.sh b/tests/queries/0_stateless/02883_named_collections_override.sh index a08c795127d..915ce280226 100755 --- a/tests/queries/0_stateless/02883_named_collections_override.sh +++ b/tests/queries/0_stateless/02883_named_collections_override.sh @@ -8,7 +8,7 @@ u1="${CLICKHOUSE_TEST_UNIQUE_NAME}_collection1" u2="${CLICKHOUSE_TEST_UNIQUE_NAME}_collection2" u3="${CLICKHOUSE_TEST_UNIQUE_NAME}_collection3" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP NAMED COLLECTION IF EXISTS $u1; DROP NAMED COLLECTION IF EXISTS $u2; diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_1.sh b/tests/queries/0_stateless/02884_async_insert_native_protocol_1.sh index 7f583087336..791515c82d6 100755 --- a/tests/queries/0_stateless/02884_async_insert_native_protocol_1.sh +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_1.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS t_async_insert_native_1; CREATE TABLE t_async_insert_native_1 (id UInt64, s String) ENGINE = MergeTree ORDER BY id; " @@ -22,7 +22,7 @@ echo '{"id": 1, "s": "aaa"}' \ | $CLICKHOUSE_CLIENT $async_insert_options -q 'INSERT INTO t_async_insert_native_1 FORMAT JSONEachRow {"id": 2, "s": "bbb"}' 2>&1 \ | grep -o "NOT_IMPLEMENTED" -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " SELECT sum(length(entries.bytes)) FROM system.asynchronous_inserts WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_async_insert_native_1'; diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_2.sh b/tests/queries/0_stateless/02884_async_insert_native_protocol_2.sh index b9b1854eaef..a8a9209ee68 100755 --- a/tests/queries/0_stateless/02884_async_insert_native_protocol_2.sh +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_2.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS t_async_insert_native_2; CREATE TABLE t_async_insert_native_2 (id UInt64, s String) ENGINE = MergeTree ORDER BY id; " @@ -18,7 +18,7 @@ echo "(3, 'ccc') (4, 'ddd') (5, 'eee')" | $CLICKHOUSE_CLIENT $async_insert_optio wait -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " SELECT * FROM t_async_insert_native_2 ORDER BY id; SYSTEM FLUSH LOGS; diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_3.sh b/tests/queries/0_stateless/02884_async_insert_native_protocol_3.sh index c9d399607d0..229f13eb821 100755 --- a/tests/queries/0_stateless/02884_async_insert_native_protocol_3.sh +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_3.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS t_async_insert_native_3; CREATE TABLE t_async_insert_native_3 (id UInt64, s String) ENGINE = MergeTree ORDER BY id; " @@ -21,7 +21,7 @@ $CLICKHOUSE_CLIENT $async_insert_options -q "INSERT INTO t_async_insert_native_3 wait -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " SELECT format, length(entries.bytes) FROM system.asynchronous_inserts WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_async_insert_native_3' ORDER BY format; diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_4.sh b/tests/queries/0_stateless/02884_async_insert_native_protocol_4.sh index 9118c11315c..e84c1ca8899 100755 --- a/tests/queries/0_stateless/02884_async_insert_native_protocol_4.sh +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_4.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS t_async_insert_native_4; CREATE TABLE t_async_insert_native_4 (id UInt64) ENGINE = MergeTree ORDER BY id; " @@ -20,7 +20,7 @@ echo "(2) (3) (4) (5)" | $CLICKHOUSE_CLIENT_WITH_LOG $async_insert_options --asy -q 'INSERT INTO t_async_insert_native_4 FORMAT Values' 2>&1 \ | grep -c "too much data" -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " SELECT * FROM t_async_insert_native_4 ORDER BY id; SYSTEM FLUSH LOGS; diff --git a/tests/queries/0_stateless/02885_ephemeral_columns_from_file.sh b/tests/queries/0_stateless/02885_ephemeral_columns_from_file.sh index 2917ec86957..065658d4d56 100755 --- a/tests/queries/0_stateless/02885_ephemeral_columns_from_file.sh +++ b/tests/queries/0_stateless/02885_ephemeral_columns_from_file.sh @@ -9,7 +9,7 @@ $CLICKHOUSE_LOCAL -q "select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.jsonl', a $CLICKHOUSE_LOCAL -q "select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.jsonl', auto, 'x UInt64 Alias y, y UInt64')" 2>&1 | grep -c "BAD_ARGUMENTS" $CLICKHOUSE_LOCAL -q "select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.jsonl', auto, 'x UInt64 Materialized 42, y UInt64')" 2>&1 | grep -c "BAD_ARGUMENTS" -$CLICKHOUSE_LOCAL -n -q " +$CLICKHOUSE_LOCAL -q " create table test (x UInt64 Ephemeral, y UInt64 default x + 1) engine=Memory; insert into test (x, y) select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.jsonl'); select * from test; diff --git a/tests/queries/0_stateless/02895_npy_output_format.sh b/tests/queries/0_stateless/02895_npy_output_format.sh index a364e447062..74000bc298f 100755 --- a/tests/queries/0_stateless/02895_npy_output_format.sh +++ b/tests/queries/0_stateless/02895_npy_output_format.sh @@ -9,7 +9,7 @@ mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* chmod 777 ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -${CLICKHOUSE_CLIENT} -n -q --ignore-error " +${CLICKHOUSE_CLIENT} -q --ignore-error " DROP DATABASE IF EXISTS npy_output_02895; CREATE DATABASE IF NOT EXISTS npy_output_02895; diff --git a/tests/queries/0_stateless/02895_peak_memory_usage_http_headers_regression.sh b/tests/queries/0_stateless/02895_peak_memory_usage_http_headers_regression.sh index d6775927f35..b4656c9e321 100755 --- a/tests/queries/0_stateless/02895_peak_memory_usage_http_headers_regression.sh +++ b/tests/queries/0_stateless/02895_peak_memory_usage_http_headers_regression.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS data; DROP TABLE IF EXISTS data2; DROP VIEW IF EXISTS mv1; diff --git a/tests/queries/0_stateless/02903_empty_order_by_throws_error.sh b/tests/queries/0_stateless/02903_empty_order_by_throws_error.sh index 64f5dd1a987..ef631d9ed1b 100755 --- a/tests/queries/0_stateless/02903_empty_order_by_throws_error.sh +++ b/tests/queries/0_stateless/02903_empty_order_by_throws_error.sh @@ -5,13 +5,13 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh # setting disabled and no order by or primary key; expect error -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS test_empty_order_by; CREATE TABLE test_empty_order_by(a UInt8) ENGINE = MergeTree() SETTINGS index_granularity = 8192; " 2>&1 \ | grep -F -q "You must provide an ORDER BY or PRIMARY KEY expression in the table definition." && echo 'OK' || echo 'FAIL' # setting disabled and primary key in table definition -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS test_empty_order_by; CREATE TABLE test_empty_order_by(a UInt8) ENGINE = MergeTree() PRIMARY KEY a SETTINGS index_granularity = 8192; SHOW CREATE TABLE test_empty_order_by; diff --git a/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh index 5065da371a8..b77e5b0b402 100755 --- a/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh +++ b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh @@ -15,7 +15,7 @@ if [[ $($CLICKHOUSE_CLIENT -q "select count()>0 from system.clusters where clust cluster=test_cluster_database_replicated fi -$CLICKHOUSE_CLIENT -nm --distributed_ddl_output_mode=none -q " +$CLICKHOUSE_CLIENT -m --distributed_ddl_output_mode=none -q " drop table if exists rmt1; drop table if exists rmt2; @@ -46,7 +46,7 @@ part_name='%' # wait while there be at least one 'No active replica has part all_0_1_1 or covering part' in logs for _ in {0..50}; do - no_active_repilica_messages=$($CLICKHOUSE_CLIENT -nm -q " + no_active_repilica_messages=$($CLICKHOUSE_CLIENT -m -q " system flush logs; select count() @@ -65,7 +65,7 @@ for _ in {0..50}; do sleep 1 done -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " system start pulling replication log rmt2; system flush logs; diff --git a/tests/queries/0_stateless/02904_empty_order_by_with_setting_enabled.sh b/tests/queries/0_stateless/02904_empty_order_by_with_setting_enabled.sh index 7ac9b488be5..5f9dc6ea077 100755 --- a/tests/queries/0_stateless/02904_empty_order_by_with_setting_enabled.sh +++ b/tests/queries/0_stateless/02904_empty_order_by_with_setting_enabled.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh # setting enabled and no order by or primary key -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET create_table_empty_primary_key_by_default = true; DROP TABLE IF EXISTS test_empty_order_by; CREATE TABLE test_empty_order_by(a UInt8) ENGINE = MergeTree() SETTINGS index_granularity = 8192; @@ -13,7 +13,7 @@ ${CLICKHOUSE_CLIENT} -n --query=" " 2>&1 \ | grep -F -q "ORDER BY tuple()" && echo 'OK' || echo 'FAIL' # setting enabled and per-column primary key -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET create_table_empty_primary_key_by_default = true; DROP TABLE IF EXISTS test_empty_order_by; CREATE TABLE test_empty_order_by(a UInt8 PRIMARY KEY, b String PRIMARY KEY) ENGINE = MergeTree() SETTINGS index_granularity = 8192; @@ -21,7 +21,7 @@ ${CLICKHOUSE_CLIENT} -n --query=" " 2>&1 \ | grep -F -q "ORDER BY (a, b)" && echo 'OK' || echo 'FAIL' # setting enabled and primary key in table definition (not per-column or order by) -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET create_table_empty_primary_key_by_default = true; DROP TABLE IF EXISTS test_empty_order_by; CREATE TABLE test_empty_order_by(a UInt8, b String) ENGINE = MergeTree() PRIMARY KEY (a) SETTINGS index_granularity = 8192; @@ -29,7 +29,7 @@ ${CLICKHOUSE_CLIENT} -n --query=" " 2>&1 \ | grep -F -q "ORDER BY a" && echo 'OK' || echo 'FAIL' # setting enabled and order by in table definition (no primary key) -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET create_table_empty_primary_key_by_default = true; DROP TABLE IF EXISTS test_empty_order_by; CREATE TABLE test_empty_order_by(a UInt8, b String) ENGINE = MergeTree() ORDER BY (a, b) SETTINGS index_granularity = 8192; diff --git a/tests/queries/0_stateless/02907_backup_mv_with_no_inner_table.sh b/tests/queries/0_stateless/02907_backup_mv_with_no_inner_table.sh index 30ec50fa20f..e37f1e51c74 100755 --- a/tests/queries/0_stateless/02907_backup_mv_with_no_inner_table.sh +++ b/tests/queries/0_stateless/02907_backup_mv_with_no_inner_table.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table if exists src; create table src (a Int32) engine = MergeTree() order by tuple(); @@ -15,14 +15,14 @@ create materialized view mv (a Int32) engine = MergeTree() order by tuple() as s uuid=$(${CLICKHOUSE_CLIENT} --query "select uuid from system.tables where table='mv' and database == currentDatabase()") inner_table=".inner_id.${uuid}" -${CLICKHOUSE_CLIENT} -nm --query "drop table \`$inner_table\` sync" +${CLICKHOUSE_CLIENT} -m --query "drop table \`$inner_table\` sync" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " set send_logs_level = 'error'; backup table ${CLICKHOUSE_DATABASE}.\`mv\` to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); " | grep -o "BACKUP_CREATED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table mv; restore table ${CLICKHOUSE_DATABASE}.\`mv\` from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); " | grep -o "RESTORED" diff --git a/tests/queries/0_stateless/02907_backup_mv_with_no_source_table.sh b/tests/queries/0_stateless/02907_backup_mv_with_no_source_table.sh index d59ebe400ee..f950954941f 100755 --- a/tests/queries/0_stateless/02907_backup_mv_with_no_source_table.sh +++ b/tests/queries/0_stateless/02907_backup_mv_with_no_source_table.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table if exists src; create table src (a Int32) engine = MergeTree() order by tuple(); @@ -15,18 +15,18 @@ drop table if exists mv; create materialized view mv to dst (a Int32) as select * from src; " -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table src; backup database ${CLICKHOUSE_DATABASE} on cluster test_shard_localhost to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); " | grep -o "BACKUP_CREATED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table mv; set allow_deprecated_database_ordinary=1; restore table ${CLICKHOUSE_DATABASE}.mv on cluster test_shard_localhost from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); " | grep -o "RESTORED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table if exists src; create table src (a Int32) engine = MergeTree() order by tuple(); @@ -37,13 +37,13 @@ drop table if exists mv; create materialized view mv to dst (a Int32) as select * from src; " -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table src; drop table dst; backup database ${CLICKHOUSE_DATABASE} on cluster test_shard_localhost to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}2'); " | grep -o "BACKUP_CREATED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table mv; set allow_deprecated_database_ordinary=1; restore table ${CLICKHOUSE_DATABASE}.mv on cluster test_shard_localhost from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}2'); diff --git a/tests/queries/0_stateless/02907_backup_restore_default_nullable.sh b/tests/queries/0_stateless/02907_backup_restore_default_nullable.sh index 8ed36a7edd7..dc5793d1638 100755 --- a/tests/queries/0_stateless/02907_backup_restore_default_nullable.sh +++ b/tests/queries/0_stateless/02907_backup_restore_default_nullable.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table if exists test; set data_type_default_nullable = 0; create table test (test String) ENGINE = MergeTree() ORDER BY tuple(); @@ -13,7 +13,7 @@ backup table ${CLICKHOUSE_DATABASE}.test on cluster test_shard_localhost to Disk ${CLICKHOUSE_CLIENT} --query "show create table test" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table test sync; set data_type_default_nullable = 1; restore table ${CLICKHOUSE_DATABASE}.test on cluster test_shard_localhost from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); diff --git a/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh b/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh index 742d24a97eb..eae307add10 100755 --- a/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh +++ b/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table if exists test; set flatten_nested = 0; create table test (test Array(Tuple(foo String, bar Float64))) ENGINE = MergeTree() ORDER BY tuple(); @@ -13,7 +13,7 @@ backup table ${CLICKHOUSE_DATABASE}.test on cluster test_shard_localhost to Disk ${CLICKHOUSE_CLIENT} --query "show create table test" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table if exists test2; set flatten_nested = 0; create table test2 (test Nested(foo String, bar Float64)) ENGINE = MergeTree() ORDER BY tuple(); @@ -22,7 +22,7 @@ backup table ${CLICKHOUSE_DATABASE}.test2 on cluster test_shard_localhost to Dis ${CLICKHOUSE_CLIENT} --query "show create table test2" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table test sync; set flatten_nested = 1; restore table ${CLICKHOUSE_DATABASE}.test on cluster test_shard_localhost from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); @@ -30,7 +30,7 @@ restore table ${CLICKHOUSE_DATABASE}.test on cluster test_shard_localhost from D ${CLICKHOUSE_CLIENT} --query "show create table test" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table test2 sync; set flatten_nested = 1; restore table ${CLICKHOUSE_DATABASE}.test2 on cluster test_shard_localhost from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}2'); diff --git a/tests/queries/0_stateless/02907_clickhouse_dictionary_bug.sh b/tests/queries/0_stateless/02907_clickhouse_dictionary_bug.sh index 57182050534..2cad15c6fcb 100755 --- a/tests/queries/0_stateless/02907_clickhouse_dictionary_bug.sh +++ b/tests/queries/0_stateless/02907_clickhouse_dictionary_bug.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " DROP DICTIONARY IF EXISTS 02907_dictionary; DROP TABLE IF EXISTS 02907_table; diff --git a/tests/queries/0_stateless/02907_system_backups_profile_events.sh b/tests/queries/0_stateless/02907_system_backups_profile_events.sh index 801056a2844..9a1d5a3db11 100755 --- a/tests/queries/0_stateless/02907_system_backups_profile_events.sh +++ b/tests/queries/0_stateless/02907_system_backups_profile_events.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple(); " @@ -12,10 +12,10 @@ create table test (a Int32) engine = MergeTree() order by tuple(); backup_id=${CLICKHOUSE_TEST_UNIQUE_NAME} backup_name="Disk('backups', '$backup_id')"; -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " backup table ${CLICKHOUSE_DATABASE}.test to $backup_name; " | grep -o "BACKUP_CREATED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " select ProfileEvents['BackupEntriesCollectorMicroseconds'] > 10 from system.backups where name='Disk(\'backups\', \'$backup_id\')' " diff --git a/tests/queries/0_stateless/02908_Npy_files_caching.sh b/tests/queries/0_stateless/02908_Npy_files_caching.sh index 4845f740972..218e13efb95 100755 --- a/tests/queries/0_stateless/02908_Npy_files_caching.sh +++ b/tests/queries/0_stateless/02908_Npy_files_caching.sh @@ -7,13 +7,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/one_dim.npy') settings optimize_count_from_files=0" $CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/one_dim.npy') settings optimize_count_from_files=1" $CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/one_dim.npy', auto, 'array Int64') settings optimize_count_from_files=1" -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " desc file('$CURDIR/data_npy/one_dim.npy'); select number_of_rows from system.schema_inference_cache where format='Npy'; " $CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/npy_big.npy') settings optimize_count_from_files=0" $CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/npy_big.npy') settings optimize_count_from_files=1" -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " desc file('$CURDIR/data_npy/npy_big.npy'); select number_of_rows from system.schema_inference_cache where format='Npy'; " diff --git a/tests/queries/0_stateless/02908_table_ttl_dependency.sh b/tests/queries/0_stateless/02908_table_ttl_dependency.sh index 70136b4a42b..0bc02426f61 100755 --- a/tests/queries/0_stateless/02908_table_ttl_dependency.sh +++ b/tests/queries/0_stateless/02908_table_ttl_dependency.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS 02908_dependent; DROP TABLE IF EXISTS 02908_main; @@ -14,11 +14,11 @@ $CLICKHOUSE_CLIENT -nm -q " CREATE TABLE 02908_dependent (a UInt32, ts DateTime) ENGINE = MergeTree ORDER BY a TTL ts + 1 WHERE a IN (SELECT a FROM ${CLICKHOUSE_DATABASE}.02908_main); " -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE 02908_main; " 2>&1 | grep -F -q "HAVE_DEPENDENT_OBJECTS" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE 02908_dependent; DROP TABLE 02908_main; " diff --git a/tests/queries/0_stateless/02909_settings_in_json_schema_cache.sh b/tests/queries/0_stateless/02909_settings_in_json_schema_cache.sh index 8da144f90ca..75d491642ea 100755 --- a/tests/queries/0_stateless/02909_settings_in_json_schema_cache.sh +++ b/tests/queries/0_stateless/02909_settings_in_json_schema_cache.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh echo '{"x" : 42}' > $CLICKHOUSE_TEST_UNIQUE_NAME.json -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.json') SETTINGS schema_inference_make_columns_nullable=1; DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.json') SETTINGS schema_inference_make_columns_nullable=0; SELECT count() from system.schema_inference_cache where format = 'JSON' and additional_format_info like '%schema_inference_make_columns_nullable%';" diff --git a/tests/queries/0_stateless/02915_input_table_function_in_subquery.sh b/tests/queries/0_stateless/02915_input_table_function_in_subquery.sh index 80e38338751..7ad38e11e96 100755 --- a/tests/queries/0_stateless/02915_input_table_function_in_subquery.sh +++ b/tests/queries/0_stateless/02915_input_table_function_in_subquery.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " CREATE TABLE IF NOT EXISTS ts_data_double_raw ( device_id UInt32 NOT NULL CODEC(ZSTD), diff --git a/tests/queries/0_stateless/02915_lazy_loading_of_base_backups.sh b/tests/queries/0_stateless/02915_lazy_loading_of_base_backups.sh index 5f0f41a956b..b6d6ca57768 100755 --- a/tests/queries/0_stateless/02915_lazy_loading_of_base_backups.sh +++ b/tests/queries/0_stateless/02915_lazy_loading_of_base_backups.sh @@ -13,40 +13,40 @@ b_backup="Disk('backups', '$b_backup_id')" c_backup_id=${CLICKHOUSE_TEST_UNIQUE_NAME}_c c_backup="Disk('backups', '$c_backup_id')" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP TABLE IF EXISTS tbl1; DROP TABLE IF EXISTS tbl2; DROP TABLE IF EXISTS tbl3; " -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " CREATE TABLE tbl1 (a Int32) ENGINE = MergeTree() ORDER BY tuple(); " # The following BACKUP command must write backup 'a'. -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " BACKUP DATABASE ${CLICKHOUSE_DATABASE} TO $a_backup SETTINGS id='$a_backup_id'; " | grep -o "BACKUP_CREATED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " CREATE TABLE tbl2 (a Int32) ENGINE = MergeTree() ORDER BY tuple(); " # The following BACKUP command must read backup 'a' and write backup 'b'. -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " BACKUP DATABASE ${CLICKHOUSE_DATABASE} TO $b_backup SETTINGS id='$b_backup_id', base_backup=$a_backup; " | grep -o "BACKUP_CREATED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " CREATE TABLE tbl3 (a Int32) ENGINE = MergeTree() ORDER BY tuple(); " # The following BACKUP command must read only backup 'b' (and not 'a') and write backup 'c'. -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " BACKUP DATABASE ${CLICKHOUSE_DATABASE} TO $c_backup SETTINGS id='$c_backup_id', base_backup=$b_backup; " | grep -o "BACKUP_CREATED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP TABLE tbl1; DROP TABLE tbl2; DROP TABLE tbl3; @@ -57,28 +57,28 @@ r2_restore_id=${CLICKHOUSE_TEST_UNIQUE_NAME}_r2 r3_restore_id=${CLICKHOUSE_TEST_UNIQUE_NAME}_r3 # The following RESTORE command must read all 3 backups 'a', 'b', c' because the table 'tbl1' was in the first backup. -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " RESTORE TABLE ${CLICKHOUSE_DATABASE}.tbl1 FROM $c_backup SETTINGS id='$r1_restore_id'; " | grep -o "RESTORED" # The following RESTORE command must read only 2 backups 'b', c' (and not 'a') because the table 'tbl2' was in the second backup. -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " RESTORE TABLE ${CLICKHOUSE_DATABASE}.tbl2 FROM $c_backup SETTINGS id='$r2_restore_id'; " | grep -o "RESTORED" # The following RESTORE command must read only 1 backup 'c' (and not 'a' or 'b') because the table 'tbl3' was in the third backup. -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " RESTORE TABLE ${CLICKHOUSE_DATABASE}.tbl3 FROM $c_backup SETTINGS id='$r3_restore_id'; " | grep -o "RESTORED" all_ids="['$a_backup_id', '$b_backup_id', '$c_backup_id', '$r1_restore_id', '$r2_restore_id', '$r3_restore_id']" id_prefix_len=`expr "${CLICKHOUSE_TEST_UNIQUE_NAME}_" : '.*'` -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " SELECT substr(id, 1 + $id_prefix_len) as short_id, ProfileEvents['BackupsOpenedForRead'], ProfileEvents['BackupsOpenedForWrite'] FROM system.backups WHERE id IN ${all_ids} ORDER BY short_id " -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP TABLE tbl1; DROP TABLE tbl2; DROP TABLE tbl3; diff --git a/tests/queries/0_stateless/02916_dictionary_access.sh b/tests/queries/0_stateless/02916_dictionary_access.sh index 08ee517ab3b..be62cc027ef 100755 --- a/tests/queries/0_stateless/02916_dictionary_access.sh +++ b/tests/queries/0_stateless/02916_dictionary_access.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) username="user_${CLICKHOUSE_TEST_UNIQUE_NAME}" dictname="dict_${CLICKHOUSE_TEST_UNIQUE_NAME}" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " CREATE DICTIONARY IF NOT EXISTS ${dictname} ( id UInt64, @@ -23,15 +23,15 @@ ${CLICKHOUSE_CLIENT} -nm --query " SELECT dictGet(${dictname}, 'value', 1); " -$CLICKHOUSE_CLIENT -nm --user="${username}" --query " +$CLICKHOUSE_CLIENT -m --user="${username}" --query " SELECT * FROM dictionary(${dictname}); " 2>&1 | grep -o ACCESS_DENIED | uniq -$CLICKHOUSE_CLIENT -nm --user="${username}" --query " +$CLICKHOUSE_CLIENT -m --user="${username}" --query " SELECT dictGet(${dictname}, 'value', 1); " 2>&1 | grep -o ACCESS_DENIED | uniq -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP DICTIONARY IF EXISTS ${dictname}; DROP USER IF EXISTS ${username}; " diff --git a/tests/queries/0_stateless/02916_joinget_dependency.sh b/tests/queries/0_stateless/02916_joinget_dependency.sh index 6477ae8c967..ff9332cb57f 100755 --- a/tests/queries/0_stateless/02916_joinget_dependency.sh +++ b/tests/queries/0_stateless/02916_joinget_dependency.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We test the dependency on the DROP -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS Sub_distributed; DROP TABLE IF EXISTS Sub; DROP TABLE IF EXISTS Mapping; @@ -20,8 +20,8 @@ $CLICKHOUSE_CLIENT -q " DROP TABLE Mapping; " 2>&1 | grep -cm1 "HAVE_DEPENDENT_OBJECTS" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE Sub_distributed; DROP TABLE Sub; DROP TABLE Mapping; -" \ No newline at end of file +" diff --git a/tests/queries/0_stateless/02930_client_file_log_comment.sh b/tests/queries/0_stateless/02930_client_file_log_comment.sh index 50cd587e4b5..393bffcaf59 100755 --- a/tests/queries/0_stateless/02930_client_file_log_comment.sh +++ b/tests/queries/0_stateless/02930_client_file_log_comment.sh @@ -14,7 +14,7 @@ echo -n 'select 4242' >> "$file2" $CLICKHOUSE_CLIENT --queries-file "$file1" "$file2" <<<'select 42' $CLICKHOUSE_CLIENT --log_comment foo --queries-file /dev/stdin <<<'select 424242' -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " system flush logs; select query, log_comment from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and event_date >= yesterday() and query = 'select 42' and type != 'QueryStart'; select query, log_comment from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and event_date >= yesterday() and query = 'select 4242' and type != 'QueryStart'; diff --git a/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh b/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh index d9e4a2c8f8b..76ab56a4570 100755 --- a/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh +++ b/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh echo "1,2" > $CLICKHOUSE_TEST_UNIQUE_NAME.csv -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " create table test (x UInt64, y UInt32, size UInt64) engine=Memory; insert into test select c1, c2, _size from file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') settings use_structure_from_insertion_table_in_table_functions=1; select * from test; diff --git a/tests/queries/0_stateless/02940_system_stacktrace_optimizations.sh b/tests/queries/0_stateless/02940_system_stacktrace_optimizations.sh index 0e23bb6c42b..9d1faf301d3 100755 --- a/tests/queries/0_stateless/02940_system_stacktrace_optimizations.sh +++ b/tests/queries/0_stateless/02940_system_stacktrace_optimizations.sh @@ -9,12 +9,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # no message at all echo "thread = 0" -$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=test -nm -q "select * from system.stack_trace where thread_id = 0" |& grep -F -o 'Send signal to' +$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=test -m -q "select * from system.stack_trace where thread_id = 0" |& grep -F -o 'Send signal to' # send messages to some threads echo "thread != 0" -$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=test -nm -q "select * from system.stack_trace where thread_id != 0 format Null" |& grep -F -o 'Send signal to' | grep -v 'Send signal to 0 threads (total)' +$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=test -m -q "select * from system.stack_trace where thread_id != 0 format Null" |& grep -F -o 'Send signal to' | grep -v 'Send signal to 0 threads (total)' # there is no thread with comm="foo", so no signals will be sent echo "thread_name = 'foo'" -$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=test -nm -q "select * from system.stack_trace where thread_name = 'foo' format Null" |& grep -F -o 'Send signal to 0 threads (total)' +$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=test -m -q "select * from system.stack_trace where thread_name = 'foo' format Null" |& grep -F -o 'Send signal to 0 threads (total)' diff --git a/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.sh b/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.sh index 27950866e81..44af2dbf26f 100755 --- a/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.sh +++ b/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.sh @@ -26,7 +26,7 @@ function wait_part() function restore_failpoints() { # restore entry error with failpoints (to avoid endless errors in logs) - $CLICKHOUSE_CLIENT -nm -q " + $CLICKHOUSE_CLIENT -m -q " system enable failpoint replicated_queue_unfail_entries; system sync replica $failed_replica; system disable failpoint replicated_queue_unfail_entries; @@ -34,7 +34,7 @@ function restore_failpoints() } trap restore_failpoints EXIT -$CLICKHOUSE_CLIENT -nm --insert_keeper_fault_injection_probability=0 -q " +$CLICKHOUSE_CLIENT -m --insert_keeper_fault_injection_probability=0 -q " drop table if exists data_r1; drop table if exists data_r2; @@ -45,7 +45,7 @@ $CLICKHOUSE_CLIENT -nm --insert_keeper_fault_injection_probability=0 -q " " # will fail ALTER_METADATA on one of replicas -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " system enable failpoint replicated_queue_fail_next_entry; alter table data_r1 drop index value_idx settings alter_sync=0; -- part all_0_0_0_1 @@ -80,7 +80,7 @@ fi # This will create MERGE_PARTS, on failed replica it will be fetched from source replica (since it does not have all parts to execute merge) $CLICKHOUSE_CLIENT -q "optimize table $success_replica final settings optimize_throw_if_noop=1, alter_sync=1" # part all_0_0_1_1 -$CLICKHOUSE_CLIENT -nm --insert_keeper_fault_injection_probability=0 -q " +$CLICKHOUSE_CLIENT -m --insert_keeper_fault_injection_probability=0 -q " insert into $success_replica (key) values (2); -- part all_2_2_0 -- Avoid 'Cannot select parts for optimization: Entry for part all_2_2_0 hasn't been read from the replication log yet' system sync replica $success_replica pull; diff --git a/tests/queries/0_stateless/02947_merge_tree_index_table_3.sh b/tests/queries/0_stateless/02947_merge_tree_index_table_3.sh index 6cb184cb1fe..ec699d974d4 100755 --- a/tests/queries/0_stateless/02947_merge_tree_index_table_3.sh +++ b/tests/queries/0_stateless/02947_merge_tree_index_table_3.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) user_name="${CLICKHOUSE_DATABASE}_test_user_02947" -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS t_merge_tree_index; DROP USER IF EXISTS $user_name; @@ -44,7 +44,7 @@ $CLICKHOUSE_CLIENT --user "$user_name" --password "password" -q "SELECT arr.size $CLICKHOUSE_CLIENT --user "$user_name" --password "password" -q "SELECT b FROM mergeTreeIndex(currentDatabase(), t_merge_tree_index, with_marks = true)" 2>&1 | grep -m1 -o "ACCESS_DENIED" || echo "OK" $CLICKHOUSE_CLIENT --user "$user_name" --password "password" -q "SELECT b.mark FROM mergeTreeIndex(currentDatabase(), t_merge_tree_index, with_marks = true)" 2>&1 | grep -m1 -o "ACCESS_DENIED" || echo "OK" -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS t_merge_tree_index; DROP USER IF EXISTS $user_name; " diff --git a/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh b/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh index daa9c571a5d..d06aba8a4b6 100755 --- a/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh +++ b/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE source_table ( id UInt64, diff --git a/tests/queries/0_stateless/02950_distributed_initial_query_event.sh b/tests/queries/0_stateless/02950_distributed_initial_query_event.sh index 7f690a681c4..737d5c6b41e 100755 --- a/tests/queries/0_stateless/02950_distributed_initial_query_event.sh +++ b/tests/queries/0_stateless/02950_distributed_initial_query_event.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh # CREATE TABLE local (x UInt8) Engine=Memory; # CREATE TABLE distributed ON CLUSTER cluster (p Date, i Int32) ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), x) -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS local; DROP TABLE IF EXISTS distributed; CREATE TABLE local (x UInt8) Engine=Memory; diff --git a/tests/queries/0_stateless/02974_backup_query_format_null.sh b/tests/queries/0_stateless/02974_backup_query_format_null.sh index ddba2f6de16..345a4f47b20 100755 --- a/tests/queries/0_stateless/02974_backup_query_format_null.sh +++ b/tests/queries/0_stateless/02974_backup_query_format_null.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP TABLE IF EXISTS tbl; CREATE TABLE tbl (a Int32) ENGINE = MergeTree() ORDER BY tuple(); INSERT INTO tbl VALUES (2), (80), (-12345); @@ -14,7 +14,7 @@ backup_name="Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}')" ${CLICKHOUSE_CLIENT} --query "BACKUP TABLE tbl TO ${backup_name} FORMAT Null" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP TABLE tbl; RESTORE ALL FROM ${backup_name} FORMAT Null " diff --git a/tests/queries/0_stateless/02981_insert_select_resize_to_max_insert_threads.sh b/tests/queries/0_stateless/02981_insert_select_resize_to_max_insert_threads.sh index e65c9654c9c..7ad5a2179f9 100755 --- a/tests/queries/0_stateless/02981_insert_select_resize_to_max_insert_threads.sh +++ b/tests/queries/0_stateless/02981_insert_select_resize_to_max_insert_threads.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nq """ +${CLICKHOUSE_CLIENT} -q """ CREATE TABLE t1_local ( n UInt64, diff --git a/tests/queries/0_stateless/03008_deduplication_random_setttings.sh b/tests/queries/0_stateless/03008_deduplication_random_setttings.sh index e9f59138177..07b99eb4e99 100755 --- a/tests/queries/0_stateless/03008_deduplication_random_setttings.sh +++ b/tests/queries/0_stateless/03008_deduplication_random_setttings.sh @@ -35,7 +35,7 @@ THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" -$CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " +$CLICKHOUSE_CLIENT --max_insert_block_size 1 -mq " $(python3 $CURDIR/03008_deduplication.python insert_several_blocks_into_table \ --insert-method $insert_method \ --table-engine $engine \ @@ -48,7 +48,7 @@ $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " ) " 1>/dev/null 2>&1 && echo 'insert_several_blocks_into_table OK' || echo "FAIL: insert_several_blocks_into_table ${THIS_RUN}" -$CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " +$CLICKHOUSE_CLIENT --max_insert_block_size 1 -mq " $(python3 $CURDIR/03008_deduplication.python mv_generates_several_blocks \ --insert-method $insert_method \ --table-engine $engine \ @@ -61,7 +61,7 @@ $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " ) " 1>/dev/null 2>&1 && echo 'mv_generates_several_blocks OK' || echo "FAIL: mv_generates_several_blocks ${THIS_RUN}" -$CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " +$CLICKHOUSE_CLIENT --max_insert_block_size 1 -mq " $(python3 $CURDIR/03008_deduplication.python several_mv_into_one_table \ --insert-method $insert_method \ --table-engine $engine \ diff --git a/tests/queries/0_stateless/03008_local_plain_rewritable.sh b/tests/queries/0_stateless/03008_local_plain_rewritable.sh index d51e180efc9..e61f9061297 100755 --- a/tests/queries/0_stateless/03008_local_plain_rewritable.sh +++ b/tests/queries/0_stateless/03008_local_plain_rewritable.sh @@ -8,7 +8,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query "drop table if exists 03008_test_local_mt sync" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " create table 03008_test_local_mt (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b) settings disk = disk( @@ -19,35 +19,35 @@ settings disk = disk( path = '/var/lib/clickhouse/disks/local_plain_rewritable/') " -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " insert into 03008_test_local_mt (*) values (1, 2, 0), (2, 2, 2), (3, 1, 9), (4, 7, 7), (5, 10, 2), (6, 12, 5); insert into 03008_test_local_mt (*) select number, number, number from numbers_mt(10000); " -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " select count(*) from 03008_test_local_mt; select (*) from 03008_test_local_mt order by tuple(a, b) limit 10; " ${CLICKHOUSE_CLIENT} --query "optimize table 03008_test_local_mt final;" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " alter table 03008_test_local_mt modify setting disk = '03008_local_plain_rewritable', old_parts_lifetime = 3600; select engine_full from system.tables WHERE database = currentDatabase() AND name = '03008_test_local_mt'; " | grep -c "old_parts_lifetime = 3600" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " select count(*) from 03008_test_local_mt; select (*) from 03008_test_local_mt order by tuple(a, b) limit 10; " -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " alter table 03008_test_local_mt update c = 0 where a % 2 = 1; alter table 03008_test_local_mt add column d Int64 after c; alter table 03008_test_local_mt drop column c; " 2>&1 | grep -Fq "SUPPORT_IS_DISABLED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " truncate table 03008_test_local_mt; select count(*) from 03008_test_local_mt; " diff --git a/tests/queries/0_stateless/03031_clickhouse_local_input.sh b/tests/queries/0_stateless/03031_clickhouse_local_input.sh index 6f59e9b9703..e2f9cf48108 100755 --- a/tests/queries/0_stateless/03031_clickhouse_local_input.sh +++ b/tests/queries/0_stateless/03031_clickhouse_local_input.sh @@ -6,15 +6,15 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) tmp_file="$CUR_DIR/$CLICKHOUSE_DATABASE.txt" echo '# foo' -$CLICKHOUSE_LOCAL --engine_file_truncate_on_insert=1 -n -q "insert into function file('$tmp_file', 'LineAsString', 'x String') select * from input('x String') format LineAsString" << "$INPUT_FILE" -$CLICKHOUSE_CLIENT --external --file="$INPUT_FILE" --name=t --structure='x String' -nm -q " +$CLICKHOUSE_CLIENT --external --file="$INPUT_FILE" --name=t --structure='x String' -m -q " select * from t; select * from t; " diff --git a/tests/queries/0_stateless/03143_prewhere_profile_events.sh b/tests/queries/0_stateless/03143_prewhere_profile_events.sh index 00daa0fe7cc..6a6b993e5f8 100755 --- a/tests/queries/0_stateless/03143_prewhere_profile_events.sh +++ b/tests/queries/0_stateless/03143_prewhere_profile_events.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} -q " DROP TABLE IF EXISTS t; CREATE TABLE t(a UInt32, b UInt32, c UInt32, d UInt32) ENGINE=MergeTree ORDER BY a SETTINGS min_bytes_for_wide_part=0, min_rows_for_wide_part=0; @@ -25,7 +25,7 @@ client_opts=( --max_threads 8 ) -${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_1" -nq " +${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_1" -q " SELECT * FROM t PREWHERE (b % 8192) = 42 @@ -33,7 +33,7 @@ PREWHERE (b % 8192) = 42 FORMAT Null " -${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_2" -nq " +${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_2" -q " SELECT * FROM t PREWHERE (b % 8192) = 42 AND (c % 8192) = 42 @@ -42,7 +42,7 @@ PREWHERE (b % 8192) = 42 AND (c % 8192) = 42 settings enable_multiple_prewhere_read_steps=1; " -${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_3" -nq " +${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_3" -q " SELECT * FROM t PREWHERE (b % 8192) = 42 AND (c % 16384) = 42 @@ -51,7 +51,7 @@ PREWHERE (b % 8192) = 42 AND (c % 16384) = 42 settings enable_multiple_prewhere_read_steps=0; " -${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_4" -nq " +${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_4" -q " SELECT b, c FROM t PREWHERE (b % 8192) = 42 AND (c % 8192) = 42 @@ -59,7 +59,7 @@ PREWHERE (b % 8192) = 42 AND (c % 8192) = 42 settings enable_multiple_prewhere_read_steps=1; " -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} -q " SYSTEM FLUSH LOGS; -- 52503 which is 43 * number of granules, 10000000 diff --git a/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh index 95aef9bbc5b..4e7b318e202 100755 --- a/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh +++ b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists tp_1; create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = MergeTree order by y partition by intDiv(y, 100) settings max_parts_to_merge_at_once=1; insert into tp_1 select number, number from numbers(3); @@ -25,7 +25,7 @@ alter table tp_1 drop projection pp; alter table tp_1 attach partition '0'; " -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " set send_logs_level='fatal'; check table tp_1 settings check_query_single_value_result = 0;" | grep -o "Found unexpected projection directories: pp.proj" @@ -34,19 +34,19 @@ $CLICKHOUSE_CLIENT -q " backup table tp_1 to Disk('backups', '$backup_id'); " | grep -o "BACKUP_CREATED" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " set send_logs_level='fatal'; drop table tp_1; restore table tp_1 from Disk('backups', '$backup_id'); " | grep -o "RESTORED" $CLICKHOUSE_CLIENT -q "select count() from tp_1;" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " set send_logs_level='fatal'; check table tp_1 settings check_query_single_value_result = 0;" | grep -o "Found unexpected projection directories: pp.proj" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " set send_logs_level='fatal'; check table tp_1" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " set send_logs_level='fatal'; drop table tp_1" diff --git a/tests/queries/0_stateless/03155_test_move_to_prewhere.sh b/tests/queries/0_stateless/03155_test_move_to_prewhere.sh index b6980b3a23a..f33a6b3ef27 100755 --- a/tests/queries/0_stateless/03155_test_move_to_prewhere.sh +++ b/tests/queries/0_stateless/03155_test_move_to_prewhere.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} -q " CREATE TABLE event_envoy ( timestamp_interval DateTime CODEC(DoubleDelta), @@ -18,7 +18,7 @@ ${CLICKHOUSE_CLIENT} -nq " INSERT INTO event_envoy SELECT now() - number, 'us-east-1', 'ch_super_fast' FROM numbers_mt(1e5); " -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} -q " CREATE TABLE event_envoy_remote ( timestamp_interval DateTime CODEC(DoubleDelta), diff --git a/tests/queries/0_stateless/03156_default_multiquery_split.sh b/tests/queries/0_stateless/03156_default_multiquery_split.sh index 8ba2f46b786..d849fb5a162 100755 --- a/tests/queries/0_stateless/03156_default_multiquery_split.sh +++ b/tests/queries/0_stateless/03156_default_multiquery_split.sh @@ -53,6 +53,6 @@ SELECT * FROM TEST2 ORDER BY value; DROP TABLE TEST1; DROP TABLE TEST2; EOF -$CLICKHOUSE_CLIENT -m -n < "$SQL_FILE_NAME" +$CLICKHOUSE_CLIENT -m < "$SQL_FILE_NAME" rm "$SQL_FILE_NAME" diff --git a/tests/queries/0_stateless/03169_time_virtual_column.sh b/tests/queries/0_stateless/03169_time_virtual_column.sh index fef1de8c6f2..b289f39accb 100755 --- a/tests/queries/0_stateless/03169_time_virtual_column.sh +++ b/tests/queries/0_stateless/03169_time_virtual_column.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo "1,2" > $CLICKHOUSE_TEST_UNIQUE_NAME.csv sleep 1 -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " select _size, (dateDiff('millisecond', _time, now()) < 600000 AND dateDiff('millisecond', _time, now()) > 0) from file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv'); " rm $CLICKHOUSE_TEST_UNIQUE_NAME.csv diff --git a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh index 7c567c0f58f..af702569794 100755 --- a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh +++ b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nq " +$CLICKHOUSE_CLIENT -q " CREATE TABLE ids (id UUID, whatever String) Engine=MergeTree ORDER BY tuple(); INSERT INTO ids VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', 'whatever'); diff --git a/tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.sh b/tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.sh index ce53f467823..583257d8fd3 100755 --- a/tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.sh +++ b/tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.sh @@ -5,25 +5,25 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh echo -e 'a,b,c\n1,2,3' > $CLICKHOUSE_TEST_UNIQUE_NAME.csv -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_skip_first_lines=1; DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_skip_first_lines=0; SELECT count() from system.schema_inference_cache where format = 'CSV' and additional_format_info like '%skip_first_lines%';" echo -e 'a,b,c\n"1",2,3' > $CLICKHOUSE_TEST_UNIQUE_NAME.csv -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_try_infer_numbers_from_strings=1; DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_try_infer_numbers_from_strings=0; SELECT count() from system.schema_inference_cache where format = 'CSV' and additional_format_info like '%try_infer_numbers_from_strings%';" echo -e 'a,b,c\n"(1,2,3)",2,3' > $CLICKHOUSE_TEST_UNIQUE_NAME.csv -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_try_infer_strings_from_quoted_tuples=1; DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_try_infer_strings_from_quoted_tuples=0; SELECT count() from system.schema_inference_cache where format = 'CSV' and additional_format_info like '%try_infer_strings_from_quoted_tuples%';" echo -e 'a\tb\tc\n1\t2\t3' > $CLICKHOUSE_TEST_UNIQUE_NAME.tsv -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.tsv') SETTINGS input_format_tsv_skip_first_lines=1; DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.tsv') SETTINGS input_format_tsv_skip_first_lines=0; SELECT count() from system.schema_inference_cache where format = 'TSV' and additional_format_info like '%skip_first_lines%';" diff --git a/tests/queries/0_stateless/03198_unload_primary_key_outdated.sh b/tests/queries/0_stateless/03198_unload_primary_key_outdated.sh index 4f217935123..c759cc34425 100755 --- a/tests/queries/0_stateless/03198_unload_primary_key_outdated.sh +++ b/tests/queries/0_stateless/03198_unload_primary_key_outdated.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n " +$CLICKHOUSE_CLIENT " DROP TABLE IF EXISTS t_unload_primary_key; CREATE TABLE t_unload_primary_key (a UInt64, b UInt64) @@ -26,7 +26,7 @@ for _ in {1..100}; do sleep 0.3 done -$CLICKHOUSE_CLIENT -n " +$CLICKHOUSE_CLIENT " SELECT name, active, primary_key_bytes_in_memory FROM system.parts WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_unload_primary_key' ORDER BY name; DROP TABLE IF EXISTS t_unload_primary_key; " diff --git a/tests/queries/0_stateless/03199_dictionary_table_access.sh b/tests/queries/0_stateless/03199_dictionary_table_access.sh index 952b466b5da..14f017c7fbc 100755 --- a/tests/queries/0_stateless/03199_dictionary_table_access.sh +++ b/tests/queries/0_stateless/03199_dictionary_table_access.sh @@ -8,7 +8,7 @@ username="user_${CLICKHOUSE_TEST_UNIQUE_NAME}" dictname="dict_${CLICKHOUSE_TEST_UNIQUE_NAME}" dicttablename="dict_table_${CLICKHOUSE_TEST_UNIQUE_NAME}" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " CREATE DICTIONARY IF NOT EXISTS ${dictname} ( id UInt64, @@ -26,15 +26,15 @@ ${CLICKHOUSE_CLIENT} -nm --query " SELECT * FROM ${dicttablename}; " -$CLICKHOUSE_CLIENT -nm --user="${username}" --query " +$CLICKHOUSE_CLIENT -m --user="${username}" --query " SELECT * FROM ${dictname}; " 2>&1 | grep -o ACCESS_DENIED | uniq -$CLICKHOUSE_CLIENT -nm --user="${username}" --query " +$CLICKHOUSE_CLIENT -m --user="${username}" --query " SELECT * FROM ${dicttablename}; " 2>&1 | grep -o ACCESS_DENIED | uniq -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP TABLE IF EXISTS ${dicttablename} SYNC; DROP DICTIONARY IF EXISTS ${dictname}; DROP USER IF EXISTS ${username}; diff --git a/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh index 8a77538f592..adbb0cb6de0 100755 --- a/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh +++ b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh echo '{"x" : 42}' > $CLICKHOUSE_TEST_UNIQUE_NAME.json -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.json') SETTINGS input_format_max_bytes_to_read_for_schema_inference=1000; SELECT additional_format_info from system.schema_inference_cache" From 8f124710ef97801f020fb88e31bcd94529a112fe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 12 Aug 2024 10:27:21 +0000 Subject: [PATCH 566/644] Remove Log engine from Kafka integration tests It doesn't work well when `thread_per_consumer` is used as writer can make readers starve when `shared_time_mutex` prefers writes over reads. --- tests/integration/test_storage_kafka/test.py | 26 ++++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 4b6c9922d74..52d6054c12a 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1019,7 +1019,7 @@ def test_kafka_formats(kafka_cluster, create_query_generator): DROP TABLE IF EXISTS test.kafka_{format_name}_mv; - CREATE MATERIALIZED VIEW test.kafka_{format_name}_mv Engine=Log AS + CREATE MATERIALIZED VIEW test.kafka_{format_name}_mv ENGINE=MergeTree ORDER BY tuple() AS SELECT *, _topic, _partition, _offset FROM test.kafka_{format_name}; """.format( topic_name=topic_name, @@ -2460,7 +2460,7 @@ def test_kafka_commit_on_block_write(kafka_cluster, create_query_generator): (generate_old_create_table_query, "kafka.*Committed offset 2.*virt2_[01]"), ( generate_new_create_table_query, - r"kafka.*Saved offset 2[0-9]* for topic-partition \[virt2_[01]:[0-9]+", + r"kafka.*Saved offset 2 for topic-partition \[virt2_[01]:[0-9]+", ), ], ) @@ -2494,7 +2494,7 @@ def test_kafka_virtual_columns2(kafka_cluster, create_query_generator, log_line) f""" {create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY tuple() AS SELECT value, _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp), toUnixTimestamp64Milli(_timestamp_ms), _headers.name, _headers.value FROM test.kafka; """ ) @@ -2729,7 +2729,7 @@ def test_kafka_produce_key_timestamp(kafka_cluster, create_query_generator, log_ DROP TABLE IF EXISTS test.consumer; {writer_create_query}; {reader_create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY tuple() AS SELECT key, value, inserted_key, toUnixTimestamp(inserted_timestamp), _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp) FROM test.kafka; """ ) @@ -2865,7 +2865,7 @@ def test_kafka_produce_consume_avro(kafka_cluster, create_query_generator): {writer_create_query}; {reader_create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY tuple() AS SELECT key, value FROM test.kafka; """ ) @@ -3537,7 +3537,7 @@ def test_bad_reschedule(kafka_cluster, create_query_generator): f""" {create_query}; - CREATE MATERIALIZED VIEW test.destination Engine=Log AS + CREATE MATERIALIZED VIEW test.destination ENGINE=MergeTree ORDER BY tuple() AS SELECT key, now() as consume_ts, @@ -3745,7 +3745,7 @@ def test_kafka_unavailable(kafka_cluster, create_query_generator, do_direct_read f""" {create_query}; - CREATE MATERIALIZED VIEW test.destination_unavailable Engine=Log AS + CREATE MATERIALIZED VIEW test.destination_unavailable ENGINE=MergeTree ORDER BY tuple() AS SELECT key, now() as consume_ts, @@ -4267,12 +4267,12 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator {create_query}; DROP TABLE IF EXISTS test.kafka_data_{format_name}_mv; - CREATE MATERIALIZED VIEW test.kafka_data_{format_name}_mv Engine=Log AS + CREATE MATERIALIZED VIEW test.kafka_data_{format_name}_mv ENGINE=MergeTree ORDER BY tuple() AS SELECT *, _topic, _partition, _offset FROM test.kafka_{format_name} WHERE length(_error) = 0; DROP TABLE IF EXISTS test.kafka_errors_{format_name}_mv; - CREATE MATERIALIZED VIEW test.kafka_errors_{format_name}_mv Engine=Log AS + CREATE MATERIALIZED VIEW test.kafka_errors_{format_name}_mv ENGINE=MergeTree ORDER BY tuple() AS SELECT {raw_message} as raw_message, _error as error, _topic as topic, _partition as partition, _offset as offset FROM test.kafka_{format_name} WHERE length(_error) > 0; """ @@ -4796,7 +4796,7 @@ def test_max_rows_per_message(kafka_cluster, create_query_generator): DROP TABLE IF EXISTS test.kafka; {create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS SELECT key, value FROM test.kafka; """ ) @@ -4875,7 +4875,7 @@ def test_row_based_formats(kafka_cluster, create_query_generator): {create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS SELECT key, value FROM test.{table_name}; INSERT INTO test.{table_name} SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}); @@ -4982,7 +4982,7 @@ def test_block_based_formats_2(kafka_cluster, create_query_generator): {create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS SELECT key, value FROM test.{table_name}; INSERT INTO test.{table_name} SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; @@ -5362,7 +5362,7 @@ def test_formats_errors(kafka_cluster): input_format_with_names_use_header=0, format_schema='key_value_message:Message'; - CREATE MATERIALIZED VIEW test.view Engine=Log AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS SELECT key, value FROM test.{table_name}; """ ) From 06ceaee50218507f49bfc714903240ad4b5d81a0 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 12 Aug 2024 11:09:45 +0000 Subject: [PATCH 567/644] Fix test 01903_correct_block_size_prediction_with_default - Don't allow random settings that affect the memory usage - Run two queries and compare the memory usage, rather than having an arbitrary hardcoded value --- ...ock_size_prediction_with_default.reference | 6 ++++ ...rect_block_size_prediction_with_default.sh | 36 +++++++++++++++++++ ...ect_block_size_prediction_with_default.sql | 13 ------- 3 files changed, 42 insertions(+), 13 deletions(-) create mode 100755 tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh delete mode 100644 tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sql diff --git a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.reference b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.reference index b70a1cb7c75..2c66db91737 100644 --- a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.reference +++ b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.reference @@ -1,3 +1,9 @@ 8 +8 +1 4 4 +1 +4 +4 +1 diff --git a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh new file mode 100755 index 00000000000..922dcb957e5 --- /dev/null +++ b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash +# Tags: no-random-merge-tree-settings, no-random-settings + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +sql="toUInt16OrNull(arrayFirst((v, k) -> (k = '4Id'), arr[2], arr[1]))" + +# Create the table and fill it +$CLICKHOUSE_CLIENT -n --query=" + CREATE TABLE test_extract(str String, arr Array(Array(String)) ALIAS extractAllGroupsHorizontal(str, '\\W(\\w+)=(\"[^\"]*?\"|[^\",}]*)')) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY tuple(); + INSERT INTO test_extract (str) WITH range(8) as range_arr, arrayMap(x-> concat(toString(x),'Id'), range_arr) as key, arrayMap(x -> rand() % 8, range_arr) as val, arrayStringConcat(arrayMap((x,y) -> concat(x,'=',toString(y)), key, val),',') as str SELECT str FROM numbers(500000); + ALTER TABLE test_extract ADD COLUMN 15Id Nullable(UInt16) DEFAULT $sql;" + +function test() +{ + # Execute two queries and compare if they have similar memory usage: + # The first query uses the default column value, while the second explicitly uses the same SQL as the default value. + # Follow https://github.com/ClickHouse/ClickHouse/issues/17317 for more info about the issue + where=$1 + + uuid_1=$($CLICKHOUSE_CLIENT --query="SELECT generateUUIDv4()") + $CLICKHOUSE_CLIENT --query="SELECT uniq(15Id) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_1 + uuid_2=$($CLICKHOUSE_CLIENT --query="SELECT generateUUIDv4()") + $CLICKHOUSE_CLIENT --query="SELECT uniq($sql) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_2 + $CLICKHOUSE_CLIENT -n --query=" + SYSTEM FLUSH LOGS; + WITH memory_1 AS (SELECT memory_usage FROM system.query_log WHERE event_time > now() - INTERVAL 5 MINUTE AND query_id='$uuid_1' AND type = 'QueryFinish' as memory_1), + memory_2 AS (SELECT memory_usage FROM system.query_log WHERE event_time > now() - INTERVAL 5 MINUTE AND query_id='$uuid_2' AND type = 'QueryFinish' as memory_2) + SELECT memory_1.memory_usage <= 1.2 * memory_2.memory_usage FROM memory_1, memory_2;" +} + +test "" +test "PREWHERE 15Id < 4" +test "WHERE 15Id < 4" diff --git a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sql b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sql deleted file mode 100644 index 2eec08635eb..00000000000 --- a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sql +++ /dev/null @@ -1,13 +0,0 @@ --- Tags: no-random-merge-tree-settings - -CREATE TABLE test_extract(str String, arr Array(Array(String)) ALIAS extractAllGroupsHorizontal(str, '\\W(\\w+)=("[^"]*?"|[^",}]*)')) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY tuple(); - -INSERT INTO test_extract (str) WITH range(8) as range_arr, arrayMap(x-> concat(toString(x),'Id'), range_arr) as key, arrayMap(x -> rand() % 8, range_arr) as val, arrayStringConcat(arrayMap((x,y) -> concat(x,'=',toString(y)), key, val),',') as str SELECT str FROM numbers(500000); - -ALTER TABLE test_extract ADD COLUMN `15Id` Nullable(UInt16) DEFAULT toUInt16OrNull(arrayFirst((v, k) -> (k = '4Id'), arr[2], arr[1])); - -SELECT uniq(15Id) FROM test_extract SETTINGS max_threads=1, max_memory_usage=100000000; - -SELECT uniq(15Id) FROM test_extract PREWHERE 15Id < 4 SETTINGS max_threads=1, max_memory_usage=100000000; - -SELECT uniq(15Id) FROM test_extract WHERE 15Id < 4 SETTINGS max_threads=1, max_memory_usage=100000000; From 38f3131e11d1d777101a975361eb585fd6263300 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Aug 2024 11:39:36 +0000 Subject: [PATCH 568/644] Fix review comments --- src/Columns/ColumnDynamic.cpp | 43 +++++++++++-------- src/Columns/ColumnDynamic.h | 12 ++++++ src/DataTypes/DataTypeDynamic.cpp | 4 +- .../Serializations/SerializationDynamic.cpp | 2 +- src/Functions/FunctionsConversion.cpp | 2 +- ..._read_shared_subcolumns_small.reference.j2 | 3 ++ 6 files changed, 43 insertions(+), 23 deletions(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 7246be29592..b1d28342a28 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -115,8 +115,8 @@ bool ColumnDynamic::addNewVariant(const DataTypePtr & new_variant, const String if (variant_info.variant_name_to_discriminator.contains(new_variant_name)) return true; - /// Check if we reached maximum number of variants (don't count shared variant). - if (variant_info.variant_names.size() - 1 == max_dynamic_types) + /// Check if we reached maximum number of variants. + if (!canAddNewVariant()) { /// Dynamic column should always have shared variant. if (!variant_info.variant_name_to_discriminator.contains(getSharedVariantTypeName())) @@ -194,8 +194,8 @@ std::vector * ColumnDynamic::combineVariants(const { const DataTypes & current_variants = assert_cast(*variant_info.variant_type).getVariants(); - /// We cannot combine Variants if total number of variants exceeds max_dynamic_types (don't count shared variant). - if (current_variants.size() + num_new_variants - 1 > max_dynamic_types) + /// We cannot combine Variants if total number of variants exceeds max_dynamic_types. + if (!canAddNewVariants(num_new_variants)) { /// Remember that we cannot combine our variant with this one, so we will not try to do it again. variants_with_failed_combination.insert(other_variant_info.variant_name); @@ -403,11 +403,11 @@ void ColumnDynamic::doInsertRangeFrom(const IColumn & src_, size_t start, size_t auto shared_variant_discr = getSharedVariantDiscriminator(); variant_col.insertRangeFrom(*dynamic_src.variant_column, start, length, *global_discriminators_mapping, shared_variant_discr); - /// We should process insertion from srs shared variant separately, because it can contain + /// We should process insertion from src shared variant separately, because it can contain /// values that should be extracted into our variants. insertRangeFrom above didn't insert /// values into our shared variant (we specified shared_variant_discr as special skip discriminator). - /// Check if srs shared variant is empty, nothing to do in this case. + /// Check if src shared variant is empty, nothing to do in this case. if (dynamic_src.getSharedVariant().empty()) return; @@ -466,7 +466,7 @@ void ColumnDynamic::doInsertRangeFrom(const IColumn & src_, size_t start, size_t other_to_new_discriminators.reserve(dynamic_src.variant_info.variant_names.size()); /// Check if we cannot add any more new variants. In this case we will insert all new variants into shared variant. - if (variant_info.variant_names.size() - 1 == max_dynamic_types) + if (!canAddNewVariant()) { auto shared_variant_discr = getSharedVariantDiscriminator(); for (const auto & variant_name : dynamic_src.variant_info.variant_names) @@ -496,7 +496,7 @@ void ColumnDynamic::doInsertRangeFrom(const IColumn & src_, size_t start, size_t /// Add new variants from sorted list until we reach max_dynamic_types. for (const auto & [_, discr] : new_variants_with_sizes) { - if (new_variants.size() - 1 == max_dynamic_types) + if (!canAddNewVariant(new_variants.size())) break; new_variants.push_back(src_variants[discr]); } @@ -846,13 +846,17 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_ /// Check if both values are in shared variant. if (left_discr == left_shared_variant_discr && right_discr == right_shared_variant_discr) { - /// Extract type names from both values. + /// First check if both type and value are equal. auto left_value = getSharedVariant().getDataAt(left_variant.offsetAt(n)); + auto right_value = right_dynamic.getSharedVariant().getDataAt(right_variant.offsetAt(m)); + if (left_value == right_value) + return 0; + + /// Extract type names from both values. ReadBufferFromMemory buf_left(left_value.data, left_value.size); auto left_data_type = decodeDataType(buf_left); auto left_data_type_name = left_data_type->getName(); - auto right_value = right_dynamic.getSharedVariant().getDataAt(right_variant.offsetAt(m)); ReadBufferFromMemory buf_right(right_value.data, right_value.size); auto right_data_type = decodeDataType(buf_right); auto right_data_type_name = right_data_type->getName(); @@ -977,8 +981,6 @@ ColumnPtr ColumnDynamic::compress() const void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source_columns) { - LOG_DEBUG(getLogger("ColumnDynamic"), "takeDynamicStructureFromSourceColumns"); - if (!empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "takeDynamicStructureFromSourceColumns should be called only on empty Dynamic column"); @@ -1050,8 +1052,10 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source DataTypePtr result_variant_type; Statistics new_statistics(Statistics::Source::MERGE); + /// Reset max_dynamic_types to global_max_dynamic_types. + max_dynamic_types = global_max_dynamic_types; /// Check if the number of all dynamic types exceeds the limit. - if (all_variants.size() - 1 > global_max_dynamic_types) + if (!canAddNewVariants(0, all_variants.size())) { /// Create list of variants with their sizes and sort it. std::vector> variants_with_sizes; @@ -1065,11 +1069,13 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source /// Take first max_dynamic_types variants from sorted list and fill shared_variants_statistics with the rest. DataTypes result_variants; - result_variants.reserve(global_max_dynamic_types + 1); + result_variants.reserve(max_dynamic_types + 1); /// +1 for shared variant. + /// Add shared variant. + result_variants.push_back(getSharedVariantDataType()); for (const auto & [size, variant] : variants_with_sizes) { /// Add variant to the resulting variants list until we reach max_dynamic_types. - if (result_variants.size() < global_max_dynamic_types) + if (canAddNewVariant(result_variants.size())) result_variants.push_back(variant); /// Add all remaining variants into shared_variants_statistics until we reach its max size. else if (new_statistics.shared_variants_statistics.size() < Statistics::MAX_SHARED_VARIANT_STATISTICS_SIZE) @@ -1078,8 +1084,6 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source break; } - /// Add shared variant. - result_variants.push_back(getSharedVariantDataType()); result_variant_type = std::make_shared(result_variants); } else @@ -1094,8 +1098,9 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source new_statistics.variants_statistics[variant_name] = total_sizes[variant_name]; statistics = std::make_shared(std::move(new_statistics)); - /// Reduce max_dynamic_types to the number of selected variants (without shared variant), so there will be no possibility + /// Reduce max_dynamic_types to the number of selected variants, so there will be no possibility /// to extend selected variants on inerts into this column during merges. + /// -1 because we don't count shared variant in the limit. max_dynamic_types = variant_info.variant_names.size() - 1; /// Now we have the resulting Variant that will be used in all merged columns. @@ -1112,7 +1117,7 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source { /// Try to find this variant in current source column. auto it = source_variant_info.variant_name_to_discriminator.find(variant_info.variant_names[i]); - if (it != source_variant_info.variant_name_to_discriminator.end()) + if (it != source_variant_info.variant_name_to_discriminator.end()) /// Add shared variant. variants_source_columns[i].push_back(source_dynamic_column.getVariantColumn().getVariantPtrByGlobalDiscriminator(it->second)); } } diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index a595a990964..1f050c9079e 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -27,6 +27,10 @@ namespace DB class ColumnDynamic final : public COWHelper, ColumnDynamic> { public: + /// Maximum limit on dynamic types. We use ColumnVariant to store all the types, + /// so the limit cannot be greater then ColumnVariant::MAX_NESTED_COLUMNS. + /// We also always have reserved variant for shared variant. + static constexpr size_t MAX_DYNAMIC_TYPES_LIMIT = ColumnVariant::MAX_NESTED_COLUMNS - 1; static constexpr const char * SHARED_VARIANT_TYPE_NAME = "SharedVariant"; struct Statistics @@ -359,6 +363,14 @@ public: size_t getMaxDynamicTypes() const { return max_dynamic_types; } + /// Check if we can add new variant types. + /// Shared variant doesn't count in the limit but always presents, + /// so we should subtract 1 from the total types count. + bool canAddNewVariants(size_t current_variants_count, size_t new_variants_count) { return current_variants_count + new_variants_count - 1 <= max_dynamic_types; } + bool canAddNewVariant(size_t current_variants_count) { return canAddNewVariants(current_variants_count, 1); } + bool canAddNewVariants(size_t new_variants_count) { return canAddNewVariants(variant_info.variant_names.size(), new_variants_count); } + bool canAddNewVariant() { return canAddNewVariants(variant_info.variant_names.size(), 1); } + void setVariantType(const DataTypePtr & variant_type); void setMaxDynamicPaths(size_t max_dynamic_type_); diff --git a/src/DataTypes/DataTypeDynamic.cpp b/src/DataTypes/DataTypeDynamic.cpp index e00638a50ab..04e76df57fe 100644 --- a/src/DataTypes/DataTypeDynamic.cpp +++ b/src/DataTypes/DataTypeDynamic.cpp @@ -73,8 +73,8 @@ static DataTypePtr create(const ASTPtr & arguments) auto * literal = argument->arguments->children[1]->as(); - if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.get() > ColumnVariant::MAX_NESTED_COLUMNS - 1) - throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'max_types' argument for Dynamic type should be a positive integer between 0 and 254"); + if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.get() > ColumnDynamic::MAX_DYNAMIC_TYPES_LIMIT) + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'max_types' argument for Dynamic type should be a positive integer between 0 and {}", ColumnDynamic::MAX_DYNAMIC_TYPES_LIMIT); return std::make_shared(literal->value.get()); } diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index 5fadb6e4de4..9cd0adcc2ed 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -288,7 +288,7 @@ ISerialization::DeserializeBinaryBulkStatePtr SerializationDynamic::deserializeD statistics.variants_statistics[variant->getName()] = variant_size; } - /// Second, rend statistics for shared variants. + /// Second, read statistics for shared variants. size_t statistics_size; readVarUInt(statistics_size, *structure_stream); String variant_name; diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 21b98cf505c..660efb46b37 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -4335,7 +4335,7 @@ private: auto type = decodeDataType(buf); auto type_name = type->getName(); auto it = shared_variant_to_index.find(type_name); - /// Check if didn't created column for this variant yet. + /// Check if we didn't create column for this variant yet. if (it == shared_variant_to_index.end()) { it = shared_variant_to_index.emplace(type_name, variant_columns_from_shared_variant.size()).first; diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 index 9c1f8fa45e8..de12c6b8737 100644 --- a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 @@ -1,3 +1,4 @@ +Memory Array(Array(Dynamic)) Array(Variant(String, UInt64)) LowCardinality(String) @@ -818,6 +819,7 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] +MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000 Array(Array(Dynamic)) Array(Variant(String, UInt64)) LowCardinality(String) @@ -1638,6 +1640,7 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] +MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1 Array(Array(Dynamic)) Array(Variant(String, UInt64)) LowCardinality(String) From e9d16bc0549fe2a1fc44e50bd173ebc9358ed838 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 12 Aug 2024 08:32:14 +0200 Subject: [PATCH 569/644] Use new mc restart --- docker/test/stateless/run.sh | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 69052cf4771..c70cbe1fe45 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -195,16 +195,18 @@ ORDER BY tuple()" # use async inserts to avoid creating too many parts ./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" queue_size=1000000 batch_size=500 ./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" queue_size=1000000 batch_size=500 + max_retries=100 retry=1 - while [ $retry -le $max_retries ]; do echo "clickminio restart attempt $retry:" - output=$(./mc admin service restart clickminio 2>&1) - echo "$output" + output=$(./mc admin service restart clickminio --wait --json 2>&1 | jq -r .status) + echo "Output of restart status: $output" - if echo "$output" | grep -q "Restarted \`clickminio\` successfully"; then + expected_output="success +success" + if [ "$output" = "$expected_output" ]; then echo "Restarted clickminio successfully." break fi @@ -218,7 +220,6 @@ if [ $retry -gt $max_retries ]; then echo "Failed to restart clickminio after $max_retries attempts." fi -./mc admin service restart clickminio ./mc admin trace clickminio > /test_output/minio.log & MC_ADMIN_PID=$! From b58a22aba7f590ada33bfce95dd525c4c8a414ae Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Aug 2024 11:41:42 +0000 Subject: [PATCH 570/644] Update test --- .../03036_dynamic_read_shared_subcolumns_small.sql.j2 | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 index 0c123d5f6fe..dde4f3f53c3 100644 --- a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 @@ -6,6 +6,7 @@ drop table if exists test; {% for engine in ['Memory', 'MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000', 'MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1'] -%} +select '{{ engine }}'; create table test (id UInt64, d Dynamic(max_types=2)) engine={{ engine }}; insert into test select number, number from numbers(10); From 737948470d6f2cd69f1842396984ec17aea49b65 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 12 Aug 2024 11:54:34 +0000 Subject: [PATCH 571/644] Fix syntax --- tests/integration/test_storage_kafka/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 52d6054c12a..bef90e1b9d3 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4796,7 +4796,7 @@ def test_max_rows_per_message(kafka_cluster, create_query_generator): DROP TABLE IF EXISTS test.kafka; {create_query}; - CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY (key, value) AS SELECT key, value FROM test.kafka; """ ) @@ -4875,7 +4875,7 @@ def test_row_based_formats(kafka_cluster, create_query_generator): {create_query}; - CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY (key, value) AS SELECT key, value FROM test.{table_name}; INSERT INTO test.{table_name} SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}); @@ -4982,7 +4982,7 @@ def test_block_based_formats_2(kafka_cluster, create_query_generator): {create_query}; - CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY (key, value) AS SELECT key, value FROM test.{table_name}; INSERT INTO test.{table_name} SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; @@ -5362,7 +5362,7 @@ def test_formats_errors(kafka_cluster): input_format_with_names_use_header=0, format_schema='key_value_message:Message'; - CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY (key, value) AS SELECT key, value FROM test.{table_name}; """ ) From 34643ee16c5452a08feac63aaaea605a8c912b37 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 12 Aug 2024 13:30:25 +0000 Subject: [PATCH 572/644] Run test only from modified files --- tests/ci/integration_tests_runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 84718462ab5..7b9e7d1f63e 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -69,9 +69,9 @@ def get_changed_tests_to_run(pr_info, repo_path): return [] for fpath in changed_files: - if "tests/integration/test_" in fpath: + if re.search("tests/integration/test_.*/test.*\.py", fpath) is not None: logging.info("File %s changed and seems like integration test", fpath) - result.add(fpath.split("/")[2]) + result.add("/".join(fpath.split("/")[2:])) return filter_existing_tests(result, repo_path) From f7c6eabb498b47b21c13dbf55efbda551902d09c Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 12 Aug 2024 13:44:05 +0000 Subject: [PATCH 573/644] Small fix to filter by current_database in system.query_log --- ...01903_correct_block_size_prediction_with_default.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh index 922dcb957e5..e898a9d5ee8 100755 --- a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh +++ b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh @@ -20,15 +20,15 @@ function test() # Follow https://github.com/ClickHouse/ClickHouse/issues/17317 for more info about the issue where=$1 - uuid_1=$($CLICKHOUSE_CLIENT --query="SELECT generateUUIDv4()") + uuid_1=$(cat /proc/sys/kernel/random/uuid) $CLICKHOUSE_CLIENT --query="SELECT uniq(15Id) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_1 - uuid_2=$($CLICKHOUSE_CLIENT --query="SELECT generateUUIDv4()") + uuid_2=$(cat /proc/sys/kernel/random/uuid) $CLICKHOUSE_CLIENT --query="SELECT uniq($sql) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_2 $CLICKHOUSE_CLIENT -n --query=" SYSTEM FLUSH LOGS; - WITH memory_1 AS (SELECT memory_usage FROM system.query_log WHERE event_time > now() - INTERVAL 5 MINUTE AND query_id='$uuid_1' AND type = 'QueryFinish' as memory_1), - memory_2 AS (SELECT memory_usage FROM system.query_log WHERE event_time > now() - INTERVAL 5 MINUTE AND query_id='$uuid_2' AND type = 'QueryFinish' as memory_2) - SELECT memory_1.memory_usage <= 1.2 * memory_2.memory_usage FROM memory_1, memory_2;" + WITH memory_1 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_1' AND type = 'QueryFinish' as memory_1), + memory_2 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_2' AND type = 'QueryFinish' as memory_2) + SELECT memory_1.memory_usage <= 1.2 * memory_2.memory_usage FROM memory_1, memory_2;" } test "" From 6cde029ed9cd6a22937193e12863974f1fa8f160 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 12 Aug 2024 13:48:44 +0000 Subject: [PATCH 574/644] Fix style --- tests/ci/integration_tests_runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 7b9e7d1f63e..f5dbef4f6db 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -69,7 +69,7 @@ def get_changed_tests_to_run(pr_info, repo_path): return [] for fpath in changed_files: - if re.search("tests/integration/test_.*/test.*\.py", fpath) is not None: + if re.search(r"tests/integration/test_.*/test.*\.py", fpath) is not None: logging.info("File %s changed and seems like integration test", fpath) result.add("/".join(fpath.split("/")[2:])) return filter_existing_tests(result, repo_path) From da5b9582a990f7a2c05c1a3dede3739fb9cbfcae Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 12 Aug 2024 13:54:17 +0000 Subject: [PATCH 575/644] Fix indent --- .../01903_correct_block_size_prediction_with_default.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh index e898a9d5ee8..075d9a1dacf 100755 --- a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh +++ b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh @@ -26,9 +26,9 @@ function test() $CLICKHOUSE_CLIENT --query="SELECT uniq($sql) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_2 $CLICKHOUSE_CLIENT -n --query=" SYSTEM FLUSH LOGS; - WITH memory_1 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_1' AND type = 'QueryFinish' as memory_1), - memory_2 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_2' AND type = 'QueryFinish' as memory_2) - SELECT memory_1.memory_usage <= 1.2 * memory_2.memory_usage FROM memory_1, memory_2;" + WITH memory_1 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_1' AND type = 'QueryFinish' as memory_1), + memory_2 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_2' AND type = 'QueryFinish' as memory_2) + SELECT memory_1.memory_usage <= 1.2 * memory_2.memory_usage FROM memory_1, memory_2;" } test "" From eb3ffb71847fc5a204af31665a4594b0918fc1d7 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 15:09:16 +0000 Subject: [PATCH 576/644] Add supportsReplication --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 625b1281c61..f925cb773f5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2351,7 +2351,7 @@ size_t MergeTreeData::clearOldTemporaryDirectories(const String & root_path, siz /// We don't control the amount of refs for temporary parts so we cannot decide can we remove blobs /// or not. So we are not doing it bool keep_shared = false; - if (disk->supportZeroCopyReplication() && settings->allow_remote_fs_zero_copy_replication) + if (disk->supportZeroCopyReplication() && settings->allow_remote_fs_zero_copy_replication && supportsReplication()) { LOG_WARNING(log, "Since zero-copy replication is enabled we are not going to remove blobs from shared storage for {}", full_path); keep_shared = true; From a39b9cf643bff565728be4083eb024ff5254f363 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 11 May 2024 13:05:24 +0000 Subject: [PATCH 577/644] Un-screw usearch's build description No directory 'SimSIMD-map' exists, the build only worked because SimSIMD support in usearch was (accidentally?) disabled. This commit corrects the build description. SimSIMD support in usearch will be enabled by a later commit. --- contrib/CMakeLists.txt | 2 +- contrib/usearch-cmake/CMakeLists.txt | 8 +++----- src/Storages/MergeTree/MergeTreeIndexUSearch.h | 1 - 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index b33e7083e32..98b992e1080 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -207,7 +207,7 @@ if (ARCH_S390X) endif() add_contrib (annoy-cmake annoy) -option(ENABLE_USEARCH "Enable USearch (Approximate Neighborhood Search, HNSW) support" ${ENABLE_LIBRARIES}) +option(ENABLE_USEARCH "Enable USearch" ${ENABLE_LIBRARIES}) if (ENABLE_USEARCH) add_contrib (FP16-cmake FP16) add_contrib (robin-map-cmake robin-map) diff --git a/contrib/usearch-cmake/CMakeLists.txt b/contrib/usearch-cmake/CMakeLists.txt index 29fbe57106c..0b6f60e106b 100644 --- a/contrib/usearch-cmake/CMakeLists.txt +++ b/contrib/usearch-cmake/CMakeLists.txt @@ -1,9 +1,7 @@ -set(USEARCH_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/usearch") -set(USEARCH_SOURCE_DIR "${USEARCH_PROJECT_DIR}/include") - set(FP16_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/FP16") set(ROBIN_MAP_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/robin-map") -set(SIMSIMD_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/SimSIMD-map") +set(SIMSIMD_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/SimSIMD") +set(USEARCH_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/usearch") add_library(_usearch INTERFACE) @@ -11,7 +9,7 @@ target_include_directories(_usearch SYSTEM INTERFACE ${FP16_PROJECT_DIR}/include ${ROBIN_MAP_PROJECT_DIR}/include ${SIMSIMD_PROJECT_DIR}/include - ${USEARCH_SOURCE_DIR}) + ${USEARCH_PROJECT_DIR}/include) add_library(ch_contrib::usearch ALIAS _usearch) target_compile_definitions(_usearch INTERFACE ENABLE_USEARCH) diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.h b/src/Storages/MergeTree/MergeTreeIndexUSearch.h index 41de94402c9..e6068790d22 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.h +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.h @@ -113,4 +113,3 @@ private: #endif - From d7211f9d12d33c54929fb24991fe7e46939be67d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Aug 2024 09:22:38 +0000 Subject: [PATCH 578/644] Fix CMake integration of usearch and annoy Registers usearch and annoy properly via configure_config.cmake and config.h.in like all other 3rd party libs, instead of (mis)using target_compile_definitions. --- contrib/annoy-cmake/CMakeLists.txt | 1 - contrib/usearch-cmake/CMakeLists.txt | 1 - src/Common/config.h.in | 2 ++ src/Processors/QueryPlan/ReadFromMergeTree.cpp | 6 ++++-- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 4 +++- src/Storages/MergeTree/MergeTreeIndexUSearch.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeIndexUSearch.h | 8 ++++++-- src/Storages/MergeTree/MergeTreeIndices.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeIndices.h | 5 +++-- src/configure_config.cmake | 6 ++++++ 11 files changed, 31 insertions(+), 16 deletions(-) diff --git a/contrib/annoy-cmake/CMakeLists.txt b/contrib/annoy-cmake/CMakeLists.txt index bdef7d92132..f6579c12412 100644 --- a/contrib/annoy-cmake/CMakeLists.txt +++ b/contrib/annoy-cmake/CMakeLists.txt @@ -20,5 +20,4 @@ add_library(_annoy INTERFACE) target_include_directories(_annoy SYSTEM INTERFACE ${ANNOY_SOURCE_DIR}) add_library(ch_contrib::annoy ALIAS _annoy) -target_compile_definitions(_annoy INTERFACE ENABLE_ANNOY) target_compile_definitions(_annoy INTERFACE ANNOYLIB_MULTITHREADED_BUILD) diff --git a/contrib/usearch-cmake/CMakeLists.txt b/contrib/usearch-cmake/CMakeLists.txt index 0b6f60e106b..6be622275ae 100644 --- a/contrib/usearch-cmake/CMakeLists.txt +++ b/contrib/usearch-cmake/CMakeLists.txt @@ -12,4 +12,3 @@ target_include_directories(_usearch SYSTEM INTERFACE ${USEARCH_PROJECT_DIR}/include) add_library(ch_contrib::usearch ALIAS _usearch) -target_compile_definitions(_usearch INTERFACE ENABLE_USEARCH) diff --git a/src/Common/config.h.in b/src/Common/config.h.in index e3f8882850f..0fa5f4313b2 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -58,6 +58,8 @@ #cmakedefine01 USE_FILELOG #cmakedefine01 USE_ODBC #cmakedefine01 USE_BLAKE3 +#cmakedefine01 USE_ANNOY +#cmakedefine01 USE_USEARCH #cmakedefine01 USE_SKIM #cmakedefine01 USE_PRQL #cmakedefine01 USE_ULID diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 901d7c61167..0ec7bde933c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -52,6 +52,8 @@ #include #include +#include "config.h" + using namespace DB; namespace @@ -1476,11 +1478,11 @@ static void buildIndexes( MergeTreeIndexConditionPtr condition; if (index_helper->isVectorSearch()) { -#ifdef ENABLE_ANNOY +#if USE_ANNOY if (const auto * annoy = typeid_cast(index_helper.get())) condition = annoy->createIndexCondition(query_info, context); #endif -#ifdef ENABLE_USEARCH +#if USE_USEARCH if (const auto * usearch = typeid_cast(index_helper.get())) condition = usearch->createIndexCondition(query_info, context); #endif diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index b68e48eeb3a..cec0e0926f0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -1,7 +1,7 @@ -#ifdef ENABLE_ANNOY - #include +#if USE_ANNOY + #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 282920c608e..8e0e0e621a0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -1,6 +1,8 @@ #pragma once -#ifdef ENABLE_ANNOY +#include "config.h" + +#if USE_ANNOY #include diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index efd9bb754e1..5a532803d84 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -1,10 +1,10 @@ -#ifdef ENABLE_USEARCH +#include + +#if USE_USEARCH #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wpass-failed" -#include - #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.h b/src/Storages/MergeTree/MergeTreeIndexUSearch.h index e6068790d22..6923ef2f807 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.h +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.h @@ -1,12 +1,16 @@ #pragma once -#ifdef ENABLE_USEARCH +#include "config.h" -#include +#if USE_USEARCH #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wpass-failed" + +#include + #include + #pragma clang diagnostic pop namespace DB diff --git a/src/Storages/MergeTree/MergeTreeIndices.cpp b/src/Storages/MergeTree/MergeTreeIndices.cpp index bded961db8e..32ac629e706 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.cpp +++ b/src/Storages/MergeTree/MergeTreeIndices.cpp @@ -127,12 +127,12 @@ MergeTreeIndexFactory::MergeTreeIndexFactory() registerCreator("hypothesis", hypothesisIndexCreator); registerValidator("hypothesis", hypothesisIndexValidator); -#ifdef ENABLE_ANNOY +#if USE_ANNOY registerCreator("annoy", annoyIndexCreator); registerValidator("annoy", annoyIndexValidator); #endif -#ifdef ENABLE_USEARCH +#if USE_USEARCH registerCreator("usearch", usearchIndexCreator); registerValidator("usearch", usearchIndexValidator); #endif diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index 1be73e1c811..355f1b69356 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -15,6 +15,7 @@ #include #include +#include "config.h" constexpr auto INDEX_FILE_PREFIX = "skp_idx_"; @@ -230,12 +231,12 @@ void bloomFilterIndexValidator(const IndexDescription & index, bool attach); MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index); void hypothesisIndexValidator(const IndexDescription & index, bool attach); -#ifdef ENABLE_ANNOY +#if USE_ANNOY MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index); void annoyIndexValidator(const IndexDescription & index, bool attach); #endif -#ifdef ENABLE_USEARCH +#if USE_USEARCH MergeTreeIndexPtr usearchIndexCreator(const IndexDescription& index); void usearchIndexValidator(const IndexDescription& index, bool attach); #endif diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 5b24f79ef6e..702875b1f40 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -164,6 +164,12 @@ endif() if (TARGET ch_contrib::bcrypt) set(USE_BCRYPT 1) endif() +if (TARGET ch_contrib::annoy) + set(USE_ANNOY 1) +endif() +if (TARGET ch_contrib::usearch) + set(USE_USEARCH 1) +endif() if (TARGET ch_contrib::ssh) set(USE_SSH 1) endif() From 7c419399216a714f9dcffe7835f951718851bceb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Aug 2024 09:36:39 +0000 Subject: [PATCH 579/644] Fix test results (no analyzer support yet ...) --- tests/queries/0_stateless/02354_vector_search_queries.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02354_vector_search_queries.sql b/tests/queries/0_stateless/02354_vector_search_queries.sql index 64051aa8544..87d27be0ea4 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.sql +++ b/tests/queries/0_stateless/02354_vector_search_queries.sql @@ -8,6 +8,8 @@ SET allow_experimental_annoy_index = 1; SET allow_experimental_usearch_index = 1; +SET enable_analyzer = 0; + SELECT 'ARRAY, 10 rows, index_granularity = 8192, GRANULARITY = 1 million --> 1 granule, 1 indexed block'; DROP TABLE IF EXISTS tab_annoy; From 218421c255cadbe65406e6a040d05942cc4efc3e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Aug 2024 09:47:50 +0000 Subject: [PATCH 580/644] Remove Annoy indexes Annoy indexes fell out of favor in the community, at least when it comes to vector databases. Such indexes work okay-ish low dimensions but they suffers badly from a curse of dimensionality which makes them inapt for a high number of dimensions. Now that Annoy is gone, issue (*) also disappears and we can drop 'no-ubsan', 'no-cpu-aarch64', and 'no-asan' from tests. (*) spotify/annoy#456 --- .gitmodules | 3 - contrib/CMakeLists.txt | 1 - contrib/annoy | 1 - contrib/annoy-cmake/CMakeLists.txt | 23 - .../mergetree-family/annindexes.md | 87 +--- src/CMakeLists.txt | 4 - src/Common/config.h.in | 1 - src/Core/Settings.h | 6 +- src/Databases/DatabaseReplicated.cpp | 1 - src/Interpreters/InterpreterCreateQuery.cpp | 2 - src/Parsers/ASTIndexDeclaration.h | 1 - src/Parsers/ParserCreateIndexQuery.cpp | 4 +- src/Parsers/ParserCreateQuery.cpp | 4 +- .../QueryPlan/ReadFromMergeTree.cpp | 5 - .../MergeTree/MergeTreeIOSettings.cpp | 1 - src/Storages/MergeTree/MergeTreeIOSettings.h | 2 - .../MergeTree/MergeTreeIndexAnnoy.cpp | 416 ------------------ src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 114 ----- src/Storages/MergeTree/MergeTreeIndices.cpp | 4 - src/Storages/MergeTree/MergeTreeIndices.h | 5 - src/configure_config.cmake | 3 - .../02354_vector_search_bugs.reference | 10 - .../0_stateless/02354_vector_search_bugs.sql | 75 +--- ...ector_search_default_granularity.reference | 6 +- ...2354_vector_search_default_granularity.sql | 21 +- ...r_search_index_creation_negative.reference | 2 +- ..._vector_search_index_creation_negative.sql | 22 +- .../02354_vector_search_queries.reference | 99 ----- .../02354_vector_search_queries.sql | 119 +---- 29 files changed, 32 insertions(+), 1010 deletions(-) delete mode 160000 contrib/annoy delete mode 100644 contrib/annoy-cmake/CMakeLists.txt delete mode 100644 src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp delete mode 100644 src/Storages/MergeTree/MergeTreeIndexAnnoy.h diff --git a/.gitmodules b/.gitmodules index 7fdfb1103c5..0a66031de8d 100644 --- a/.gitmodules +++ b/.gitmodules @@ -230,9 +230,6 @@ [submodule "contrib/minizip-ng"] path = contrib/minizip-ng url = https://github.com/zlib-ng/minizip-ng -[submodule "contrib/annoy"] - path = contrib/annoy - url = https://github.com/ClickHouse/annoy [submodule "contrib/qpl"] path = contrib/qpl url = https://github.com/intel/qpl diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 98b992e1080..dc2ad2a3150 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -205,7 +205,6 @@ add_contrib (morton-nd-cmake morton-nd) if (ARCH_S390X) add_contrib(crc32-s390x-cmake crc32-s390x) endif() -add_contrib (annoy-cmake annoy) option(ENABLE_USEARCH "Enable USearch" ${ENABLE_LIBRARIES}) if (ENABLE_USEARCH) diff --git a/contrib/annoy b/contrib/annoy deleted file mode 160000 index f2ac8e7b48f..00000000000 --- a/contrib/annoy +++ /dev/null @@ -1 +0,0 @@ -Subproject commit f2ac8e7b48f9a9cf676d3b58286e5455aba8e956 diff --git a/contrib/annoy-cmake/CMakeLists.txt b/contrib/annoy-cmake/CMakeLists.txt deleted file mode 100644 index f6579c12412..00000000000 --- a/contrib/annoy-cmake/CMakeLists.txt +++ /dev/null @@ -1,23 +0,0 @@ -option(ENABLE_ANNOY "Enable Annoy index support" ${ENABLE_LIBRARIES}) - -# Annoy index should be disabled with undefined sanitizer. Because of memory storage optimizations -# (https://github.com/ClickHouse/annoy/blob/9d8a603a4cd252448589e84c9846f94368d5a289/src/annoylib.h#L442-L463) -# UBSan fails and leads to crash. Simmilar issue is already opened in Annoy repo -# https://github.com/spotify/annoy/issues/456 -# Problem with aligment can lead to errors like -# (https://stackoverflow.com/questions/46790550/c-undefined-behavior-strict-aliasing-rule-or-incorrect-alignment) -# or will lead to crash on arm https://developer.arm.com/documentation/ka003038/latest -# This issues should be resolved before annoy became non-experimental (--> setting "allow_experimental_annoy_index") -if ((NOT ENABLE_ANNOY) OR (SANITIZE STREQUAL "undefined") OR (ARCH_AARCH64)) - message (STATUS "Not using annoy") - return() -endif() - -set(ANNOY_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/annoy") -set(ANNOY_SOURCE_DIR "${ANNOY_PROJECT_DIR}/src") - -add_library(_annoy INTERFACE) -target_include_directories(_annoy SYSTEM INTERFACE ${ANNOY_SOURCE_DIR}) - -add_library(ch_contrib::annoy ALIAS _annoy) -target_compile_definitions(_annoy INTERFACE ANNOYLIB_MULTITHREADED_BUILD) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 5a81313f62e..9a80542522e 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -126,81 +126,8 @@ was specified for ANN indexes, the default value is 100 million. # Available ANN Indexes {#available_ann_indexes} -- [Annoy](/docs/en/engines/table-engines/mergetree-family/annindexes.md#annoy-annoy) - - [USearch](/docs/en/engines/table-engines/mergetree-family/annindexes.md#usearch-usearch) -## Annoy {#annoy} - -Annoy indexes are currently experimental, to use them you first need to `SET allow_experimental_annoy_index = 1`. They are also currently -disabled on ARM due to memory safety problems with the algorithm. - -This type of ANN index is based on the [Annoy library](https://github.com/spotify/annoy) which recursively divides the space into random -linear surfaces (lines in 2D, planes in 3D etc.). - -
- -
- -Syntax to create an Annoy index over an [Array(Float32)](../../../sql-reference/data-types/array.md) column: - -```sql -CREATE TABLE table_with_annoy_index -( - id Int64, - vectors Array(Float32), - INDEX [ann_index_name] vectors TYPE annoy([Distance[, NumTrees]]) [GRANULARITY N] -) -ENGINE = MergeTree -ORDER BY id; -``` - -Annoy currently supports two distance functions: -- `L2Distance`, also called Euclidean distance, is the length of a line segment between two points in Euclidean space - ([Wikipedia](https://en.wikipedia.org/wiki/Euclidean_distance)). -- `cosineDistance`, also called cosine similarity, is the cosine of the angle between two (non-zero) vectors - ([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)). - -For normalized data, `L2Distance` is usually a better choice, otherwise `cosineDistance` is recommended to compensate for scale. If no -distance function was specified during index creation, `L2Distance` is used as default. - -Parameter `NumTrees` is the number of trees which the algorithm creates (default if not specified: 100). Higher values of `NumTree` mean -more accurate search results but slower index creation / query times (approximately linearly) as well as larger index sizes. - -:::note -All arrays must have same length. To avoid errors, you can use a -[CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints), for example, `CONSTRAINT constraint_name_1 CHECK -length(vectors) = 256`. Also, empty `Arrays` and unspecified `Array` values in INSERT statements (i.e. default values) are not supported. -::: - -The creation of Annoy indexes (whenever a new part is build, e.g. at the end of a merge) is a relatively slow process. You can increase -setting `max_threads_for_annoy_index_creation` (default: 4) which controls how many threads are used to create an Annoy index. Please be -careful with this setting, it is possible that multiple indexes are created in parallel in which case there can be overparallelization. - -Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many tree nodes are inspected during SELECTs. Larger -values mean more accurate results at the cost of longer query runtime: - -```sql -SELECT * -FROM table_name -ORDER BY L2Distance(vectors, Point) -LIMIT N -SETTINGS annoy_index_search_k_nodes=100; -``` - -:::note -The Annoy index currently does not work with per-table, non-default `index_granularity` settings (see -[here](https://github.com/ClickHouse/ClickHouse/pull/51325#issuecomment-1605920475)). If necessary, the value must be changed in config.xml. -::: - ## USearch {#usearch} This type of ANN index is based on the [USearch library](https://github.com/unum-cloud/usearch), which implements the [HNSW @@ -211,6 +138,8 @@ that are expensive to load and compare. The library also has several hardware-sp distance computations on modern Arm (NEON and SVE) and x86 (AVX2 and AVX-512) CPUs and OS-specific optimizations to allow efficient navigation around immutable persistent files, without loading them into RAM. +USearch indexes are currently experimental, to use them you first need to `SET allow_experimental_usearch_index = 1`. +
-
- -Syntax to create an USearch index over an [Array](../../../sql-reference/data-types/array.md) column: - -```sql -CREATE TABLE table_with_usearch_index -( - id Int64, - vectors Array(Float32), - INDEX [ann_index_name] vectors TYPE usearch([Distance[, ScalarKind]]) [GRANULARITY N] -) -ENGINE = MergeTree -ORDER BY id; -``` - -USearch currently supports two distance functions: -- `L2Distance`, also called Euclidean distance, is the length of a line segment between two points in Euclidean space - ([Wikipedia](https://en.wikipedia.org/wiki/Euclidean_distance)). -- `cosineDistance`, also called cosine similarity, is the cosine of the angle between two (non-zero) vectors - ([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)). - -USearch allows storing the vectors in reduced precision formats. Supported scalar kinds are `f64`, `f32`, `f16` or `i8`. If no scalar kind -was specified during index creation, `f16` is used as default. - -For normalized data, `L2Distance` is usually a better choice, otherwise `cosineDistance` is recommended to compensate for scale. If no -distance function was specified during index creation, `L2Distance` is used as default. - -:::note -All arrays must have same length. To avoid errors, you can use a -[CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints), for example, `CONSTRAINT constraint_name_1 CHECK -length(vectors) = 256`. Also, empty `Arrays` and unspecified `Array` values in INSERT statements (i.e. default values) are not supported. -::: - -:::note -The USearch index currently does not work with per-table, non-default `index_granularity` settings (see -[here](https://github.com/ClickHouse/ClickHouse/pull/51325#issuecomment-1605920475)). If necessary, the value must be changed in config.xml. -::: - diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 23ad12bb017..e9f3b95dbc1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -907,9 +907,9 @@ class IColumn; M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions", 0) \ M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ M(Bool, allow_experimental_time_series_table, false, "Allows experimental TimeSeries table engine", 0) \ + M(Bool, allow_experimental_vector_similarity_index, false, "Allow experimental vector similarity index", 0) \ M(Bool, allow_experimental_variant_type, false, "Allow Variant data type", 0) \ M(Bool, allow_experimental_dynamic_type, false, "Allow Dynamic data type", 0) \ - M(Bool, allow_experimental_usearch_index, false, "Allows to use USearch index. Disabled by default because this feature is experimental", 0) \ M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \ M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \ M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \ @@ -1036,6 +1036,7 @@ class IColumn; MAKE_OBSOLETE(M, Bool, allow_experimental_annoy_index, false) \ MAKE_OBSOLETE(M, UInt64, max_threads_for_annoy_index_creation, 4) \ MAKE_OBSOLETE(M, Int64, annoy_index_search_k_nodes, -1) \ + MAKE_OBSOLETE(M, Bool, allow_experimental_usearch_index, false) \ MAKE_OBSOLETE(M, Bool, optimize_move_functions_out_of_any, false) \ MAKE_OBSOLETE(M, Bool, allow_experimental_undrop_table_query, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_s3queue, true) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 511723f1873..8fabd1ecf91 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -85,6 +85,7 @@ static std::initializer_listsetSetting("allow_experimental_object_type", 1); query_context->setSetting("allow_experimental_variant_type", 1); query_context->setSetting("allow_experimental_dynamic_type", 1); - query_context->setSetting("allow_experimental_usearch_index", 1); + query_context->setSetting("allow_experimental_vector_similarity_index", 1); query_context->setSetting("allow_experimental_bigint_types", 1); query_context->setSetting("allow_experimental_window_functions", 1); query_context->setSetting("allow_experimental_geo_types", 1); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a1ffcf07588..95143031707 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -787,8 +787,8 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti if (index_desc.type == INVERTED_INDEX_NAME && !settings.allow_experimental_inverted_index) throw Exception(ErrorCodes::ILLEGAL_INDEX, "Please use index type 'full_text' instead of 'inverted'"); /// ---- - if (index_desc.type == "usearch" && !settings.allow_experimental_usearch_index) - throw Exception(ErrorCodes::INCORRECT_QUERY, "USearch index is disabled. Turn on allow_experimental_usearch_index"); + if (index_desc.type == "vector_similarity" && !settings.allow_experimental_vector_similarity_index) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Vector similarity index is disabled. Turn on allow_experimental_vector_similarity_index"); properties.indices.push_back(index_desc); } diff --git a/src/Parsers/ASTIndexDeclaration.h b/src/Parsers/ASTIndexDeclaration.h index 90645f12b7c..72f3f017a99 100644 --- a/src/Parsers/ASTIndexDeclaration.h +++ b/src/Parsers/ASTIndexDeclaration.h @@ -13,7 +13,7 @@ class ASTIndexDeclaration : public IAST { public: static const auto DEFAULT_INDEX_GRANULARITY = 1uz; - static const auto DEFAULT_USEARCH_INDEX_GRANULARITY = 100'000'000uz; + static const auto DEFAULT_VECTOR_SIMILARITY_INDEX_GRANULARITY = 100'000'000uz; ASTIndexDeclaration(ASTPtr expression, ASTPtr type, const String & name_); diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index e7cfd753f99..ed89b80edca 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -89,8 +89,8 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected else { auto index_type = index->getType(); - if (index_type && index_type->name == "usearch") - index->granularity = ASTIndexDeclaration::DEFAULT_USEARCH_INDEX_GRANULARITY; + if (index_type && index_type->name == "vector_similarity") + index->granularity = ASTIndexDeclaration::DEFAULT_VECTOR_SIMILARITY_INDEX_GRANULARITY; else index->granularity = ASTIndexDeclaration::DEFAULT_INDEX_GRANULARITY; } diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index b31fe21c4cc..cc4e02f46a3 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -214,8 +214,8 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe else { auto index_type = index->getType(); - if (index_type->name == "usearch") - index->granularity = ASTIndexDeclaration::DEFAULT_USEARCH_INDEX_GRANULARITY; + if (index_type->name == "vector_similarity") + index->granularity = ASTIndexDeclaration::DEFAULT_VECTOR_SIMILARITY_INDEX_GRANULARITY; else index->granularity = ASTIndexDeclaration::DEFAULT_INDEX_GRANULARITY; } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 3324cc4e42a..1f30725b4d0 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -24,7 +24,7 @@ #include #include #include -#include +#include #include #include #include @@ -1478,8 +1478,8 @@ static void buildIndexes( if (index_helper->isVectorSimilarityIndex()) { #if USE_USEARCH - if (const auto * usearch_index = typeid_cast(index_helper.get())) - condition = usearch_index->createIndexCondition(query_info, context); + if (const auto * vector_similarity_index = typeid_cast(index_helper.get())) + condition = vector_similarity_index->createIndexCondition(query_info, context); #endif if (!condition) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown vector search index {}", index_helper->index.name); diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp similarity index 76% rename from src/Storages/MergeTree/MergeTreeIndexUSearch.cpp rename to src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 1aa6c9c14d4..6f3b1b043cd 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -1,4 +1,4 @@ -#include +#include #if USE_USEARCH @@ -90,7 +90,7 @@ void USearchIndexWithSerialization::serialize(WriteBuffer & ostr) const auto result = Base::save_to_stream(callback); if (result.error) - throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not save USearch index, error: " + String(result.error.release())); + throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not save vector similarity index, error: " + String(result.error.release())); } void USearchIndexWithSerialization::deserialize(ReadBuffer & istr) @@ -104,7 +104,7 @@ void USearchIndexWithSerialization::deserialize(ReadBuffer & istr) auto result = Base::load_from_stream(callback); if (result.error) /// See the comment in MergeTreeIndexGranuleVectorSimilarity::deserializeBinary why we throw here - throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not load USearch index, error: " + String(result.error.release()) + " Please drop the index and create it again."); + throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not load vector similarity index, error: " + String(result.error.release()) + " Please drop the index and create it again."); } USearchIndexWithSerialization::Statistics USearchIndexWithSerialization::getStatistics() const @@ -121,16 +121,16 @@ USearchIndexWithSerialization::Statistics USearchIndexWithSerialization::getStat return statistics; } -MergeTreeIndexGranuleUSearch::MergeTreeIndexGranuleUSearch( +MergeTreeIndexGranuleVectorSimilarity::MergeTreeIndexGranuleVectorSimilarity( const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_) - : MergeTreeIndexGranuleUSearch(index_name_, index_sample_block_, metric_kind_, scalar_kind_, nullptr) + : MergeTreeIndexGranuleVectorSimilarity(index_name_, index_sample_block_, metric_kind_, scalar_kind_, nullptr) { } -MergeTreeIndexGranuleUSearch::MergeTreeIndexGranuleUSearch( +MergeTreeIndexGranuleVectorSimilarity::MergeTreeIndexGranuleVectorSimilarity( const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, @@ -144,7 +144,7 @@ MergeTreeIndexGranuleUSearch::MergeTreeIndexGranuleUSearch( { } -void MergeTreeIndexGranuleUSearch::serializeBinary(WriteBuffer & ostr) const +void MergeTreeIndexGranuleVectorSimilarity::serializeBinary(WriteBuffer & ostr) const { if (empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty minmax index {}", backQuote(index_name)); @@ -158,18 +158,18 @@ void MergeTreeIndexGranuleUSearch::serializeBinary(WriteBuffer & ostr) const index->serialize(ostr); auto statistics = index->getStatistics(); - LOG_TRACE(logger, "Wrote USearch index: max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}", + LOG_TRACE(logger, "Wrote vector similarity index: max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}", statistics.max_level, statistics.connectivity, statistics.size, statistics.capacity, ReadableSize(statistics.memory_usage)); } -void MergeTreeIndexGranuleUSearch::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion /*version*/) +void MergeTreeIndexGranuleVectorSimilarity::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion /*version*/) { UInt64 file_version; readIntBinary(file_version, istr); if (file_version != FILE_FORMAT_VERSION) throw Exception( ErrorCodes::FORMAT_VERSION_TOO_OLD, - "USearch index could not be loaded because its version is too old (current version: {}, persisted version: {}). Please drop the index and create it again.", + "Vector similarity index could not be loaded because its version is too old (current version: {}, persisted version: {}). Please drop the index and create it again.", FILE_FORMAT_VERSION, file_version); /// More fancy error handling would be: Set a flag on the index that it failed to load. During usage return all granules, i.e. /// behave as if the index does not exist. Since format changes are expected to happen only rarely and it is "only" an index, keep it simple for now. @@ -181,11 +181,11 @@ void MergeTreeIndexGranuleUSearch::deserializeBinary(ReadBuffer & istr, MergeTre index->deserialize(istr); auto statistics = index->getStatistics(); - LOG_TRACE(logger, "Loaded USearch index: max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}", + LOG_TRACE(logger, "Loaded vector similarity index: max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}", statistics.max_level, statistics.connectivity, statistics.size, statistics.capacity, ReadableSize(statistics.memory_usage)); } -MergeTreeIndexAggregatorUSearch::MergeTreeIndexAggregatorUSearch( +MergeTreeIndexAggregatorVectorSimilarity::MergeTreeIndexAggregatorVectorSimilarity( const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, @@ -197,14 +197,14 @@ MergeTreeIndexAggregatorUSearch::MergeTreeIndexAggregatorUSearch( { } -MergeTreeIndexGranulePtr MergeTreeIndexAggregatorUSearch::getGranuleAndReset() +MergeTreeIndexGranulePtr MergeTreeIndexAggregatorVectorSimilarity::getGranuleAndReset() { - auto granule = std::make_shared(index_name, index_sample_block, metric_kind, scalar_kind, index); + auto granule = std::make_shared(index_name, index_sample_block, metric_kind, scalar_kind, index); index = nullptr; return granule; } -void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t * pos, size_t limit) +void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_t * pos, size_t limit) { if (*pos >= block.rows()) throw Exception( @@ -239,8 +239,8 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t * pos, if (column_array->empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Array is unexpectedly empty"); - /// The Usearch algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays - /// are INSERTed into an Usearch-indexed column or if no value was specified at all in which case the arrays take on their default + /// The vector similarity algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays + /// are INSERTed into an vector-similarity-indexed column or if no value was specified at all in which case the arrays take on their default /// values which is also empty. if (column_array->isDefaultAt(0)) throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); @@ -262,13 +262,13 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t * pos, /// Reserving space is mandatory if (!index->reserve(roundUpToPowerOfTwoOrZero(index->size() + num_rows))) - throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for usearch index"); + throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for vector similarity index"); for (size_t current_row = 0; current_row < num_rows; ++current_row) { auto rc = index->add(static_cast(index->size()), &column_array_data_float_data[column_array_offsets[current_row - 1]]); if (!rc) - throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not add data to USearch index, error: " + String(rc.error.release())); + throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index, error: " + String(rc.error.release())); ProfileEvents::increment(ProfileEvents::USearchAddCount); ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, rc.visited_members); @@ -281,7 +281,7 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t * pos, *pos += rows_read; } -MergeTreeIndexConditionUSearch::MergeTreeIndexConditionUSearch( +MergeTreeIndexConditionVectorSimilarity::MergeTreeIndexConditionVectorSimilarity( const IndexDescription & /*index_description*/, const SelectQueryInfo & query, unum::usearch::metric_kind_t metric_kind_, @@ -291,12 +291,12 @@ MergeTreeIndexConditionUSearch::MergeTreeIndexConditionUSearch( { } -bool MergeTreeIndexConditionUSearch::mayBeTrueOnGranule(MergeTreeIndexGranulePtr) const +bool MergeTreeIndexConditionVectorSimilarity::mayBeTrueOnGranule(MergeTreeIndexGranulePtr) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "mayBeTrueOnGranule is not supported for ANN skip indexes"); } -bool MergeTreeIndexConditionUSearch::alwaysUnknownOrTrue() const +bool MergeTreeIndexConditionVectorSimilarity::alwaysUnknownOrTrue() const { String index_distance_function; switch (metric_kind) @@ -308,14 +308,14 @@ bool MergeTreeIndexConditionUSearch::alwaysUnknownOrTrue() const return vector_similarity_condition.alwaysUnknownOrTrue(index_distance_function); } -std::vector MergeTreeIndexConditionUSearch::getUsefulRanges(MergeTreeIndexGranulePtr granule_) const +std::vector MergeTreeIndexConditionVectorSimilarity::getUsefulRanges(MergeTreeIndexGranulePtr granule_) const { const UInt64 limit = vector_similarity_condition.getLimit(); const UInt64 index_granularity = vector_similarity_condition.getIndexGranularity(); const std::vector reference_vector = vector_similarity_condition.getReferenceVector(); - const auto granule = std::dynamic_pointer_cast(granule_); + const auto granule = std::dynamic_pointer_cast(granule_); if (granule == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule has the wrong type"); @@ -328,7 +328,7 @@ std::vector MergeTreeIndexConditionUSearch::getUsefulRanges(MergeTreeInd auto result = index->search(reference_vector.data(), limit); if (result.error) - throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not search in USearch index, error: " + String(result.error.release())); + throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not search in vector similarity index, error: " + String(result.error.release())); ProfileEvents::increment(ProfileEvents::USearchSearchCount); ProfileEvents::increment(ProfileEvents::USearchSearchVisitedMembers, result.visited_members); @@ -350,34 +350,34 @@ std::vector MergeTreeIndexConditionUSearch::getUsefulRanges(MergeTreeInd return granules; } -MergeTreeIndexUSearch::MergeTreeIndexUSearch(const IndexDescription & index_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_) +MergeTreeIndexVectorSimilarity::MergeTreeIndexVectorSimilarity(const IndexDescription & index_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_) : IMergeTreeIndex(index_) , metric_kind(metric_kind_) , scalar_kind(scalar_kind_) { } -MergeTreeIndexGranulePtr MergeTreeIndexUSearch::createIndexGranule() const +MergeTreeIndexGranulePtr MergeTreeIndexVectorSimilarity::createIndexGranule() const { - return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind); + return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind); } -MergeTreeIndexAggregatorPtr MergeTreeIndexUSearch::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const +MergeTreeIndexAggregatorPtr MergeTreeIndexVectorSimilarity::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const { - return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind); + return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind); } -MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const +MergeTreeIndexConditionPtr MergeTreeIndexVectorSimilarity::createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const { - return std::make_shared(index, query, metric_kind, context); + return std::make_shared(index, query, metric_kind, context); }; -MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const ActionsDAG *, ContextPtr) const +MergeTreeIndexConditionPtr MergeTreeIndexVectorSimilarity::createIndexCondition(const ActionsDAG *, ContextPtr) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeIndexAnnoy cannot be created with ActionsDAG"); } -MergeTreeIndexPtr usearchIndexCreator(const IndexDescription & index) +MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index) { static constexpr auto default_metric_kind = unum::usearch::metric_kind_t::l2sq_k; auto metric_kind = default_metric_kind; @@ -389,25 +389,25 @@ MergeTreeIndexPtr usearchIndexCreator(const IndexDescription & index) if (index.arguments.size() > 1) scalar_kind = quantizationToScalarKind.at(index.arguments[1].safeGet()); - return std::make_shared(index, metric_kind, scalar_kind); + return std::make_shared(index, metric_kind, scalar_kind); } -void usearchIndexValidator(const IndexDescription & index, bool /* attach */) +void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* attach */) { - /// Check number and type of USearch index arguments: + /// Check number and type of index arguments: if (index.arguments.size() > 2) - throw Exception(ErrorCodes::INCORRECT_QUERY, "USearch index must not have more than one parameters"); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Vector similarity index must not have more than one parameters"); if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "First argument of USearch index (distance function) must be of type String"); + throw Exception(ErrorCodes::INCORRECT_QUERY, "First argument of vector similarity index (distance function) must be of type String"); if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Second argument of USearch index (scalar type) must be of type String"); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Second argument of vector similarity index (scalar type) must be of type String"); /// Check that the index is created on a single column if (index.column_names.size() != 1 || index.data_types.size() != 1) - throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "USearch indexes must be created on a single column"); + throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Vector similarity indexes must be created on a single column"); /// Check that a supported metric was passed as first argument @@ -420,16 +420,15 @@ void usearchIndexValidator(const IndexDescription & index, bool /* attach */) throw Exception(ErrorCodes::INCORRECT_DATA, "Unrecognized scalar kind (second argument) for vector index. Supported kinds are: {}", keysAsString(quantizationToScalarKind)); /// Check data type of indexed column: - DataTypePtr data_type = index.sample_block.getDataTypes()[0]; if (const auto * data_type_array = typeid_cast(data_type.get())) { TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); if (!WhichDataType(nested_type_index).isFloat32()) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "USearch can only be created on columns of type Array(Float32)"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity index can only be created on columns of type Array(Float32)"); } else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "USearch can only be created on columns of type Array(Float32)"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity index can only be created on columns of type Array(Float32)"); } } diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.h b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h similarity index 84% rename from src/Storages/MergeTree/MergeTreeIndexUSearch.h rename to src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h index d4df6658a90..95ea3cd5240 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.h +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h @@ -48,22 +48,22 @@ public: using USearchIndexWithSerializationPtr = std::shared_ptr; -struct MergeTreeIndexGranuleUSearch final : public IMergeTreeIndexGranule +struct MergeTreeIndexGranuleVectorSimilarity final : public IMergeTreeIndexGranule { - MergeTreeIndexGranuleUSearch( + MergeTreeIndexGranuleVectorSimilarity( const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_); - MergeTreeIndexGranuleUSearch( + MergeTreeIndexGranuleVectorSimilarity( const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_, USearchIndexWithSerializationPtr index_); - ~MergeTreeIndexGranuleUSearch() override = default; + ~MergeTreeIndexGranuleVectorSimilarity() override = default; void serializeBinary(WriteBuffer & ostr) const override; void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; @@ -76,7 +76,7 @@ struct MergeTreeIndexGranuleUSearch final : public IMergeTreeIndexGranule const unum::usearch::scalar_kind_t scalar_kind; USearchIndexWithSerializationPtr index; - LoggerPtr logger = getLogger("USearchIndex"); + LoggerPtr logger = getLogger("VectorSimilarityIndex"); private: /// The version of the persistence format of USearch index. Increment whenever you change the format. @@ -87,15 +87,15 @@ private: }; -struct MergeTreeIndexAggregatorUSearch final : IMergeTreeIndexAggregator +struct MergeTreeIndexAggregatorVectorSimilarity final : IMergeTreeIndexAggregator { - MergeTreeIndexAggregatorUSearch( + MergeTreeIndexAggregatorVectorSimilarity( const String & index_name_, const Block & index_sample_block, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_); - ~MergeTreeIndexAggregatorUSearch() override = default; + ~MergeTreeIndexAggregatorVectorSimilarity() override = default; bool empty() const override { return !index || index->size() == 0; } MergeTreeIndexGranulePtr getGranuleAndReset() override; @@ -109,16 +109,16 @@ struct MergeTreeIndexAggregatorUSearch final : IMergeTreeIndexAggregator }; -class MergeTreeIndexConditionUSearch final : public IMergeTreeIndexCondition +class MergeTreeIndexConditionVectorSimilarity final : public IMergeTreeIndexCondition { public: - MergeTreeIndexConditionUSearch( + MergeTreeIndexConditionVectorSimilarity( const IndexDescription & index_description, const SelectQueryInfo & query, unum::usearch::metric_kind_t metric_kind_, ContextPtr context); - ~MergeTreeIndexConditionUSearch() override = default; + ~MergeTreeIndexConditionVectorSimilarity() override = default; bool alwaysUnknownOrTrue() const override; bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr granule) const override; @@ -130,15 +130,15 @@ private: }; -class MergeTreeIndexUSearch : public IMergeTreeIndex +class MergeTreeIndexVectorSimilarity : public IMergeTreeIndex { public: - MergeTreeIndexUSearch( + MergeTreeIndexVectorSimilarity( const IndexDescription & index_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_); - ~MergeTreeIndexUSearch() override = default; + ~MergeTreeIndexVectorSimilarity() override = default; MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; diff --git a/src/Storages/MergeTree/MergeTreeIndices.cpp b/src/Storages/MergeTree/MergeTreeIndices.cpp index f07449f762c..89aed7873a4 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.cpp +++ b/src/Storages/MergeTree/MergeTreeIndices.cpp @@ -129,8 +129,8 @@ MergeTreeIndexFactory::MergeTreeIndexFactory() registerValidator("hypothesis", hypothesisIndexValidator); #if USE_USEARCH - registerCreator("usearch", usearchIndexCreator); - registerValidator("usearch", usearchIndexValidator); + registerCreator("vector_similarity", vectorSimilarityIndexCreator); + registerValidator("vector_similarity", vectorSimilarityIndexValidator); #endif registerCreator("inverted", fullTextIndexCreator); diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index 3dee79aae85..48ef2a4739e 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -239,8 +239,8 @@ MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index); void hypothesisIndexValidator(const IndexDescription & index, bool attach); #if USE_USEARCH -MergeTreeIndexPtr usearchIndexCreator(const IndexDescription & index); -void usearchIndexValidator(const IndexDescription & index, bool attach); +MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index); +void vectorSimilarityIndexValidator(const IndexDescription & index, bool attach); #endif MergeTreeIndexPtr fullTextIndexCreator(const IndexDescription & index); diff --git a/tests/queries/0_stateless/02354_vector_search_bugs.sql b/tests/queries/0_stateless/02354_vector_search_bugs.sql index de36683ede1..2ef75d0a7fe 100644 --- a/tests/queries/0_stateless/02354_vector_search_bugs.sql +++ b/tests/queries/0_stateless/02354_vector_search_bugs.sql @@ -2,21 +2,21 @@ -- Tests various bugs and special cases for vector indexes. -SET allow_experimental_usearch_index = 1; +SET allow_experimental_vector_similarity_index = 1; SET enable_analyzer = 1; -- 0 vs. 1 produce slightly different error codes, make it future-proof DROP TABLE IF EXISTS tab; SELECT 'Issue #52258: Empty Arrays or Arrays with default values are rejected'; -CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree() ORDER BY id; +CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree() ORDER BY id; INSERT INTO tab VALUES (1, []); -- { serverError INCORRECT_DATA } INSERT INTO tab (id) VALUES (1); -- { serverError INCORRECT_DATA } DROP TABLE tab; SELECT 'It is possible to create parts with different Array vector sizes but there will be an error at query time'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; SYSTEM STOP MERGES tab; INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2]); INSERT INTO tab values (2, [2.2, 2.3, 2.4]) (3, [3.1, 3.2, 3.3]); @@ -31,7 +31,7 @@ DROP TABLE tab; SELECT 'Correctness of index with > 1 mark'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0, min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192; -- disable adaptive granularity due to bug +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0, min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192; -- disable adaptive granularity due to bug INSERT INTO tab SELECT number, [toFloat32(number), 0.0] from numbers(10000); WITH [1.0, 0.0] AS reference_vec diff --git a/tests/queries/0_stateless/02354_vector_search_default_granularity.sql b/tests/queries/0_stateless/02354_vector_search_default_granularity.sql index ff659b56033..a19a0d17536 100644 --- a/tests/queries/0_stateless/02354_vector_search_default_granularity.sql +++ b/tests/queries/0_stateless/02354_vector_search_default_granularity.sql @@ -2,17 +2,17 @@ -- Tests that vector search indexes use a (non-standard) index granularity of 100 mio by default. -SET allow_experimental_usearch_index = 1; +SET allow_experimental_vector_similarity_index = 1; -- After CREATE TABLE DROP TABLE IF EXISTS tab; -CREATE TABLE tab (id Int32, vec Array(Float32), INDEX idx(vec) TYPE usearch) ENGINE = MergeTree ORDER BY id; +CREATE TABLE tab (id Int32, vec Array(Float32), INDEX idx(vec) TYPE vector_similarity) ENGINE = MergeTree ORDER BY id; SELECT granularity FROM system.data_skipping_indices WHERE database = currentDatabase() AND table = 'tab' AND name = 'idx'; -- After ALTER TABLE DROP TABLE tab; CREATE TABLE tab (id Int32, vec Array(Float32)) ENGINE = MergeTree ORDER BY id; -ALTER TABLE tab ADD INDEX idx(vec) TYPE usearch; +ALTER TABLE tab ADD INDEX idx(vec) TYPE vector_similarity; SELECT granularity FROM system.data_skipping_indices WHERE database = currentDatabase() AND table = 'tab' AND name = 'idx'; DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_detach_attach.sql b/tests/queries/0_stateless/02354_vector_search_detach_attach.sql index 92e8efd918b..36241dfabf7 100644 --- a/tests/queries/0_stateless/02354_vector_search_detach_attach.sql +++ b/tests/queries/0_stateless/02354_vector_search_detach_attach.sql @@ -2,10 +2,10 @@ -- Tests that vector similarity indexes can be detached/attached. -SET allow_experimental_usearch_index = 1; +SET allow_experimental_vector_similarity_index = 1; DROP TABLE IF EXISTS tab; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [0.0, 2.0]), (6, [0.0, 2.1]), (7, [0.0, 2.2]), (8, [0.0, 2.3]), (9, [0.0, 2.4]); DETACH TABLE tab SYNC; diff --git a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql index 60bd54d1dbe..912f7d7fcae 100644 --- a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql +++ b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql @@ -2,36 +2,36 @@ -- Tests that various conditions are checked during creation of vector search indexes. -SET allow_experimental_usearch_index = 1; +SET allow_experimental_vector_similarity_index = 1; DROP TABLE IF EXISTS tab; SELECT 'At most two index arguments'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch('too', 'many', 'args')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('too', 'many', 'args')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } SELECT '1st argument (distance function) must be String'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } SELECT 'Unsupported distance functions are rejected'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch('invalidDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('invalidDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } SELECT '2nd argument (scalar kind) must be String'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } SELECT 'Unsupported scalar kinds are rejected'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch('L2Distance', 'invalidKind')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('L2Distance', 'invalidKind')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } SELECT 'Must be created on single column'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx (vec, id) TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx (vec, id) TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } SELECT 'Must be created on Array(Float32) columns'; SET allow_suspicious_low_cardinality_types = 1; -CREATE TABLE tab(id Int32, vec Float32, INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, vec LowCardinality(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, vec Nullable(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec Float32, INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec LowCardinality(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec Nullable(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } SELECT 'Rejects INSERTs of Arrays with different sizes'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2, 3.3]); -- { serverError INCORRECT_DATA } DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_queries.reference b/tests/queries/0_stateless/02354_vector_search_queries.reference index 22ad46f802c..7c8e4c0ca59 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.reference +++ b/tests/queries/0_stateless/02354_vector_search_queries.reference @@ -1,9 +1,9 @@ 10 rows, index_granularity = 8192, GRANULARITY = 1 million --> 1 granule, 1 indexed block -- Usearch: ORDER-BY-type +- ORDER-BY-type 5 [0,2] 0 6 [0,2.1] 0.09999990463256836 7 [0,2.2] 0.20000004768371582 -- Usearch: ORDER-BY-type, EXPLAIN +- ORDER-BY-type, EXPLAIN Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) @@ -16,15 +16,15 @@ Expression (Projection) Granules: 1/1 Skip Name: idx - Description: usearch GRANULARITY 100000000 + Description: vector_similarity GRANULARITY 100000000 Parts: 1/1 Granules: 1/1 12 rows, index_granularity = 3, GRANULARITY = 2 --> 4 granules, 2 indexed block -- Usearch: ORDER-BY-type +- ORDER-BY-type 6 [0,2] 0 7 [0,2.1] 0.09999990463256836 8 [0,2.2] 0.20000004768371582 -- Usearch: ORDER-BY-type, EXPLAIN +- ORDER-BY-type, EXPLAIN Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) @@ -37,11 +37,11 @@ Expression (Projection) Granules: 4/4 Skip Name: idx - Description: usearch GRANULARITY 2 + Description: vector_similarity GRANULARITY 2 Parts: 1/1 Granules: 2/4 Special cases -- Usearch: ORDER-BY-type +- ORDER-BY-type 6 [1,9.3] 0.005731362878640178 1 [2,3.2] 0.15200169244542905 7 [5.5,4.7] 0.3503476876550442 diff --git a/tests/queries/0_stateless/02354_vector_search_queries.sql b/tests/queries/0_stateless/02354_vector_search_queries.sql index 555f47b364f..50537ad6244 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.sql +++ b/tests/queries/0_stateless/02354_vector_search_queries.sql @@ -2,7 +2,7 @@ -- Tests various simple approximate nearest neighborhood (ANN) queries that utilize vector search indexes. -SET allow_experimental_usearch_index = 1; +SET allow_experimental_vector_similarity_index = 1; SET enable_analyzer = 0; @@ -10,18 +10,18 @@ SELECT '10 rows, index_granularity = 8192, GRANULARITY = 1 million --> 1 granule DROP TABLE IF EXISTS tab; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [0.0, 2.0]), (6, [0.0, 2.1]), (7, [0.0, 2.2]), (8, [0.0, 2.3]), (9, [0.0, 2.4]); -SELECT '- Usearch: ORDER-BY-type'; +SELECT '- ORDER-BY-type'; WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) FROM tab ORDER BY L2Distance(vec, reference_vec) LIMIT 3; -SELECT '- Usearch: ORDER-BY-type, EXPLAIN'; +SELECT '- ORDER-BY-type, EXPLAIN'; EXPLAIN indexes = 1 WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) @@ -34,17 +34,17 @@ DROP TABLE tab; SELECT '12 rows, index_granularity = 3, GRANULARITY = 2 --> 4 granules, 2 indexed block'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch() GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity() GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [1.5, 0.0]), (6, [0.0, 2.0]), (7, [0.0, 2.1]), (8, [0.0, 2.2]), (9, [0.0, 2.3]), (10, [0.0, 2.4]), (11, [0.0, 2.5]); -SELECT '- Usearch: ORDER-BY-type'; +SELECT '- ORDER-BY-type'; WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) FROM tab ORDER BY L2Distance(vec, reference_vec) LIMIT 3; -SELECT '- Usearch: ORDER-BY-type, EXPLAIN'; +SELECT '- ORDER-BY-type, EXPLAIN'; EXPLAIN indexes = 1 WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) @@ -58,10 +58,10 @@ DROP TABLE tab; SELECT 'Special cases'; -- Not a systematic test, just to check that no bad things happen. -- Just for jun, use metric = 'cosineDistance', scalarKind = 'f64' -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch('cosineDistance', 'f64') GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('cosineDistance', 'f64') GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; INSERT INTO tab VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); -SELECT '- Usearch: ORDER-BY-type'; +SELECT '- ORDER-BY-type'; WITH [0.0, 2.0] AS reference_vec SELECT id, vec, cosineDistance(vec, reference_vec) FROM tab From cc5c64e1ede7284d91ada1f28edbb18a457f5894 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 19:48:51 +0000 Subject: [PATCH 606/644] Add migration helper for legacy 'annoy' and 'usearch' indexes types Index types 'annoy' and 'usearch' were removed and replaced by 'vector_similarity' indexes in an earlier commit. This means unfortuantely, that if customers have tables with these indexes and upgrade, their database might not start anymore - the system loads the metadata at startup, thinks something is wrong with such tables, and halts immediately. This commit adds support for loading and attaching such indexes back. Data insert or use (search) return an error which recommends a migration to 'vector_similarity' indexes. The implementation is generally similar to what has recently been implemented for 'full_text' indexes [1, 2]. [1] https://github.com/ClickHouse/ClickHouse/pull/64656 [2] https://github.com/ClickHouse/ClickHouse/pull/64846 --- .../QueryPlan/ReadFromMergeTree.cpp | 3 ++ .../MergeTreeIndexLegacyVectorSimilarity.cpp | 45 +++++++++++++++++++ .../MergeTreeIndexLegacyVectorSimilarity.h | 26 +++++++++++ src/Storages/MergeTree/MergeTreeIndices.cpp | 10 +++++ src/Storages/MergeTree/MergeTreeIndices.h | 3 ++ ...earch_legacy_index_compatibility.reference | 2 + ...ctor_search_legacy_index_compatibility.sql | 43 ++++++++++++++++++ 7 files changed, 132 insertions(+) create mode 100644 src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.cpp create mode 100644 src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.h create mode 100644 tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.reference create mode 100644 tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.sql diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 1f30725b4d0..348019d7d10 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -1481,6 +1482,8 @@ static void buildIndexes( if (const auto * vector_similarity_index = typeid_cast(index_helper.get())) condition = vector_similarity_index->createIndexCondition(query_info, context); #endif + if (const auto * legacy_vector_similarity_index = typeid_cast(index_helper.get())) + condition = legacy_vector_similarity_index->createIndexCondition(query_info, context); if (!condition) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown vector search index {}", index_helper->index.name); } diff --git a/src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.cpp new file mode 100644 index 00000000000..29de109d4fc --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.cpp @@ -0,0 +1,45 @@ +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_INDEX; +} + +MergeTreeIndexLegacyVectorSimilarity::MergeTreeIndexLegacyVectorSimilarity(const IndexDescription & index_) + : IMergeTreeIndex(index_) +{ +} + +MergeTreeIndexGranulePtr MergeTreeIndexLegacyVectorSimilarity::createIndexGranule() const +{ + throw Exception(ErrorCodes::ILLEGAL_INDEX, "Indexes of type 'annoy' or 'usearch' are no longer supported. Please drop and recreate the index as type 'vector_similarity'"); +} + +MergeTreeIndexAggregatorPtr MergeTreeIndexLegacyVectorSimilarity::createIndexAggregator(const MergeTreeWriterSettings &) const +{ + throw Exception(ErrorCodes::ILLEGAL_INDEX, "Indexes of type 'annoy' or 'usearch' are no longer supported. Please drop and recreate the index as type 'vector_similarity'"); +} + +MergeTreeIndexConditionPtr MergeTreeIndexLegacyVectorSimilarity::createIndexCondition(const SelectQueryInfo &, ContextPtr) const +{ + throw Exception(ErrorCodes::ILLEGAL_INDEX, "Indexes of type 'annoy' or 'usearch' are no longer supported. Please drop and recreate the index as type 'vector_similarity'"); +}; + +MergeTreeIndexConditionPtr MergeTreeIndexLegacyVectorSimilarity::createIndexCondition(const ActionsDAG *, ContextPtr) const +{ + throw Exception(ErrorCodes::ILLEGAL_INDEX, "Indexes of type 'annoy' or 'usearch' are no longer supported. Please drop and recreate the index as type 'vector_similarity'"); +} + +MergeTreeIndexPtr legacyVectorSimilarityIndexCreator(const IndexDescription & index) +{ + return std::make_shared(index); +} + +void legacyVectorSimilarityIndexValidator(const IndexDescription &, bool) +{ +} + +} diff --git a/src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.h b/src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.h new file mode 100644 index 00000000000..1015401823d --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.h @@ -0,0 +1,26 @@ +#pragma once + +#include + +/// Walking corpse implementation for removed skipping index of type "annoy" and "usearch". +/// Its only purpose is to allow loading old tables with indexes of these types. +/// Data insertion and index usage/search will throw an exception, suggesting to migrate to "vector_similarity" indexes. + +namespace DB +{ + +class MergeTreeIndexLegacyVectorSimilarity : public IMergeTreeIndex +{ +public: + explicit MergeTreeIndexLegacyVectorSimilarity(const IndexDescription & index_); + ~MergeTreeIndexLegacyVectorSimilarity() override = default; + + MergeTreeIndexGranulePtr createIndexGranule() const override; + MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings &) const override; + MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo &, ContextPtr) const; + MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAG *, ContextPtr) const override; + + bool isVectorSimilarityIndex() const override { return true; } +}; + +} diff --git a/src/Storages/MergeTree/MergeTreeIndices.cpp b/src/Storages/MergeTree/MergeTreeIndices.cpp index 89aed7873a4..d2fc0e84b56 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.cpp +++ b/src/Storages/MergeTree/MergeTreeIndices.cpp @@ -132,6 +132,16 @@ MergeTreeIndexFactory::MergeTreeIndexFactory() registerCreator("vector_similarity", vectorSimilarityIndexCreator); registerValidator("vector_similarity", vectorSimilarityIndexValidator); #endif + /// ------ + /// TODO: remove this block at the end of 2024. + /// Index types 'annoy' and 'usearch' are no longer supported as of June 2024. Their successor is index type 'vector_similarity'. + /// To support loading tables with old indexes during a transition period, register dummy indexes which allow load/attaching but + /// throw an exception when the user attempts to use them. + registerCreator("annoy", legacyVectorSimilarityIndexCreator); + registerValidator("annoy", legacyVectorSimilarityIndexValidator); + registerCreator("usearch", legacyVectorSimilarityIndexCreator); + registerValidator("usearch", legacyVectorSimilarityIndexValidator); + /// ------ registerCreator("inverted", fullTextIndexCreator); registerValidator("inverted", fullTextIndexValidator); diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index 48ef2a4739e..c52d7ffe131 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -243,6 +243,9 @@ MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index); void vectorSimilarityIndexValidator(const IndexDescription & index, bool attach); #endif +MergeTreeIndexPtr legacyVectorSimilarityIndexCreator(const IndexDescription & index); +void legacyVectorSimilarityIndexValidator(const IndexDescription & index, bool attach); + MergeTreeIndexPtr fullTextIndexCreator(const IndexDescription & index); void fullTextIndexValidator(const IndexDescription & index, bool attach); diff --git a/tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.reference b/tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.reference new file mode 100644 index 00000000000..030bfa9b1bd --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.reference @@ -0,0 +1,2 @@ +Annoy +Usearch diff --git a/tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.sql b/tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.sql new file mode 100644 index 00000000000..0889aa74f7a --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.sql @@ -0,0 +1,43 @@ +-- Indexes of type 'annoy' or 'usearch' are no longer supported. +-- Test what happens when ClickHouse encounters tables with the old index type. + +DROP TABLE IF EXISTS tab; + +SELECT 'Annoy'; + +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX vec_idx vec TYPE annoy()) ENGINE = MergeTree ORDER BY id; + +INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [0.0, 2.0]), (6, [0.0, 2.1]), (7, [0.0, 2.2]), (8, [0.0, 2.3]), (9, [0.0, 2.4]); -- { serverError ILLEGAL_INDEX } + +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; +-- (*) The search succeeds because the index contains no data (i.e. some shortcut) +-- If it had data (can't really test in SQL tests ...), this statement would also return an error, trust me. + +-- Detach and attach should work. +DETACH TABLE tab; +ATTACH TABLE tab; + +DROP TABLE tab; + +SELECT 'Usearch'; + +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX vec_idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; + +INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [0.0, 2.0]), (6, [0.0, 2.1]), (7, [0.0, 2.2]), (8, [0.0, 2.3]), (9, [0.0, 2.4]); -- { serverError ILLEGAL_INDEX } + +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; +-- see above: (*) + +-- Detach and attach should work. +DETACH TABLE tab; +ATTACH TABLE tab; + +DROP TABLE tab; From d2e79f0b92936eb3ec3f6409fe6db18a3091919d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Aug 2024 15:28:38 +0000 Subject: [PATCH 607/644] Rework vector index parameters USearch (similar to FAISS) allows to specify the distance function, quantization, and various HNSW meta-parameters for index creation and sarch. Some users wished for greater configurability, so let's expose them. Index creation now requires either - 2 parameters (with the other 4 parameters taking on default values), or - 6 parameters for full control This commit also remove quantization `f64` (that would be upsampling). --- .../mergetree-family/annindexes.md | 12 +- .../MergeTreeIndexVectorSimilarity.cpp | 166 ++++++++++++------ .../MergeTreeIndexVectorSimilarity.h | 23 ++- .../0_stateless/02354_vector_search_bugs.sql | 6 +- ...2354_vector_search_default_granularity.sql | 4 +- .../02354_vector_search_detach_attach.sql | 2 +- ...r_search_index_creation_negative.reference | 12 +- ..._vector_search_index_creation_negative.sql | 48 +++-- ...4_vector_search_multiple_indexes.reference | 0 .../02354_vector_search_multiple_indexes.sql | 14 ++ .../02354_vector_search_queries.sql | 8 +- 11 files changed, 203 insertions(+), 92 deletions(-) create mode 100644 tests/queries/0_stateless/02354_vector_search_multiple_indexes.reference create mode 100644 tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 63c061a0d46..354fac6ea74 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -43,12 +43,22 @@ CREATE TABLE table ( id Int64, vectors Array(Float32), - INDEX [index_name vectors TYPE vector_similarity([Distance[, ScalarKind]]) [GRANULARITY [N]] + INDEX index_name vec TYPE vector_similarity(method, distance_function[, quantization, connectivity, expansion_add, expansion_search]) [GRANULARITY N] ) ENGINE = MergeTree ORDER BY id; ``` +Parameters: +- `method`: Supports currently only `hnsw`. +- `distance_function`: either `L2Distance` (the [Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance) - the length of a + line between two points in Euclidean space), or `cosineDistance` (the [cosine + distance](https://en.wikipedia.org/wiki/Cosine_similarity#Cosine_distance)- the angle between two non-zero vectors). +- `quantization`: either `f32`, `f16`, or `i8` for storing the vector with reduced precision (optional, default: `f32`) +- `m`: the number of neighbors per graph node (optional, default: 16) +- `ef_construction`: (optional, default: 128) +- `ef_search`: (optional, default: 64) + Vector similarity indexes are based on the [USearch library](https://github.com/unum-cloud/usearch), which implements the [HNSW algorithm](https://arxiv.org/abs/1603.09320), i.e., a hierarchical graph where each point represents a vector and the edges represent similarity. Such hierarchical structures can be very efficient on large collections. They may often fetch 0.05% or less data from the diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 6f3b1b043cd..5b0793fa0c8 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -45,6 +45,9 @@ namespace ErrorCodes namespace { +/// The only indexing method currently supported by USearch +std::set methods = {"hnsw"}; + /// Maps from user-facing name to internal name std::unordered_map distanceFunctionToMetricKind = { {"L2Distance", unum::usearch::metric_kind_t::l2sq_k}, @@ -52,22 +55,37 @@ std::unordered_map distanceFunctionToMetri /// Maps from user-facing name to internal name std::unordered_map quantizationToScalarKind = { - {"f64", unum::usearch::scalar_kind_t::f64_k}, {"f32", unum::usearch::scalar_kind_t::f32_k}, {"f16", unum::usearch::scalar_kind_t::f16_k}, {"i8", unum::usearch::scalar_kind_t::i8_k}}; +template +concept is_set = std::same_as>; + +template +concept is_unordered_map = std::same_as>; + template -String keysAsString(const T & t) +String joinByComma(const T & t) { - String result; - for (const auto & [k, _] : t) + if constexpr (is_set) { - if (!result.empty()) - result += ", "; - result += k; + return fmt::format("{}", fmt::join(t, ", ")); } - return result; + else if constexpr (is_unordered_map) + { + String joined_keys; + for (const auto & [k, _] : t) + { + if (!joined_keys.empty()) + joined_keys += ", "; + joined_keys += k; + } + return joined_keys; + } + /// TODO once our libcxx is recent enough, replace above by + /// return fmt::format("{}", fmt::join(std::views::keys(t)), ", ")); + std::unreachable(); } } @@ -75,8 +93,10 @@ String keysAsString(const T & t) USearchIndexWithSerialization::USearchIndexWithSerialization( size_t dimensions, unum::usearch::metric_kind_t metric_kind, - unum::usearch::scalar_kind_t scalar_kind) - : Base(Base::make(unum::usearch::metric_punned_t(dimensions, metric_kind, scalar_kind))) + unum::usearch::scalar_kind_t scalar_kind, + UsearchHnswParams usearch_hnsw_params) + : Base(Base::make(unum::usearch::metric_punned_t(dimensions, metric_kind, scalar_kind), + unum::usearch::index_dense_config_t(usearch_hnsw_params.m, usearch_hnsw_params.ef_construction, usearch_hnsw_params.ef_search))) { } @@ -125,8 +145,9 @@ MergeTreeIndexGranuleVectorSimilarity::MergeTreeIndexGranuleVectorSimilarity( const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, - unum::usearch::scalar_kind_t scalar_kind_) - : MergeTreeIndexGranuleVectorSimilarity(index_name_, index_sample_block_, metric_kind_, scalar_kind_, nullptr) + unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_) + : MergeTreeIndexGranuleVectorSimilarity(index_name_, index_sample_block_, metric_kind_, scalar_kind_, usearch_hnsw_params_, nullptr) { } @@ -135,11 +156,13 @@ MergeTreeIndexGranuleVectorSimilarity::MergeTreeIndexGranuleVectorSimilarity( const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_, USearchIndexWithSerializationPtr index_) : index_name(index_name_) , index_sample_block(index_sample_block_) , metric_kind(metric_kind_) , scalar_kind(scalar_kind_) + , usearch_hnsw_params(usearch_hnsw_params_) , index(std::move(index_)) { } @@ -153,8 +176,8 @@ void MergeTreeIndexGranuleVectorSimilarity::serializeBinary(WriteBuffer & ostr) /// Number of dimensions is required in the index constructor, /// so it must be written and read separately from the other part - writeIntBinary(static_cast(index->dimensions()), ostr); // write dimension - // + writeIntBinary(static_cast(index->dimensions()), ostr); + index->serialize(ostr); auto statistics = index->getStatistics(); @@ -176,7 +199,7 @@ void MergeTreeIndexGranuleVectorSimilarity::deserializeBinary(ReadBuffer & istr, UInt64 dimension; readIntBinary(dimension, istr); - index = std::make_shared(dimension, metric_kind, scalar_kind); + index = std::make_shared(dimension, metric_kind, scalar_kind, usearch_hnsw_params); index->deserialize(istr); @@ -189,17 +212,19 @@ MergeTreeIndexAggregatorVectorSimilarity::MergeTreeIndexAggregatorVectorSimilari const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, - unum::usearch::scalar_kind_t scalar_kind_) + unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_) : index_name(index_name_) , index_sample_block(index_sample_block_) , metric_kind(metric_kind_) , scalar_kind(scalar_kind_) + , usearch_hnsw_params(usearch_hnsw_params_) { } MergeTreeIndexGranulePtr MergeTreeIndexAggregatorVectorSimilarity::getGranuleAndReset() { - auto granule = std::make_shared(index_name, index_sample_block, metric_kind, scalar_kind, index); + auto granule = std::make_shared(index_name, index_sample_block, metric_kind, scalar_kind, usearch_hnsw_params, index); index = nullptr; return granule; } @@ -258,15 +283,15 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_ throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name); if (!index) - index = std::make_shared(dimensions, metric_kind, scalar_kind); + index = std::make_shared(dimensions, metric_kind, scalar_kind, usearch_hnsw_params); /// Reserving space is mandatory if (!index->reserve(roundUpToPowerOfTwoOrZero(index->size() + num_rows))) throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for vector similarity index"); - for (size_t current_row = 0; current_row < num_rows; ++current_row) + for (size_t row = 0; row < num_rows; ++row) { - auto rc = index->add(static_cast(index->size()), &column_array_data_float_data[column_array_offsets[current_row - 1]]); + auto rc = index->add(static_cast(index->size()), &column_array_data_float_data[column_array_offsets[row - 1]]); if (!rc) throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index, error: " + String(rc.error.release())); @@ -313,8 +338,6 @@ std::vector MergeTreeIndexConditionVectorSimilarity::getUsefulRanges(Mer const UInt64 limit = vector_similarity_condition.getLimit(); const UInt64 index_granularity = vector_similarity_condition.getIndexGranularity(); - const std::vector reference_vector = vector_similarity_condition.getReferenceVector(); - const auto granule = std::dynamic_pointer_cast(granule_); if (granule == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule has the wrong type"); @@ -326,6 +349,8 @@ std::vector MergeTreeIndexConditionVectorSimilarity::getUsefulRanges(Mer "does not match the dimension in the index ({})", vector_similarity_condition.getDimensions(), index->dimensions()); + const std::vector reference_vector = vector_similarity_condition.getReferenceVector(); + auto result = index->search(reference_vector.data(), limit); if (result.error) throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not search in vector similarity index, error: " + String(result.error.release())); @@ -350,21 +375,26 @@ std::vector MergeTreeIndexConditionVectorSimilarity::getUsefulRanges(Mer return granules; } -MergeTreeIndexVectorSimilarity::MergeTreeIndexVectorSimilarity(const IndexDescription & index_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_) +MergeTreeIndexVectorSimilarity::MergeTreeIndexVectorSimilarity( + const IndexDescription & index_, + unum::usearch::metric_kind_t metric_kind_, + unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_) : IMergeTreeIndex(index_) , metric_kind(metric_kind_) , scalar_kind(scalar_kind_) + , usearch_hnsw_params(usearch_hnsw_params_) { } MergeTreeIndexGranulePtr MergeTreeIndexVectorSimilarity::createIndexGranule() const { - return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind); + return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind, usearch_hnsw_params); } MergeTreeIndexAggregatorPtr MergeTreeIndexVectorSimilarity::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const { - return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind); + return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind, usearch_hnsw_params); } MergeTreeIndexConditionPtr MergeTreeIndexVectorSimilarity::createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const @@ -379,56 +409,82 @@ MergeTreeIndexConditionPtr MergeTreeIndexVectorSimilarity::createIndexCondition( MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index) { - static constexpr auto default_metric_kind = unum::usearch::metric_kind_t::l2sq_k; - auto metric_kind = default_metric_kind; - if (!index.arguments.empty()) - metric_kind = distanceFunctionToMetricKind.at(index.arguments[0].safeGet()); + const bool has_six_args = (index.arguments.size() == 6); - static constexpr auto default_scalar_kind = unum::usearch::scalar_kind_t::f16_k; - auto scalar_kind = default_scalar_kind; - if (index.arguments.size() > 1) - scalar_kind = quantizationToScalarKind.at(index.arguments[1].safeGet()); + unum::usearch::metric_kind_t metric_kind = distanceFunctionToMetricKind.at(index.arguments[1].safeGet()); - return std::make_shared(index, metric_kind, scalar_kind); + /// use defaults for the other parameters + unum::usearch::scalar_kind_t scalar_kind = unum::usearch::scalar_kind_t::f32_k; + UsearchHnswParams usearch_hnsw_params; + + if (has_six_args) + { + scalar_kind = quantizationToScalarKind.at(index.arguments[2].safeGet()); + usearch_hnsw_params = {.m = index.arguments[3].safeGet(), + .ef_construction = index.arguments[4].safeGet(), + .ef_search = index.arguments[5].safeGet()}; + } + + return std::make_shared(index, metric_kind, scalar_kind, usearch_hnsw_params); } void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* attach */) { - /// Check number and type of index arguments: + const bool has_two_args = (index.arguments.size() == 2); + const bool has_six_args = (index.arguments.size() == 6); - if (index.arguments.size() > 2) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Vector similarity index must not have more than one parameters"); + /// Check number and type of arguments + if (!has_two_args && !has_six_args) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Vector similarity index must have two or six arguments"); + if (index.arguments[0].getType() != Field::Types::String) + throw Exception(ErrorCodes::INCORRECT_QUERY, "First argument of vector similarity index (method) must be of type String"); + if (index.arguments[1].getType() != Field::Types::String) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Second argument of vector similarity index (metric) must be of type String"); + if (has_six_args) + { + if (index.arguments[2].getType() != Field::Types::String) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Third argument of vector similarity index (quantization) must be of type String"); + if (index.arguments[3].getType() != Field::Types::UInt64) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Fourth argument of vector similarity index (M) must be of type UInt64"); + if (index.arguments[4].getType() != Field::Types::UInt64) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Fifth argument of vector similarity index (ef_construction) must be of type UInt64"); + if (index.arguments[5].getType() != Field::Types::UInt64) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Sixth argument of vector similarity index (ef_search) must be of type UInt64"); + } - if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "First argument of vector similarity index (distance function) must be of type String"); - if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Second argument of vector similarity index (scalar type) must be of type String"); + /// Check that passed arguments are supported + if (!methods.contains(index.arguments[0].safeGet())) + throw Exception(ErrorCodes::INCORRECT_DATA, "First argument (method) of vector similarity index is not supported. Supported methods are: {}", joinByComma(methods)); + if (!distanceFunctionToMetricKind.contains(index.arguments[1].safeGet())) + throw Exception(ErrorCodes::INCORRECT_DATA, "Second argument (distance function) of vector similarity index is not supported. Supported distance function are: {}", joinByComma(distanceFunctionToMetricKind)); + if (has_six_args) + { + if (!quantizationToScalarKind.contains(index.arguments[2].safeGet())) + throw Exception(ErrorCodes::INCORRECT_DATA, "Third argument (quantization) of vector similarity index is not supported. Supported quantizations are: {}", joinByComma(quantizationToScalarKind)); + if (index.arguments[3].safeGet() < 2) + throw Exception(ErrorCodes::INCORRECT_DATA, "Fourth argument (M) of vector similarity index must be > 1"); + if (index.arguments[4].safeGet() < 1) + throw Exception(ErrorCodes::INCORRECT_DATA, "Fifth argument (ef_construction) of vector similarity index must be > 0"); + if (index.arguments[5].safeGet() < 1) + throw Exception(ErrorCodes::INCORRECT_DATA, "Sixth argument (ef_search) of vector similarity index must be > 0"); + } /// Check that the index is created on a single column - if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Vector similarity indexes must be created on a single column"); - /// Check that a supported metric was passed as first argument - - if (!index.arguments.empty() && !distanceFunctionToMetricKind.contains(index.arguments[0].safeGet())) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unrecognized metric kind (first argument) for vector index. Supported kinds are: {}", keysAsString(distanceFunctionToMetricKind)); - - /// Check that a supported kind was passed as a second argument - - if (index.arguments.size() > 1 && !quantizationToScalarKind.contains(index.arguments[1].safeGet())) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unrecognized scalar kind (second argument) for vector index. Supported kinds are: {}", keysAsString(quantizationToScalarKind)); - - /// Check data type of indexed column: + /// Check data type of the indexed column: DataTypePtr data_type = index.sample_block.getDataTypes()[0]; if (const auto * data_type_array = typeid_cast(data_type.get())) { TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); if (!WhichDataType(nested_type_index).isFloat32()) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity index can only be created on columns of type Array(Float32)"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float32)"); } else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity index can only be created on columns of type Array(Float32)"); + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float32)"); + } } } diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h index 95ea3cd5240..f7098c1626c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h @@ -14,6 +14,13 @@ namespace DB { +struct UsearchHnswParams +{ + size_t m = unum::usearch::default_connectivity(); + size_t ef_construction = unum::usearch::default_expansion_add(); + size_t ef_search = unum::usearch::default_expansion_search(); +}; + using USearchIndex = unum::usearch::index_dense_gt; class USearchIndexWithSerialization : public USearchIndex @@ -24,7 +31,8 @@ public: USearchIndexWithSerialization( size_t dimensions, unum::usearch::metric_kind_t metric_kind, - unum::usearch::scalar_kind_t scalar_kind); + unum::usearch::scalar_kind_t scalar_kind, + UsearchHnswParams usearch_hnsw_params); void serialize(WriteBuffer & ostr) const; void deserialize(ReadBuffer & istr); @@ -54,13 +62,15 @@ struct MergeTreeIndexGranuleVectorSimilarity final : public IMergeTreeIndexGranu const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, - unum::usearch::scalar_kind_t scalar_kind_); + unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_); MergeTreeIndexGranuleVectorSimilarity( const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_, USearchIndexWithSerializationPtr index_); ~MergeTreeIndexGranuleVectorSimilarity() override = default; @@ -74,6 +84,7 @@ struct MergeTreeIndexGranuleVectorSimilarity final : public IMergeTreeIndexGranu const Block index_sample_block; const unum::usearch::metric_kind_t metric_kind; const unum::usearch::scalar_kind_t scalar_kind; + const UsearchHnswParams usearch_hnsw_params; USearchIndexWithSerializationPtr index; LoggerPtr logger = getLogger("VectorSimilarityIndex"); @@ -93,7 +104,8 @@ struct MergeTreeIndexAggregatorVectorSimilarity final : IMergeTreeIndexAggregato const String & index_name_, const Block & index_sample_block, unum::usearch::metric_kind_t metric_kind_, - unum::usearch::scalar_kind_t scalar_kind_); + unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_); ~MergeTreeIndexAggregatorVectorSimilarity() override = default; @@ -105,6 +117,7 @@ struct MergeTreeIndexAggregatorVectorSimilarity final : IMergeTreeIndexAggregato const Block index_sample_block; const unum::usearch::metric_kind_t metric_kind; const unum::usearch::scalar_kind_t scalar_kind; + const UsearchHnswParams usearch_hnsw_params; USearchIndexWithSerializationPtr index; }; @@ -136,7 +149,8 @@ public: MergeTreeIndexVectorSimilarity( const IndexDescription & index_, unum::usearch::metric_kind_t metric_kind_, - unum::usearch::scalar_kind_t scalar_kind_); + unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_); ~MergeTreeIndexVectorSimilarity() override = default; @@ -149,6 +163,7 @@ public: private: const unum::usearch::metric_kind_t metric_kind; const unum::usearch::scalar_kind_t scalar_kind; + const UsearchHnswParams usearch_hnsw_params; }; } diff --git a/tests/queries/0_stateless/02354_vector_search_bugs.sql b/tests/queries/0_stateless/02354_vector_search_bugs.sql index 2ef75d0a7fe..7c66b4b8e45 100644 --- a/tests/queries/0_stateless/02354_vector_search_bugs.sql +++ b/tests/queries/0_stateless/02354_vector_search_bugs.sql @@ -9,14 +9,14 @@ DROP TABLE IF EXISTS tab; SELECT 'Issue #52258: Empty Arrays or Arrays with default values are rejected'; -CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree() ORDER BY id; +CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree() ORDER BY id; INSERT INTO tab VALUES (1, []); -- { serverError INCORRECT_DATA } INSERT INTO tab (id) VALUES (1); -- { serverError INCORRECT_DATA } DROP TABLE tab; SELECT 'It is possible to create parts with different Array vector sizes but there will be an error at query time'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; SYSTEM STOP MERGES tab; INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2]); INSERT INTO tab values (2, [2.2, 2.3, 2.4]) (3, [3.1, 3.2, 3.3]); @@ -31,7 +31,7 @@ DROP TABLE tab; SELECT 'Correctness of index with > 1 mark'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0, min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192; -- disable adaptive granularity due to bug +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0, min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192; -- disable adaptive granularity due to bug INSERT INTO tab SELECT number, [toFloat32(number), 0.0] from numbers(10000); WITH [1.0, 0.0] AS reference_vec diff --git a/tests/queries/0_stateless/02354_vector_search_default_granularity.sql b/tests/queries/0_stateless/02354_vector_search_default_granularity.sql index a19a0d17536..acb69cb6ff8 100644 --- a/tests/queries/0_stateless/02354_vector_search_default_granularity.sql +++ b/tests/queries/0_stateless/02354_vector_search_default_granularity.sql @@ -6,13 +6,13 @@ SET allow_experimental_vector_similarity_index = 1; -- After CREATE TABLE DROP TABLE IF EXISTS tab; -CREATE TABLE tab (id Int32, vec Array(Float32), INDEX idx(vec) TYPE vector_similarity) ENGINE = MergeTree ORDER BY id; +CREATE TABLE tab (id Int32, vec Array(Float32), INDEX idx(vec) TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; SELECT granularity FROM system.data_skipping_indices WHERE database = currentDatabase() AND table = 'tab' AND name = 'idx'; -- After ALTER TABLE DROP TABLE tab; CREATE TABLE tab (id Int32, vec Array(Float32)) ENGINE = MergeTree ORDER BY id; -ALTER TABLE tab ADD INDEX idx(vec) TYPE vector_similarity; +ALTER TABLE tab ADD INDEX idx(vec) TYPE vector_similarity('hnsw', 'L2Distance'); SELECT granularity FROM system.data_skipping_indices WHERE database = currentDatabase() AND table = 'tab' AND name = 'idx'; DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_detach_attach.sql b/tests/queries/0_stateless/02354_vector_search_detach_attach.sql index 36241dfabf7..f92eaddbbed 100644 --- a/tests/queries/0_stateless/02354_vector_search_detach_attach.sql +++ b/tests/queries/0_stateless/02354_vector_search_detach_attach.sql @@ -5,7 +5,7 @@ SET allow_experimental_vector_similarity_index = 1; DROP TABLE IF EXISTS tab; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [0.0, 2.0]), (6, [0.0, 2.1]), (7, [0.0, 2.2]), (8, [0.0, 2.3]), (9, [0.0, 2.4]); DETACH TABLE tab SYNC; diff --git a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference index bee3236f436..b6d034208d0 100644 --- a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference +++ b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference @@ -1,8 +1,10 @@ -At most two index arguments -1st argument (distance function) must be String -Unsupported distance functions are rejected -2nd argument (scalar kind) must be String -Unsupported scalar kinds are rejected +Two or six index arguments +1st argument (method) must be String and hnsw +2nd argument (distance function) must be String and L2Distance or cosineDistance +3nd argument (quantization), if given, must be String and f32, f16, ... +4nd argument (M), if given, must be UInt64 and > 1 +5nd argument (ef_construction), if given, must be UInt64 and > 0 +6nd argument (ef_search), if given, must be UInt64 and > 0 Must be created on single column Must be created on Array(Float32) columns Rejects INSERTs of Arrays with different sizes diff --git a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql index 912f7d7fcae..7c2ddfe81fc 100644 --- a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql +++ b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql @@ -6,32 +6,46 @@ SET allow_experimental_vector_similarity_index = 1; DROP TABLE IF EXISTS tab; -SELECT 'At most two index arguments'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('too', 'many', 'args')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +SELECT 'Two or six index arguments'; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('cant_have_one_arg')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('cant', 'have', 'three_args')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('cant', 'have', 'more', 'than', 'six', 'args', '!')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } -SELECT '1st argument (distance function) must be String'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +SELECT '1st argument (method) must be String and hnsw'; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity(3, 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('not_hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } -SELECT 'Unsupported distance functions are rejected'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('invalidDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +SELECT '2nd argument (distance function) must be String and L2Distance or cosineDistance'; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'invalid_distance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } -SELECT '2nd argument (scalar kind) must be String'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } - -SELECT 'Unsupported scalar kinds are rejected'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('L2Distance', 'invalidKind')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +SELECT '3nd argument (quantization), if given, must be String and f32, f16, ...'; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 1, 1, 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'invalid', 2, 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +SELECT '4nd argument (M), if given, must be UInt64 and > 1'; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 'invalid', 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 1, 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +SELECT '5nd argument (ef_construction), if given, must be UInt64 and > 0'; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 'invalid', 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 0, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +SELECT '6nd argument (ef_search), if given, must be UInt64 and > 0'; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 1, 'invalid')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 1, 0)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } SELECT 'Must be created on single column'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx (vec, id) TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx (vec, id) TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } SELECT 'Must be created on Array(Float32) columns'; SET allow_suspicious_low_cardinality_types = 1; -CREATE TABLE tab(id Int32, vec Float32, INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, vec LowCardinality(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, vec Nullable(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec UInt64, INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec Float32, INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec LowCardinality(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec Nullable(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } SELECT 'Rejects INSERTs of Arrays with different sizes'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2, 3.3]); -- { serverError INCORRECT_DATA } DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_multiple_indexes.reference b/tests/queries/0_stateless/02354_vector_search_multiple_indexes.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql b/tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql new file mode 100644 index 00000000000..f1cfc041233 --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql @@ -0,0 +1,14 @@ +-- Tags: no-fasttest, no-ordinary-database + +-- Tests that multiple vector similarity indexes can be created on the same column (even if that makes no sense) + +SET allow_experimental_vector_similarity_index = 1; + +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (id Int32, vec Array(Float32), PRIMARY KEY id, INDEX vec_idx(vec) TYPE vector_similarity('hnsw', 'L2Distance')); + +ALTER TABLE tab ADD INDEX idx(vec) TYPE minmax; +ALTER TABLE tab ADD INDEX vec_idx1(vec) TYPE vector_similarity('hnsw', 'cosineDistance'); +ALTER TABLE tab ADD INDEX vec_idx2(vec) TYPE vector_similarity('hnsw', 'L2Distance'); -- silly but creating the same index also works for non-vector indexes ... + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_queries.sql b/tests/queries/0_stateless/02354_vector_search_queries.sql index 50537ad6244..dbf0fca32ab 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.sql +++ b/tests/queries/0_stateless/02354_vector_search_queries.sql @@ -10,7 +10,7 @@ SELECT '10 rows, index_granularity = 8192, GRANULARITY = 1 million --> 1 granule DROP TABLE IF EXISTS tab; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [0.0, 2.0]), (6, [0.0, 2.1]), (7, [0.0, 2.2]), (8, [0.0, 2.3]), (9, [0.0, 2.4]); @@ -34,7 +34,7 @@ DROP TABLE tab; SELECT '12 rows, index_granularity = 3, GRANULARITY = 2 --> 4 granules, 2 indexed block'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity() GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [1.5, 0.0]), (6, [0.0, 2.0]), (7, [0.0, 2.1]), (8, [0.0, 2.2]), (9, [0.0, 2.3]), (10, [0.0, 2.4]), (11, [0.0, 2.5]); SELECT '- ORDER-BY-type'; @@ -56,9 +56,9 @@ DROP TABLE tab; SELECT 'Special cases'; -- Not a systematic test, just to check that no bad things happen. --- Just for jun, use metric = 'cosineDistance', scalarKind = 'f64' +-- Test with non-default metric, M, ef_construction, ef_search -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('cosineDistance', 'f64') GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f32', 42, 99, 66) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; INSERT INTO tab VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); SELECT '- ORDER-BY-type'; From fb76cb90b1badef334b96b61d976136fd38d535d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 11 Aug 2024 09:31:36 +0000 Subject: [PATCH 608/644] Allow un-quoted skip index parameters Previously, only this syntax to create a skip index worked: INDEX index_name column_name TYPE vector_similarity('hnsw', 'L2Distance') Now, this syntax will work as well: INDEX index_name column_name TYPE vector_similarity(hnsw, L2Distance) --- .../mergetree-family/annindexes.md | 15 +++++++++++- src/Storages/IndicesDescription.cpp | 12 +++++++--- ...search_unquoted_index_parameters.reference | 0 ...ector_search_unquoted_index_parameters.sql | 23 +++++++++++++++++++ 4 files changed, 46 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02354_vector_search_unquoted_index_parameters.reference create mode 100644 tests/queries/0_stateless/02354_vector_search_unquoted_index_parameters.sql diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 354fac6ea74..e73d6f07a32 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -43,7 +43,7 @@ CREATE TABLE table ( id Int64, vectors Array(Float32), - INDEX index_name vec TYPE vector_similarity(method, distance_function[, quantization, connectivity, expansion_add, expansion_search]) [GRANULARITY N] + INDEX index_name vectors TYPE vector_similarity(method, distance_function[, quantization, connectivity, expansion_add, expansion_search]) [GRANULARITY N] ) ENGINE = MergeTree ORDER BY id; @@ -59,6 +59,19 @@ Parameters: - `ef_construction`: (optional, default: 128) - `ef_search`: (optional, default: 64) +Example: + +```sql +CREATE TABLE table +( + id Int64, + vectors Array(Float32), + INDEX idx vectors TYPE vector_similarity('hnsw', 'L2Distance') -- Alternative syntax: TYPE vector_similarity(hnsw, L2Distance) +) +ENGINE = MergeTree +ORDER BY id; +``` + Vector similarity indexes are based on the [USearch library](https://github.com/unum-cloud/usearch), which implements the [HNSW algorithm](https://arxiv.org/abs/1603.09320), i.e., a hierarchical graph where each point represents a vector and the edges represent similarity. Such hierarchical structures can be very efficient on large collections. They may often fetch 0.05% or less data from the diff --git a/src/Storages/IndicesDescription.cpp b/src/Storages/IndicesDescription.cpp index cef8fd85f97..753fbf1d635 100644 --- a/src/Storages/IndicesDescription.cpp +++ b/src/Storages/IndicesDescription.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -130,10 +131,15 @@ IndexDescription IndexDescription::getIndexFromAST(const ASTPtr & definition_ast { for (size_t i = 0; i < index_type->arguments->children.size(); ++i) { - const auto * argument = index_type->arguments->children[i]->as(); - if (!argument) + const auto & child = index_type->arguments->children[i]; + if (const auto * ast_literal = child->as(); ast_literal != nullptr) + /// E.g. INDEX index_name column_name TYPE vector_similarity('hnsw', 'f32') + result.arguments.emplace_back(ast_literal->value); + else if (const auto * ast_identifier = child->as(); ast_identifier != nullptr) + /// E.g. INDEX index_name column_name TYPE vector_similarity(hnsw, f32) + result.arguments.emplace_back(ast_identifier->name()); + else throw Exception(ErrorCodes::INCORRECT_QUERY, "Only literals can be skip index arguments"); - result.arguments.emplace_back(argument->value); } } diff --git a/tests/queries/0_stateless/02354_vector_search_unquoted_index_parameters.reference b/tests/queries/0_stateless/02354_vector_search_unquoted_index_parameters.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02354_vector_search_unquoted_index_parameters.sql b/tests/queries/0_stateless/02354_vector_search_unquoted_index_parameters.sql new file mode 100644 index 00000000000..da6494bf831 --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_unquoted_index_parameters.sql @@ -0,0 +1,23 @@ +-- Tags: no-fasttest, no-ordinary-database + +SET allow_experimental_vector_similarity_index = 1; + +-- Tests that quoted and unquoted parameters can be passed to vector search indexes. + +DROP TABLE IF EXISTS tab1; +DROP TABLE IF EXISTS tab2; + +CREATE TABLE tab1 (id Int32, vec Array(Float32), PRIMARY KEY id, INDEX vec_idx(vec) TYPE vector_similarity('hnsw', 'L2Distance')); +CREATE TABLE tab2 (id Int32, vec Array(Float32), PRIMARY KEY id, INDEX vec_idx(vec) TYPE vector_similarity(hnsw, L2Distance)); + +DROP TABLE tab1; +DROP TABLE tab2; + +CREATE TABLE tab1 (id Int32, vec Array(Float32), PRIMARY KEY id); +CREATE TABLE tab2 (id Int32, vec Array(Float32), PRIMARY KEY id); + +ALTER TABLE tab1 ADD INDEX idx1(vec) TYPE vector_similarity('hnsw', 'L2Distance'); +ALTER TABLE tab2 ADD INDEX idx2(vec) TYPE vector_similarity(hnsw, L2Distance); + +DROP TABLE tab1; +DROP TABLE tab2; From ea1cd665750f82bbeaf66f67b1d85e014afdec18 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 12 Aug 2024 17:32:43 +0200 Subject: [PATCH 609/644] fix tidy --- src/Storages/VirtualColumnUtils.cpp | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 1abac56d266..3143c7f78f6 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -38,7 +38,6 @@ #include #include #include -#include #include #include #include @@ -46,15 +45,7 @@ #include #include #include -#include "Functions/FunctionsLogical.h" -#include "Functions/IFunction.h" -#include "Functions/IFunctionAdaptors.h" -#include "Functions/indexHint.h" #include -#include -#include -#include -#include namespace DB From 3a6e05eb43cbb9937cded286ac7259b2f7168057 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 12 Aug 2024 18:03:42 +0200 Subject: [PATCH 610/644] try to fix includes --- src/Storages/VirtualColumnUtils.cpp | 55 ++++++++++++++++------------- 1 file changed, 30 insertions(+), 25 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 3143c7f78f6..d932f5cc469 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -1,43 +1,40 @@ #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 +#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 @@ -45,7 +42,15 @@ #include #include #include +#include "Functions/FunctionsLogical.h" +#include "Functions/IFunction.h" +#include "Functions/IFunctionAdaptors.h" +#include "Functions/indexHint.h" #include +#include +#include +#include +#include namespace DB From 858b7e55d0df3db1412d538f701c30584b5783bf Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 12 Aug 2024 16:16:50 +0000 Subject: [PATCH 611/644] Improve condition in case the default column consumes slightly more memory It never happened in the few hundreds of tests I ran successfully, but we'd rather be safe than sorry. --- .../01903_correct_block_size_prediction_with_default.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh index 075d9a1dacf..1482730af2c 100755 --- a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh +++ b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh @@ -28,7 +28,8 @@ function test() SYSTEM FLUSH LOGS; WITH memory_1 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_1' AND type = 'QueryFinish' as memory_1), memory_2 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_2' AND type = 'QueryFinish' as memory_2) - SELECT memory_1.memory_usage <= 1.2 * memory_2.memory_usage FROM memory_1, memory_2;" + SELECT memory_1.memory_usage <= 1.2 * memory_2.memory_usage OR + memory_2.memory_usage <= 1.2 * memory_1.memory_usage FROM memory_1, memory_2;" } test "" From f7af4c5643af2ee87b81a7972c0bb91cf723c8a2 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 12 Aug 2024 17:27:43 +0100 Subject: [PATCH 612/644] don't report system-wide metrics when cgroup metrics present --- src/Common/AsynchronousMetrics.cpp | 20 ++++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 02c130d3caa..9b6a7428411 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -75,12 +75,8 @@ AsynchronousMetrics::AsynchronousMetrics( , protocol_server_metrics_func(protocol_server_metrics_func_) { #if defined(OS_LINUX) - openFileIfExists("/proc/meminfo", meminfo); - openFileIfExists("/proc/loadavg", loadavg); - openFileIfExists("/proc/stat", proc_stat); openFileIfExists("/proc/cpuinfo", cpuinfo); openFileIfExists("/proc/sys/fs/file-nr", file_nr); - openFileIfExists("/proc/uptime", uptime); openFileIfExists("/proc/net/dev", net_dev); /// CGroups v2 @@ -103,6 +99,19 @@ AsynchronousMetrics::AsynchronousMetrics( if (!cgroupcpu_stat) openFileIfExists("/sys/fs/cgroup/cpuacct/cpuacct.stat", cgroupcpuacct_stat); + if (!cgroupcpu_stat && !cgroupcpuacct_stat) + { + /// The following metrics are not cgroup-aware and we've found cgroup-specific metric files for the similar metrics, + /// so we're better not reporting them at all to avoid confusion + openFileIfExists("/proc/loadavg", loadavg); + openFileIfExists("/proc/stat", proc_stat); + openFileIfExists("/proc/uptime", uptime); + } + + /// The same story for memory metrics + if (!cgroupmem_limit_in_bytes) + openFileIfExists("/proc/meminfo", meminfo); + openFileIfExists("/proc/sys/vm/max_map_count", vm_max_map_count); openFileIfExists("/proc/self/maps", vm_maps); @@ -1193,8 +1202,7 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) tryLogCurrentException(__PRETTY_FUNCTION__); } } - - if (meminfo) + else if (meminfo) { try { From f0f10bc0099e659bfc0bf31079e89832f9db4b17 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 12 Aug 2024 17:30:12 +0100 Subject: [PATCH 613/644] remove cgroupsV2MemoryControllerEnabled() --- base/base/cgroupsv2.cpp | 24 ------------------------ base/base/cgroupsv2.h | 4 ---- base/base/getMemoryAmount.cpp | 3 --- 3 files changed, 31 deletions(-) diff --git a/base/base/cgroupsv2.cpp b/base/base/cgroupsv2.cpp index 4372696c2b7..d8f95b23ae7 100644 --- a/base/base/cgroupsv2.cpp +++ b/base/base/cgroupsv2.cpp @@ -27,27 +27,6 @@ bool cgroupsV2Enabled() #endif } -bool cgroupsV2MemoryControllerEnabled() -{ -#if defined(OS_LINUX) - chassert(cgroupsV2Enabled()); - /// According to https://docs.kernel.org/admin-guide/cgroup-v2.html, file "cgroup.controllers" defines which controllers are available - /// for the current + child cgroups. The set of available controllers can be restricted from level to level using file - /// "cgroups.subtree_control". It is therefore sufficient to check the bottom-most nested "cgroup.controllers" file. - fs::path cgroup_dir = cgroupV2PathOfProcess(); - if (cgroup_dir.empty()) - return false; - std::ifstream controllers_file(cgroup_dir / "cgroup.controllers"); - if (!controllers_file.is_open()) - return false; - std::string controllers; - std::getline(controllers_file, controllers); - return controllers.find("memory") != std::string::npos; -#else - return false; -#endif -} - fs::path cgroupV2PathOfProcess() { #if defined(OS_LINUX) @@ -77,9 +56,6 @@ std::optional getCgroupsV2PathContainingFile(std::string_view file_ if (!cgroupsV2Enabled()) return {}; - if (!cgroupsV2MemoryControllerEnabled()) - return {}; - fs::path current_cgroup = cgroupV2PathOfProcess(); if (current_cgroup.empty()) return {}; diff --git a/base/base/cgroupsv2.h b/base/base/cgroupsv2.h index 9d8e178a866..925a399471e 100644 --- a/base/base/cgroupsv2.h +++ b/base/base/cgroupsv2.h @@ -12,10 +12,6 @@ static inline const std::filesystem::path default_cgroups_mount = "/sys/fs/cgrou /// Is cgroups v2 enabled on the system? bool cgroupsV2Enabled(); -/// Is the memory controller of cgroups v2 enabled on the system? -/// Assumes that cgroupsV2Enabled() is enabled. -bool cgroupsV2MemoryControllerEnabled(); - /// Detects which cgroup v2 the process belongs to and returns the filesystem path to the cgroup. /// Returns an empty path the cgroup cannot be determined. /// Assumes that cgroupsV2Enabled() is enabled. diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index 03aab1eac72..bbfbecdbffd 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -19,9 +19,6 @@ std::optional getCgroupsV2MemoryLimit() if (!cgroupsV2Enabled()) return {}; - if (!cgroupsV2MemoryControllerEnabled()) - return {}; - std::filesystem::path current_cgroup = cgroupV2PathOfProcess(); if (current_cgroup.empty()) return {}; From 05b595094868dd29e59ea9c766d0829f57ce94f9 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 12 Aug 2024 17:31:56 +0100 Subject: [PATCH 614/644] small fix --- base/base/cgroupsv2.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/base/base/cgroupsv2.cpp b/base/base/cgroupsv2.cpp index d8f95b23ae7..b4ca8271d64 100644 --- a/base/base/cgroupsv2.cpp +++ b/base/base/cgroupsv2.cpp @@ -60,8 +60,8 @@ std::optional getCgroupsV2PathContainingFile(std::string_view file_ if (current_cgroup.empty()) return {}; - /// Return the bottom-most nested current memory file. If there is no such file at the current - /// level, try again at the parent level as memory settings are inherited. + /// Return the bottom-most nested file. If there is no such file at the current + /// level, try again at the parent level as settings are inherited. while (current_cgroup != default_cgroups_mount.parent_path()) { const auto path = current_cgroup / file_name; From c22265b889684b7fa34ba6816ce3910143ef7226 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 12 Aug 2024 17:11:11 +0000 Subject: [PATCH 615/644] Some fixups --- docs/en/operations/query-cache.md | 16 +++++----- docs/en/operations/settings/settings.md | 8 ++--- .../operations/system-tables/query_cache.md | 4 +-- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- src/Interpreters/Cache/QueryCache.cpp | 25 ++++++++-------- src/Interpreters/Cache/QueryCache.h | 11 ++++--- src/Interpreters/executeQuery.cpp | 5 ++-- .../System/StorageSystemQueryCache.cpp | 16 +++++----- .../02494_query_cache_tag.reference | 12 ++++---- .../0_stateless/02494_query_cache_tag.sql | 30 ++++++++++--------- 11 files changed, 66 insertions(+), 65 deletions(-) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index a6c4d74f4ac..384938e28f6 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -143,16 +143,18 @@ value can be specified at session, profile or query level using setting [query_c Entries in the query cache are compressed by default. This reduces the overall memory consumption at the cost of slower writes into / reads from the query cache. To disable compression, use setting [query_cache_compress_entries](settings/settings.md#query-cache-compress-entries). -Entries in the query cache can separate by tag, using setting [query_cache_tag](settings/settings.md#query-cache-tag). Queries with different tags are considered different entries. For example, the result of query +Sometimes it is useful to keep multiple results for the same query cached. This can be achieved using setting +[query_cache_tag](settings/settings.md#query-cache-tag) that acts as as a label (or namespace) for a query cache entries. The query cache +considers results of the same query with different tags different. -``` sql -SELECT 1 SETTINGS use_query_cache = true; -SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one'; -SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one diff'; +Example for creating three different query cache entries for the same query: + +```sql +SELECT 1 SETTINGS use_query_cache = true; -- query_cache_tag is implicitly '' (empty string) +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'tag 1'; +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'tag 2'; ``` -have different entries in the query cache, find the specified tag in system table [system.query_cache](system-tables/query_cache.md) - ClickHouse reads table data in blocks of [max_block_size](settings/settings.md#setting-max_block_size) rows. Due to filtering, aggregation, etc., result blocks are typically much smaller than 'max_block_size' but there are also cases where they are much bigger. Setting [query_cache_squash_partial_results](settings/settings.md#query-cache-squash-partial-results) (enabled by default) controls if result blocks diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7b855665efb..e4a126249ca 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1802,14 +1802,14 @@ Default value: `0`. ## query_cache_tag {#query-cache-tag} -An arbitrary string to separate entries in the [query cache](../query-cache.md). -Queries with different values of this setting are considered different. +A string which acts as a label for [query cache](../query-cache.md) entries. +The same queries with different tags are considered different by the query cache. Possible values: -- string: name of query cache tag +- Any string -Default value: `''`. +Default value: `''` ## query_cache_max_size_in_bytes {#query-cache-max-size-in-bytes} diff --git a/docs/en/operations/system-tables/query_cache.md b/docs/en/operations/system-tables/query_cache.md index 393b37d3616..9c48574a329 100644 --- a/docs/en/operations/system-tables/query_cache.md +++ b/docs/en/operations/system-tables/query_cache.md @@ -9,12 +9,12 @@ Columns: - `query` ([String](../../sql-reference/data-types/string.md)) — Query string. - `result_size` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Size of the query cache entry. +- `tag` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — Tag of the query cache entry. - `stale` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is stale. - `shared` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is shared between multiple users. - `compressed` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is compressed. - `expires_at` ([DateTime](../../sql-reference/data-types/datetime.md)) — When the query cache entry becomes stale. - `key_hash` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — A hash of the query string, used as a key to find query cache entries. -- `tag` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — An arbitrary string to separate entries in the query cache. **Example** @@ -27,12 +27,12 @@ Row 1: ────── query: SELECT 1 SETTINGS use_query_cache = 1 result_size: 128 +tag: stale: 0 shared: 0 compressed: 1 expires_at: 2023-10-13 13:35:45 key_hash: 12188185624808016954 -tag: 1 row in set. Elapsed: 0.004 sec. ``` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 75579f20187..f9ffab0ea57 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -676,7 +676,7 @@ class IColumn; M(Bool, query_cache_squash_partial_results, true, "Squash partial result blocks to blocks of size 'max_block_size'. Reduces performance of inserts into the query cache but improves the compressability of cache entries.", 0) \ M(Seconds, query_cache_ttl, 60, "After this time in seconds entries in the query cache become stale", 0) \ M(Bool, query_cache_share_between_users, false, "Allow other users to read entry in the query cache", 0) \ - M(String, query_cache_tag, "", "An arbitrary string to separate entries in the query cache. Queries with different values of this setting are considered different.", 0) \ + M(String, query_cache_tag, "", "A string which acts as a label for query cache entries. The same queries with different tags are considered different by the query cache.", 0) \ M(Bool, enable_sharing_sets_for_mutations, true, "Allow sharing set objects build for IN subqueries between different tasks of the same mutation. This reduces memory usage and CPU consumption", 0) \ \ M(Bool, optimize_rewrite_sum_if_to_count_if, true, "Rewrite sumIf() and sum(if()) function countIf() function when logically equivalent", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8fd16504e95..0528287e83e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -82,9 +82,9 @@ static std::initializer_list user_id_, const std::vector & current_user_roles_, bool is_shared_, std::chrono::time_point expires_at_, - bool is_compressed_, - const String & tag_) - : ast_hash(calculateAstHash(ast_, current_database, settings, tag_)) + bool is_compressed_) + : ast_hash(calculateAstHash(ast_, current_database, settings)) , header(header_) , user_id(user_id_) , current_user_roles(current_user_roles_) @@ -247,12 +242,18 @@ QueryCache::Key::Key( , expires_at(expires_at_) , is_compressed(is_compressed_) , query_string(queryStringFromAST(ast_)) - , tag(tag_) + , tag(settings.query_cache_tag) { } -QueryCache::Key::Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional user_id_, const std::vector & current_user_roles_, const String & tag_) - : QueryCache::Key(ast_, current_database, settings, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false, tag_) /// dummy values for everything != AST, current database, user name/roles +QueryCache::Key::Key( + ASTPtr ast_, + const String & current_database, + const Settings & settings, + std::optional user_id_, + const std::vector & current_user_roles_) + : QueryCache::Key(ast_, current_database, settings, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) + /// ^^ dummy values for everything != AST, current database, user name/roles { } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 54de5edb145..c7ebaf4d26a 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -88,8 +88,9 @@ public: /// SYSTEM.QUERY_CACHE. const String query_string; - /// An arbitrary string to separate entries in the query cache. - /// Queries with different values of this setting are considered different. + /// A tag (namespace) for distinguish multiple entries of the same query. + /// This member has currently no use besides that SYSTEM.QUERY_CACHE can populate the 'tag' column conveniently without having to + /// compute the tag from the query AST. const String tag; /// Ctor to construct a Key for writing into query cache. @@ -100,15 +101,13 @@ public: std::optional user_id_, const std::vector & current_user_roles_, bool is_shared_, std::chrono::time_point expires_at_, - bool is_compressed, - const String & tag_); + bool is_compressed); /// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name). Key(ASTPtr ast_, const String & current_database, const Settings & settings, - std::optional user_id_, const std::vector & current_user_roles_, - const String & tag_); + std::optional user_id_, const std::vector & current_user_roles_); bool operator==(const Key & other) const; }; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 6422d3128fa..fe87eed5570 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1129,7 +1129,7 @@ static std::tuple executeQueryImpl( { if (can_use_query_cache && settings.enable_reads_from_query_cache) { - QueryCache::Key key(ast, context->getCurrentDatabase(), *settings_copy, context->getUserID(), context->getCurrentRoles(), settings.query_cache_tag); + QueryCache::Key key(ast, context->getCurrentDatabase(), *settings_copy, context->getUserID(), context->getCurrentRoles()); QueryCache::Reader reader = query_cache->createReader(key); if (reader.hasCacheEntryForKey()) { @@ -1258,8 +1258,7 @@ static std::tuple executeQueryImpl( context->getUserID(), context->getCurrentRoles(), settings.query_cache_share_between_users, std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl), - settings.query_cache_compress_entries, - settings.query_cache_tag); + settings.query_cache_compress_entries); const size_t num_query_runs = settings.query_cache_min_query_runs ? query_cache->recordQueryRun(key) : 1; /// try to avoid locking a mutex in recordQueryRun() if (num_query_runs <= settings.query_cache_min_query_runs) diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index f81d50e8806..b3532ba40a7 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -16,12 +16,12 @@ ColumnsDescription StorageSystemQueryCache::getColumnsDescription() { {"query", std::make_shared(), "Query string."}, {"result_size", std::make_shared(), "Size of the query cache entry."}, + {"tag", std::make_shared(std::make_shared()), "Tag of the query cache entry."}, {"stale", std::make_shared(), "If the query cache entry is stale."}, {"shared", std::make_shared(), "If the query cache entry is shared between multiple users."}, {"compressed", std::make_shared(), "If the query cache entry is compressed."}, {"expires_at", std::make_shared(), "When the query cache entry becomes stale."}, - {"key_hash", std::make_shared(), "A hash of the query string, used as a key to find query cache entries."}, - {"tag", std::make_shared(std::make_shared()), "An arbitrary string to separate entries in the query cache."} + {"key_hash", std::make_shared(), "A hash of the query string, used as a key to find query cache entries."} }; } @@ -53,12 +53,12 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr res_columns[0]->insert(key.query_string); /// approximates the original query string res_columns[1]->insert(QueryCache::QueryCacheEntryWeight()(*query_result)); - res_columns[2]->insert(key.expires_at < std::chrono::system_clock::now()); - res_columns[3]->insert(key.is_shared); - res_columns[4]->insert(key.is_compressed); - res_columns[5]->insert(std::chrono::system_clock::to_time_t(key.expires_at)); - res_columns[6]->insert(key.ast_hash.low64); /// query cache considers aliases (issue #56258) - res_columns[7]->insert(key.tag); + res_columns[2]->insert(key.tag); + res_columns[3]->insert(key.expires_at < std::chrono::system_clock::now()); + res_columns[4]->insert(key.is_shared); + res_columns[5]->insert(key.is_compressed); + res_columns[6]->insert(std::chrono::system_clock::to_time_t(key.expires_at)); + res_columns[7]->insert(key.ast_hash.low64); /// query cache considers aliases (issue #56258) } } diff --git a/tests/queries/0_stateless/02494_query_cache_tag.reference b/tests/queries/0_stateless/02494_query_cache_tag.reference index 055d3d4c5bb..f7be5c06ecf 100644 --- a/tests/queries/0_stateless/02494_query_cache_tag.reference +++ b/tests/queries/0_stateless/02494_query_cache_tag.reference @@ -1,14 +1,12 @@ 1 -1 +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = \'abc\' abc --- 1 1 -1 -2 +SELECT 1 SETTINGS use_query_cache = true +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = \'abc\' abc --- 1 1 -1 -2 -1 -3 +SELECT 1 SETTINGS use_query_cache = true abc +SELECT 1 SETTINGS use_query_cache = true def diff --git a/tests/queries/0_stateless/02494_query_cache_tag.sql b/tests/queries/0_stateless/02494_query_cache_tag.sql index 054607058e8..62d36f6ebe6 100644 --- a/tests/queries/0_stateless/02494_query_cache_tag.sql +++ b/tests/queries/0_stateless/02494_query_cache_tag.sql @@ -3,30 +3,32 @@ SYSTEM DROP QUERY CACHE; --- Cache the query after the query invocation -SELECT 1 SETTINGS use_query_cache = true; -SELECT COUNT(*) FROM system.query_cache; +-- Store the result a single query with a tag in the query cache and check that the system table knows about the tag +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'abc'; + +SELECT query, tag FROM system.query_cache; SELECT '---'; SYSTEM DROP QUERY CACHE; --- Queries with tag value of this setting or not are considered different cache entries. -SELECT 1 SETTINGS use_query_cache = true; -SELECT COUNT(*) FROM system.query_cache; -SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one'; -SELECT COUNT(*) FROM system.query_cache; +-- Store the result of the same query with two different tags. The cache should store two entries. +SELECT 1 SETTINGS use_query_cache = true; -- default query_cache_tag = '' +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'abc'; +SELECT query, tag FROM system.query_cache ORDER BY ALL; SELECT '---'; SYSTEM DROP QUERY CACHE; --- Queries with different tags values of this setting are considered different cache entries. +-- Like before but the tag is set standalone. + +SET query_cache_tag = 'abc'; SELECT 1 SETTINGS use_query_cache = true; -SELECT COUNT(*) FROM system.query_cache; -SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one'; -SELECT COUNT(*) FROM system.query_cache; -SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one diff'; -SELECT COUNT(*) FROM system.query_cache; + +SET query_cache_tag = 'def'; +SELECT 1 SETTINGS use_query_cache = true; + +SELECT query, tag FROM system.query_cache ORDER BY ALL; SYSTEM DROP QUERY CACHE; From 38405dd7cdfb7189c1a1184c0eb8b3e23fda55e6 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 12 Aug 2024 18:14:22 +0000 Subject: [PATCH 616/644] add projection merge doc --- docs/en/operations/settings/merge-tree-settings.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 67fa45c20cd..a3bd919d3ce 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -1041,3 +1041,14 @@ Compression rates of LZ4 or ZSTD improve on average by 20-40%. This setting works best for tables with no primary key or a low-cardinality primary key, i.e. a table with only few distinct primary key values. High-cardinality primary keys, e.g. involving timestamp columns of type `DateTime64`, are not expected to benefit from this setting. + +### deduplicate_merge_projection_mode + +Whether to allow create projection for the table with non-classic MergeTree, that is not (Replicated, Shared) MergeTree. If allowed, what is the action when merge projections, either drop or rebuild. So classic MergeTree would ignore this setting. +It also controls `OPTIMIZE DEDUPLICATE` as well, but has effect on all MergeTree family members. + +Possible values: + +- throw, drop, rebuild + +Default value: throw \ No newline at end of file From aa7a2bcb02f6c2f48bcc7acca3bcec2f1a16130b Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 12 Aug 2024 20:34:02 +0200 Subject: [PATCH 617/644] Fix typo --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 21e4a6599ea..4f51dc6b8d3 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -4073,7 +4073,7 @@ getSubcolumn(col_name, subcol_name) **Returned value** -- Returns the extracted sub-colum. +- Returns the extracted sub-column. **Example** From eab8594570e703a766f2f91ae3d13b0ed640b554 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 12 Aug 2024 20:35:33 +0200 Subject: [PATCH 618/644] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 862f38976ce..51246d990fa 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1697,6 +1697,8 @@ getOSKernelVersion getServerPort getSetting getSizeOfEnumType +getSubcolumn +getTypeSerializationStreams getblockinfo getevents ghcnd From 45a14fa0ce3ae94a374bbf955ba0fb7109b7e678 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 12 Aug 2024 18:54:06 +0000 Subject: [PATCH 619/644] Fix spelling --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index ffd9fae7f45..03ec8e1752c 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2115,6 +2115,7 @@ namenode namepassword nameprofile namequota +namespace namespaces natively nats From 469c1698b0dbf8a91a6e94a2bab0669f33bf7be2 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 12 Aug 2024 19:31:57 +0000 Subject: [PATCH 620/644] Fix 'Refresh set entry already exists' --- src/Interpreters/InterpreterSystemQuery.cpp | 17 +++-- src/Interpreters/InterpreterSystemQuery.h | 2 +- src/Storages/MaterializedView/RefreshSet.cpp | 75 +++++++++++-------- src/Storages/MaterializedView/RefreshSet.h | 23 +++--- src/Storages/MaterializedView/RefreshTask.cpp | 10 +-- src/Storages/MaterializedView/RefreshTask.h | 3 +- .../MaterializedView/RefreshTask_fwd.h | 1 + src/Storages/StorageMaterializedView.cpp | 1 - .../System/StorageSystemViewRefreshes.cpp | 3 + 9 files changed, 77 insertions(+), 58 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index ef6d1040c5e..1cd55a0020c 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -663,13 +663,16 @@ BlockIO InterpreterSystemQuery::execute() startStopAction(ActionLocks::ViewRefresh, false); break; case Type::REFRESH_VIEW: - getRefreshTask()->run(); + for (const auto & task : getRefreshTasks()) + task->run(); break; case Type::CANCEL_VIEW: - getRefreshTask()->cancel(); + for (const auto & task : getRefreshTasks()) + task->cancel(); break; case Type::TEST_VIEW: - getRefreshTask()->setFakeTime(query.fake_time_for_view); + for (const auto & task : getRefreshTasks()) + task->setFakeTime(query.fake_time_for_view); break; case Type::DROP_REPLICA: dropReplica(query); @@ -1242,15 +1245,15 @@ void InterpreterSystemQuery::flushDistributed(ASTSystemQuery & query) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "SYSTEM RESTART DISK is not supported"); } -RefreshTaskHolder InterpreterSystemQuery::getRefreshTask() +RefreshTaskList InterpreterSystemQuery::getRefreshTasks() { auto ctx = getContext(); ctx->checkAccess(AccessType::SYSTEM_VIEWS); - auto task = ctx->getRefreshSet().getTask(table_id); - if (!task) + auto tasks = ctx->getRefreshSet().findTasks(table_id); + if (tasks.empty()) throw Exception( ErrorCodes::BAD_ARGUMENTS, "Refreshable view {} doesn't exist", table_id.getNameForLogs()); - return task; + return tasks; } diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 776dd7915f0..f44fe930b04 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -74,7 +74,7 @@ private: void flushDistributed(ASTSystemQuery & query); [[noreturn]] void restartDisk(String & name); - RefreshTaskHolder getRefreshTask(); + RefreshTaskList getRefreshTasks(); AccessRightsElements getRequiredAccessForDDLOnCluster() const; void startStopAction(StorageActionBlockType action_type, bool start); diff --git a/src/Storages/MaterializedView/RefreshSet.cpp b/src/Storages/MaterializedView/RefreshSet.cpp index a3ef327dc24..43aa0ada99b 100644 --- a/src/Storages/MaterializedView/RefreshSet.cpp +++ b/src/Storages/MaterializedView/RefreshSet.cpp @@ -27,6 +27,7 @@ RefreshSet::Handle & RefreshSet::Handle::operator=(Handle && other) noexcept parent_set = std::exchange(other.parent_set, nullptr); id = std::move(other.id); dependencies = std::move(other.dependencies); + iter = std::move(other.iter); metric_increment = std::move(other.metric_increment); return *this; } @@ -39,21 +40,21 @@ RefreshSet::Handle::~Handle() void RefreshSet::Handle::rename(StorageID new_id) { std::lock_guard lock(parent_set->mutex); - parent_set->removeDependenciesLocked(id, dependencies); - auto it = parent_set->tasks.find(id); - auto task = it->second; - parent_set->tasks.erase(it); + RefreshTaskHolder task = *iter; + parent_set->removeDependenciesLocked(task, dependencies); + parent_set->removeTaskLocked(id, iter); id = new_id; - parent_set->tasks.emplace(id, task); - parent_set->addDependenciesLocked(id, dependencies); + iter = parent_set->addTaskLocked(id, task); + parent_set->addDependenciesLocked(task, dependencies); } void RefreshSet::Handle::changeDependencies(std::vector deps) { std::lock_guard lock(parent_set->mutex); - parent_set->removeDependenciesLocked(id, dependencies); + RefreshTaskHolder task = *iter; + parent_set->removeDependenciesLocked(task, dependencies); dependencies = std::move(deps); - parent_set->addDependenciesLocked(id, dependencies); + parent_set->addDependenciesLocked(task, dependencies); } void RefreshSet::Handle::reset() @@ -63,8 +64,8 @@ void RefreshSet::Handle::reset() { std::lock_guard lock(parent_set->mutex); - parent_set->removeDependenciesLocked(id, dependencies); - parent_set->tasks.erase(id); + parent_set->removeDependenciesLocked(*iter, dependencies); + parent_set->removeTaskLocked(id, iter); } parent_set = nullptr; @@ -76,37 +77,50 @@ RefreshSet::RefreshSet() = default; void RefreshSet::emplace(StorageID id, const std::vector & dependencies, RefreshTaskHolder task) { std::lock_guard guard(mutex); - auto [it, is_inserted] = tasks.emplace(id, task); - if (!is_inserted) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Refresh set entry already exists for table {}", id.getFullTableName()); - addDependenciesLocked(id, dependencies); + const auto iter = addTaskLocked(id, task); + addDependenciesLocked(task, dependencies); - task->setRefreshSetHandleUnlock(Handle(this, id, dependencies)); + task->setRefreshSetHandleUnlock(Handle(this, id, iter, dependencies)); } -void RefreshSet::addDependenciesLocked(const StorageID & id, const std::vector & dependencies) +RefreshTaskList::iterator RefreshSet::addTaskLocked(StorageID id, RefreshTaskHolder task) +{ + RefreshTaskList & list = tasks[id]; + list.push_back(task); + return std::prev(list.end()); +} + +void RefreshSet::removeTaskLocked(StorageID id, RefreshTaskList::iterator iter) +{ + const auto it = tasks.find(id); + it->second.erase(iter); + if (it->second.empty()) + tasks.erase(it); +} + +void RefreshSet::addDependenciesLocked(RefreshTaskHolder task, const std::vector & dependencies) { for (const StorageID & dep : dependencies) - dependents[dep].insert(id); + dependents[dep].insert(task); } -void RefreshSet::removeDependenciesLocked(const StorageID & id, const std::vector & dependencies) +void RefreshSet::removeDependenciesLocked(RefreshTaskHolder task, const std::vector & dependencies) { for (const StorageID & dep : dependencies) { auto & set = dependents[dep]; - set.erase(id); + set.erase(task); if (set.empty()) dependents.erase(dep); } } -RefreshTaskHolder RefreshSet::getTask(const StorageID & id) const +RefreshTaskList RefreshSet::findTasks(const StorageID & id) const { std::lock_guard lock(mutex); - if (auto task = tasks.find(id); task != tasks.end()) - return task->second; - return nullptr; + if (auto it = tasks.find(id); it != tasks.end()) + return it->second; + return {}; } RefreshSet::InfoContainer RefreshSet::getInfo() const @@ -116,26 +130,23 @@ RefreshSet::InfoContainer RefreshSet::getInfo() const lock.unlock(); InfoContainer res; - for (const auto & [id, task] : tasks_copy) - res.push_back(task->getInfo()); + for (const auto & [id, list] : tasks_copy) + for (const auto & task : list) + res.push_back(task->getInfo()); return res; } std::vector RefreshSet::getDependents(const StorageID & id) const { std::lock_guard lock(mutex); - std::vector res; auto it = dependents.find(id); if (it == dependents.end()) return {}; - for (const StorageID & dep_id : it->second) - if (auto task = tasks.find(dep_id); task != tasks.end()) - res.push_back(task->second); - return res; + return std::vector(it->second.begin(), it->second.end()); } -RefreshSet::Handle::Handle(RefreshSet * parent_set_, StorageID id_, std::vector dependencies_) +RefreshSet::Handle::Handle(RefreshSet * parent_set_, StorageID id_, RefreshTaskList::iterator iter_, std::vector dependencies_) : parent_set(parent_set_), id(std::move(id_)), dependencies(std::move(dependencies_)) - , metric_increment(CurrentMetrics::Increment(CurrentMetrics::RefreshableViews)) {} + , iter(iter_), metric_increment(CurrentMetrics::Increment(CurrentMetrics::RefreshableViews)) {} } diff --git a/src/Storages/MaterializedView/RefreshSet.h b/src/Storages/MaterializedView/RefreshSet.h index eff445023a6..7fb583fd316 100644 --- a/src/Storages/MaterializedView/RefreshSet.h +++ b/src/Storages/MaterializedView/RefreshSet.h @@ -5,12 +5,11 @@ #include #include #include +#include namespace DB { -using DatabaseAndTableNameSet = std::unordered_set; - enum class RefreshState : RefreshTaskStateUnderlying { Disabled = 0, @@ -46,8 +45,7 @@ struct RefreshInfo class RefreshSet { public: - /// RAII thing that unregisters a task and its dependencies in destructor. - /// Storage IDs must be unique. Not thread safe. + /// RAII thing that unregisters a task and its dependencies in destructor. Not thread safe. class Handle { friend class RefreshSet; @@ -73,9 +71,10 @@ public: RefreshSet * parent_set = nullptr; StorageID id = StorageID::createEmpty(); std::vector dependencies; + RefreshTaskList::iterator iter; // in parent_set->tasks[id] std::optional metric_increment; - Handle(RefreshSet * parent_set_, StorageID id_, std::vector dependencies_); + Handle(RefreshSet * parent_set_, StorageID id_, RefreshTaskList::iterator iter_, std::vector dependencies_); }; using InfoContainer = std::vector; @@ -84,7 +83,9 @@ public: void emplace(StorageID id, const std::vector & dependencies, RefreshTaskHolder task); - RefreshTaskHolder getTask(const StorageID & id) const; + /// Finds active refreshable view(s) by database and table name. + /// Normally there's at most one, but we allow name collisions here, just in case. + RefreshTaskList findTasks(const StorageID & id) const; InfoContainer getInfo() const; @@ -92,8 +93,8 @@ public: std::vector getDependents(const StorageID & id) const; private: - using TaskMap = std::unordered_map; - using DependentsMap = std::unordered_map; + using TaskMap = std::unordered_map; + using DependentsMap = std::unordered_map, StorageID::DatabaseAndTableNameHash, StorageID::DatabaseAndTableNameEqual>; /// Protects the two maps below, not locked for any nontrivial operations (e.g. operations that /// block or lock other mutexes). @@ -102,8 +103,10 @@ private: TaskMap tasks; DependentsMap dependents; - void addDependenciesLocked(const StorageID & id, const std::vector & dependencies); - void removeDependenciesLocked(const StorageID & id, const std::vector & dependencies); + RefreshTaskList::iterator addTaskLocked(StorageID id, RefreshTaskHolder task); + void removeTaskLocked(StorageID id, RefreshTaskList::iterator iter); + void addDependenciesLocked(RefreshTaskHolder task, const std::vector & dependencies); + void removeDependenciesLocked(RefreshTaskHolder task, const std::vector & dependencies); }; } diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index aa8f51d5295..0837eaf97fd 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -33,7 +33,6 @@ RefreshTask::RefreshTask( {} RefreshTaskHolder RefreshTask::create( - const StorageMaterializedView & view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy) { @@ -46,12 +45,9 @@ RefreshTaskHolder RefreshTask::create( t->refreshTask(); }); - std::vector deps; if (strategy.dependencies) for (auto && dependency : strategy.dependencies->children) - deps.emplace_back(dependency->as()); - - context->getRefreshSet().emplace(view.getStorageID(), deps, task); + task->initial_dependencies.emplace_back(dependency->as()); return task; } @@ -61,6 +57,7 @@ void RefreshTask::initializeAndStart(std::shared_ptr vi view_to_refresh = view; if (view->getContext()->getSettingsRef().stop_refreshable_materialized_views_on_startup) stop_requested = true; + view->getContext()->getRefreshSet().emplace(view->getStorageID(), initial_dependencies, shared_from_this()); populateDependencies(); advanceNextRefreshTime(currentTime()); refresh_task->schedule(); @@ -69,7 +66,8 @@ void RefreshTask::initializeAndStart(std::shared_ptr vi void RefreshTask::rename(StorageID new_id) { std::lock_guard guard(mutex); - set_handle.rename(new_id); + if (set_handle) + set_handle.rename(new_id); } void RefreshTask::alterRefreshParams(const DB::ASTRefreshStrategy & new_strategy) diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index 1f050a97cd9..623493f6aec 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -26,7 +26,6 @@ public: /// The only proper way to construct task static RefreshTaskHolder create( - const StorageMaterializedView & view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy); @@ -84,9 +83,11 @@ private: RefreshSchedule refresh_schedule; RefreshSettings refresh_settings; // TODO: populate, use, update on alter + std::vector initial_dependencies; RefreshSet::Handle set_handle; /// StorageIDs of our dependencies that we're waiting for. + using DatabaseAndTableNameSet = std::unordered_set; DatabaseAndTableNameSet remaining_dependencies; bool time_arrived = false; diff --git a/src/Storages/MaterializedView/RefreshTask_fwd.h b/src/Storages/MaterializedView/RefreshTask_fwd.h index 1f366962eb6..9a0a122381e 100644 --- a/src/Storages/MaterializedView/RefreshTask_fwd.h +++ b/src/Storages/MaterializedView/RefreshTask_fwd.h @@ -11,5 +11,6 @@ class RefreshTask; using RefreshTaskStateUnderlying = UInt8; using RefreshTaskHolder = std::shared_ptr; using RefreshTaskObserver = std::weak_ptr; +using RefreshTaskList = std::list; } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 696136834d4..4c6c2fff209 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -203,7 +203,6 @@ StorageMaterializedView::StorageMaterializedView( { fixed_uuid = false; refresher = RefreshTask::create( - *this, getContext(), *query.refresh_strategy); refresh_on_start = mode < LoadingStrictnessLevel::ATTACH && !query.is_create_empty; diff --git a/src/Storages/System/StorageSystemViewRefreshes.cpp b/src/Storages/System/StorageSystemViewRefreshes.cpp index 30539ed6b6a..061201017a7 100644 --- a/src/Storages/System/StorageSystemViewRefreshes.cpp +++ b/src/Storages/System/StorageSystemViewRefreshes.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -19,6 +20,7 @@ ColumnsDescription StorageSystemViewRefreshes::getColumnsDescription() { {"database", std::make_shared(), "The name of the database the table is in."}, {"view", std::make_shared(), "Table name."}, + {"uuid", std::make_shared(), "Table uuid (Atomic database)."}, {"status", std::make_shared(), "Current state of the refresh."}, {"last_refresh_result", std::make_shared(), "Outcome of the latest refresh attempt."}, {"last_refresh_time", std::make_shared(std::make_shared()), @@ -63,6 +65,7 @@ void StorageSystemViewRefreshes::fillData( std::size_t i = 0; res_columns[i++]->insert(refresh.view_id.getDatabaseName()); res_columns[i++]->insert(refresh.view_id.getTableName()); + res_columns[i++]->insert(refresh.view_id.uuid); res_columns[i++]->insert(toString(refresh.state)); res_columns[i++]->insert(toString(refresh.last_refresh_result)); From 5a683796a0dc8408ed2694af672675929352bf8f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 12 Aug 2024 22:34:14 +0200 Subject: [PATCH 621/644] Update DatabaseReplicated.cpp --- src/Databases/DatabaseReplicated.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index f4aa925d6dd..6011b8e65e3 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1584,6 +1584,8 @@ void DatabaseReplicated::dropTable(ContextPtr local_context, const String & tabl } auto table = tryGetTable(table_name, getContext()); + if (!table) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} doesn't exist", table_name); if (table->getName() == "MaterializedView" || table->getName() == "WindowView") { /// Avoid recursive locking of metadata_mutex From 8136e6a45275b958a663ac0ee4682984e1536b07 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Aug 2024 21:29:26 +0000 Subject: [PATCH 622/644] Update new prepareForSquashing method for ColumnDynamic --- src/Columns/ColumnDynamic.cpp | 23 ++++++++++--------- src/DataTypes/DataTypeVariant.cpp | 2 +- .../03210_dynamic_squashing.reference | 20 +++++++++------- .../0_stateless/03210_dynamic_squashing.sql | 17 ++++++++------ 4 files changed, 35 insertions(+), 27 deletions(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index ecc2c738366..69b4c5dfc4e 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -987,7 +987,8 @@ void ColumnDynamic::prepareForSquashing(const Columns & source_columns) /// Internal variants of source dynamic columns may differ. /// We want to preallocate memory for all variants we will have after squashing. /// It may happen that the total number of variants in source columns will - /// exceed the limit, in this case we will choose the most frequent variants. + /// exceed the limit, in this case we will choose the most frequent variants + /// and insert the rest types into the shared variant. /// First, preallocate memory for variant discriminators and offsets. size_t new_size = size(); @@ -1030,17 +1031,14 @@ void ColumnDynamic::prepareForSquashing(const Columns & source_columns) DataTypePtr result_variant_type; /// Check if the number of all variants exceeds the limit. - if (all_variants.size() > max_dynamic_types || (all_variants.size() == max_dynamic_types && !total_variant_sizes.contains("String"))) + if (!canAddNewVariants(0, all_variants.size())) { /// We want to keep the most frequent variants in the resulting dynamic column. DataTypes result_variants; - result_variants.reserve(max_dynamic_types); + result_variants.reserve(max_dynamic_types + 1); /// +1 for shared variant. /// Add variants from current variant column as we will not rewrite it. for (const auto & variant : assert_cast(*variant_info.variant_type).getVariants()) result_variants.push_back(variant); - /// Add String variant in advance (if we didn't add it yet) as we must have it across variants when we reach the limit. - if (!variant_info.variant_name_to_discriminator.contains("String")) - result_variants.push_back(std::make_shared()); /// Create list of remaining variants with their sizes and sort it. std::vector> variants_with_sizes; @@ -1049,15 +1047,18 @@ void ColumnDynamic::prepareForSquashing(const Columns & source_columns) { /// Add variant to the list only of we didn't add it yet. auto variant_name = variant->getName(); - if (variant_name != "String" && !variant_info.variant_name_to_discriminator.contains(variant_name)) - variants_with_sizes.emplace_back(total_variant_sizes[variant->getName()], variant); + if (!variant_info.variant_name_to_discriminator.contains(variant_name)) + variants_with_sizes.emplace_back(total_variant_sizes[variant_name], variant); } std::sort(variants_with_sizes.begin(), variants_with_sizes.end(), std::greater()); /// Add the most frequent variants until we reach max_dynamic_types. - size_t num_new_variants = max_dynamic_types - result_variants.size(); - for (size_t i = 0; i != num_new_variants; ++i) - result_variants.push_back(variants_with_sizes[i].second); + for (const auto & [_, new_variant] : variants_with_sizes) + { + if (!canAddNewVariant(result_variants.size())) + break; + result_variants.push_back(new_variant); + } result_variant_type = std::make_shared(result_variants); } diff --git a/src/DataTypes/DataTypeVariant.cpp b/src/DataTypes/DataTypeVariant.cpp index 8a10ca7d06d..cc8d04e94da 100644 --- a/src/DataTypes/DataTypeVariant.cpp +++ b/src/DataTypes/DataTypeVariant.cpp @@ -117,7 +117,7 @@ bool DataTypeVariant::equals(const IDataType & rhs) const /// The same data types with different custom names considered different. /// For example, UInt8 and Bool. - if ((variants[i]->hasCustomName() || rhs_variant.variants[i]) && variants[i]->getName() != rhs_variant.variants[i]->getName()) + if ((variants[i]->hasCustomName() || rhs_variant.variants[i]->hasCustomName()) && variants[i]->getName() != rhs_variant.variants[i]->getName()) return false; } diff --git a/tests/queries/0_stateless/03210_dynamic_squashing.reference b/tests/queries/0_stateless/03210_dynamic_squashing.reference index 4f5b5ba098c..1c23c22f550 100644 --- a/tests/queries/0_stateless/03210_dynamic_squashing.reference +++ b/tests/queries/0_stateless/03210_dynamic_squashing.reference @@ -1,8 +1,12 @@ -Array(UInt8) -None -UInt64 -None -String -UInt64 -String -UInt64 +1 +Array(UInt8) true +None false +UInt64 false +2 +Array(UInt8) true +None false +UInt64 false +3 +Array(UInt8) true +String false +UInt64 true diff --git a/tests/queries/0_stateless/03210_dynamic_squashing.sql b/tests/queries/0_stateless/03210_dynamic_squashing.sql index 23b47184e33..da3b911e796 100644 --- a/tests/queries/0_stateless/03210_dynamic_squashing.sql +++ b/tests/queries/0_stateless/03210_dynamic_squashing.sql @@ -4,17 +4,20 @@ set max_block_size = 1000; drop table if exists test; create table test (d Dynamic) engine=MergeTree order by tuple(); -insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=2), number < 3000, range(number % 5)::Dynamic(max_types=2), number::Dynamic(max_types=2)) from numbers(1000000); -select distinct dynamicType(d) as type from test order by type; +insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(1000000); +select '1'; +select distinct dynamicType(d) as type, isDynamicElementInSharedData(d) as flag from test order by type; drop table test; -create table test (d Dynamic(max_types=2)) engine=MergeTree order by tuple(); -insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=2), number < 3000, range(number % 5)::Dynamic(max_types=2), number::Dynamic(max_types=2)) from numbers(1000000); -select distinct dynamicType(d) as type from test order by type; +create table test (d Dynamic(max_types=1)) engine=MergeTree order by tuple(); +insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(1000000); +select '2'; +select distinct dynamicType(d) as type, isDynamicElementInSharedData(d) as flag from test order by type; truncate table test; -insert into test select multiIf(number < 1000, 'Str'::Dynamic(max_types=2), number < 3000, range(number % 5)::Dynamic(max_types=2), number::Dynamic(max_types=2)) from numbers(1000000); -select distinct dynamicType(d) as type from test order by type; +insert into test select multiIf(number < 1000, 'Str'::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(1000000); +select '3'; +select distinct dynamicType(d) as type, isDynamicElementInSharedData(d) as flag from test order by type; drop table test; From 83cb991f75f242b11beb48134d6ebfb26c73bcd7 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Aug 2024 21:30:30 +0000 Subject: [PATCH 623/644] Fix special build --- src/Columns/ColumnDynamic.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index d80055c1716..e6e720765f6 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -368,10 +368,10 @@ public: /// Check if we can add new variant types. /// Shared variant doesn't count in the limit but always presents, /// so we should subtract 1 from the total types count. - bool canAddNewVariants(size_t current_variants_count, size_t new_variants_count) { return current_variants_count + new_variants_count - 1 <= max_dynamic_types; } - bool canAddNewVariant(size_t current_variants_count) { return canAddNewVariants(current_variants_count, 1); } - bool canAddNewVariants(size_t new_variants_count) { return canAddNewVariants(variant_info.variant_names.size(), new_variants_count); } - bool canAddNewVariant() { return canAddNewVariants(variant_info.variant_names.size(), 1); } + bool canAddNewVariants(size_t current_variants_count, size_t new_variants_count) const { return current_variants_count + new_variants_count - 1 <= max_dynamic_types; } + bool canAddNewVariant(size_t current_variants_count) const { return canAddNewVariants(current_variants_count, 1); } + bool canAddNewVariants(size_t new_variants_count) const { return canAddNewVariants(variant_info.variant_names.size(), new_variants_count); } + bool canAddNewVariant() const { return canAddNewVariants(variant_info.variant_names.size(), 1); } void setVariantType(const DataTypePtr & variant_type); void setMaxDynamicPaths(size_t max_dynamic_type_); From f12609440f081f19b0b21fdd15229cfdbb7cbb3d Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 12 Aug 2024 23:09:57 +0000 Subject: [PATCH 624/644] fashion --- src/Storages/MaterializedView/RefreshSet.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Storages/MaterializedView/RefreshSet.cpp b/src/Storages/MaterializedView/RefreshSet.cpp index 43aa0ada99b..7536f59c1e4 100644 --- a/src/Storages/MaterializedView/RefreshSet.cpp +++ b/src/Storages/MaterializedView/RefreshSet.cpp @@ -9,11 +9,6 @@ namespace CurrentMetrics namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - RefreshSet::Handle::Handle(Handle && other) noexcept { *this = std::move(other); From a517bc90cd9e369a4385f367e9f5e9688520c8bb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Mon, 12 Aug 2024 21:42:47 -0400 Subject: [PATCH 625/644] Update PULL_REQUEST_TEMPLATE.md --- .github/PULL_REQUEST_TEMPLATE.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 8b6e957e1d8..3dcce68ab46 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -60,7 +60,7 @@ At a minimum, the following information should be added (but add more as needed) - [ ] Exclude: All with aarch64, release, debug --- - [ ] Run only fuzzers related jobs (libFuzzer fuzzers, AST fuzzers, etc.) -- [ ] Exclude AST fuzzers +- [ ] Exclude: AST fuzzers --- - [ ] Do not test - [ ] Woolen Wolfdog From 5812dbcf2e4a9eada33a611ea7b63172f6ed0905 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 13 Aug 2024 11:53:33 +0200 Subject: [PATCH 626/644] Update 03210_dynamic_squashing.sql --- tests/queries/0_stateless/03210_dynamic_squashing.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03210_dynamic_squashing.sql b/tests/queries/0_stateless/03210_dynamic_squashing.sql index da3b911e796..d9ebc28fc43 100644 --- a/tests/queries/0_stateless/03210_dynamic_squashing.sql +++ b/tests/queries/0_stateless/03210_dynamic_squashing.sql @@ -1,3 +1,5 @@ +-- Tags: long + set allow_experimental_dynamic_type = 1; set max_block_size = 1000; From 52dea79a906ecc3d9a19599612b1c2c7708876b6 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 13 Aug 2024 15:20:37 +0200 Subject: [PATCH 627/644] Update 03210_dynamic_squashing.sql --- tests/queries/0_stateless/03210_dynamic_squashing.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03210_dynamic_squashing.sql b/tests/queries/0_stateless/03210_dynamic_squashing.sql index d9ebc28fc43..71d09263fda 100644 --- a/tests/queries/0_stateless/03210_dynamic_squashing.sql +++ b/tests/queries/0_stateless/03210_dynamic_squashing.sql @@ -6,18 +6,18 @@ set max_block_size = 1000; drop table if exists test; create table test (d Dynamic) engine=MergeTree order by tuple(); -insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(1000000); +insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(100000); select '1'; select distinct dynamicType(d) as type, isDynamicElementInSharedData(d) as flag from test order by type; drop table test; create table test (d Dynamic(max_types=1)) engine=MergeTree order by tuple(); -insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(1000000); +insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(100000); select '2'; select distinct dynamicType(d) as type, isDynamicElementInSharedData(d) as flag from test order by type; truncate table test; -insert into test select multiIf(number < 1000, 'Str'::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(1000000); +insert into test select multiIf(number < 1000, 'Str'::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(100000); select '3'; select distinct dynamicType(d) as type, isDynamicElementInSharedData(d) as flag from test order by type; From 973b2405794cebeabf9497e3b10ed6180130b891 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 13 Aug 2024 15:35:14 +0200 Subject: [PATCH 628/644] Fix min/max time columns --- src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp | 7 +++++-- .../0_stateless/03222_create_timeseries_table.reference | 0 .../queries/0_stateless/03222_create_timeseries_table.sql | 7 +++++++ 3 files changed, 12 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03222_create_timeseries_table.reference create mode 100644 tests/queries/0_stateless/03222_create_timeseries_table.sql diff --git a/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp index f9e7290e514..746a6a28274 100644 --- a/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp +++ b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp @@ -227,8 +227,11 @@ void TimeSeriesDefinitionNormalizer::addMissingColumns(ASTCreateQuery & create) /// We use Nullable(DateTime64(3)) as the default type of the `min_time` and `max_time` columns. /// It's nullable because it allows the aggregation (see aggregate_min_time_and_max_time) work correctly even /// for rows in the "tags" table which doesn't have `min_time` and `max_time` (because they have no matching rows in the "data" table). - make_new_column(TimeSeriesColumnNames::MinTime, make_nullable(timestamp_type)); - make_new_column(TimeSeriesColumnNames::MaxTime, make_nullable(timestamp_type)); + + if (!is_next_column_named(TimeSeriesColumnNames::MinTime)) + make_new_column(TimeSeriesColumnNames::MinTime, make_nullable(timestamp_type)); + if (!is_next_column_named(TimeSeriesColumnNames::MaxTime)) + make_new_column(TimeSeriesColumnNames::MaxTime, make_nullable(timestamp_type)); } /// Add missing columns for the "metrics" table. diff --git a/tests/queries/0_stateless/03222_create_timeseries_table.reference b/tests/queries/0_stateless/03222_create_timeseries_table.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03222_create_timeseries_table.sql b/tests/queries/0_stateless/03222_create_timeseries_table.sql new file mode 100644 index 00000000000..bdb29e7d366 --- /dev/null +++ b/tests/queries/0_stateless/03222_create_timeseries_table.sql @@ -0,0 +1,7 @@ +SET allow_experimental_time_series_table = 1; + +CREATE TABLE 03222_timeseries_table1 ENGINE = TimeSeries FORMAT Null; +CREATE TABLE 03222_timeseries_table2 ENGINE = TimeSeries SETTINGS store_min_time_and_max_time = 1, aggregate_min_time_and_max_time = 1 FORMAT Null; +--- This doesn't work because allow_nullable_key cannot be set in query for the internal MergeTree tables +--- CREATE TABLE 03222_timeseries_table3 ENGINE = TimeSeries SETTINGS store_min_time_and_max_time = 1, aggregate_min_time_and_max_time = 0; +CREATE TABLE 03222_timeseries_table4 ENGINE = TimeSeries SETTINGS store_min_time_and_max_time = 0 FORMAT Null; From 5da5bea8dfb768d3f6fd42f081a3d82e1c782e64 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 13 Aug 2024 13:47:20 +0000 Subject: [PATCH 629/644] Reduce flakiness of a test --- .../0_stateless/00652_mergetree_mutations.sh | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/tests/queries/0_stateless/00652_mergetree_mutations.sh b/tests/queries/0_stateless/00652_mergetree_mutations.sh index a9d7908a1af..3b0966dd2c3 100755 --- a/tests/queries/0_stateless/00652_mergetree_mutations.sh +++ b/tests/queries/0_stateless/00652_mergetree_mutations.sh @@ -70,7 +70,23 @@ sleep 1 ${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_cleaner(x) VALUES (4)" sleep 0.1 +for i in {1..10} +do + + if [ $(${CLICKHOUSE_CLIENT} --query="SELECT count() FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner'") -eq 2 ]; then + break + fi + + if [[ $i -eq 100 ]]; then + echo "Timed out while waiting for outdated mutation record to be deleted!" + fi + + sleep 1 + ${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_cleaner(x) VALUES (4)" +done + # Check that the first mutation is cleaned ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, is_done FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner' ORDER BY mutation_id" ${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner" + From 16fd24fb1f82f109e3ac34665941c2acea7bf697 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 13 Aug 2024 13:51:24 +0000 Subject: [PATCH 630/644] Update fuzzer dictionary as per commit 99282e526a8aeb175e9f3e69fc9385070d03798a Also update README so that we have consistent behavior of sort across macOS and Linux. --- tests/fuzz/README.md | 4 +- tests/fuzz/all.dict | 926 ++-- tests/fuzz/dictionaries/datatypes.dict | 4416 ++++++++++++++++- tests/fuzz/dictionaries/functions.dict | 110 + .../{key_words.dict => keywords.dict} | 25 +- 5 files changed, 4895 insertions(+), 586 deletions(-) rename tests/fuzz/dictionaries/{key_words.dict => keywords.dict} (95%) diff --git a/tests/fuzz/README.md b/tests/fuzz/README.md index 6b5b161b2d5..576ad66ed93 100644 --- a/tests/fuzz/README.md +++ b/tests/fuzz/README.md @@ -13,11 +13,11 @@ The list of datatypes generated via the following query: The list of keywords generated via the following query: ``` - clickhouse client -q "SELECT DISTINCT concat('\"', keyword, '\"') as res FROM system.keywords ORDER BY keyword" > key_words.dict + clickhouse client -q "SELECT DISTINCT concat('\"', keyword, '\"') as res FROM system.keywords ORDER BY keyword" > keywords.dict ``` Then merge all dictionaries into one (all.dict) ``` - cat ./dictionaries/* | sort | uniq > all.dict + cat ./dictionaries/* | LC_ALL=C sort | uniq > all.dict ``` \ No newline at end of file diff --git a/tests/fuzz/all.dict b/tests/fuzz/all.dict index f08e319f0d4..1c3c657d6b0 100644 --- a/tests/fuzz/all.dict +++ b/tests/fuzz/all.dict @@ -4,26 +4,26 @@ "accurateCastOrNull" "acos" "acosh" -"ADD" "ADD COLUMN" "ADD CONSTRAINT" +"ADD INDEX" +"ADD PROJECTION" +"ADD STATISTICS" +"ADD" "addDate" "addDays" "addHours" -"ADD INDEX" "addInterval" "addMicroseconds" "addMilliseconds" "addMinutes" "addMonths" "addNanoseconds" -"ADD PROJECTION" "addQuarters" "addressToLine" "addressToLineWithInlines" "addressToSymbol" "addSeconds" -"ADD STATISTIC" "addTupleOfIntervals" "addWeeks" "addYears" @@ -53,7 +53,6 @@ "ALL" "ALLOWED_LATENESS" "alphaTokens" -"ALTER" "ALTER COLUMN" "ALTER DATABASE" "ALTER LIVE VIEW" @@ -66,6 +65,7 @@ "ALTER TABLE" "ALTER TEMPORARY TABLE" "ALTER USER" +"ALTER" "analysisOfVariance" "analysisOfVarianceArgMax" "analysisOfVarianceArgMin" @@ -81,9 +81,9 @@ "analysisOfVarianceResample" "analysisOfVarianceSimpleState" "analysisOfVarianceState" -"and" -"AND" "AND STDOUT" +"AND" +"and" "anova" "anovaArgMax" "anovaArgMin" @@ -100,8 +100,8 @@ "anovaSimpleState" "anovaState" "ANTI" -"any" "ANY" +"any" "anyArgMax" "anyArgMin" "anyArray" @@ -136,6 +136,8 @@ "anyLastOrDefault" "anyLastOrNull" "anyLastResample" +"anyLastSimpleState" +"anyLastState" "anyLast_respect_nulls" "anyLast_respect_nullsArgMax" "anyLast_respect_nullsArgMin" @@ -151,14 +153,14 @@ "anyLast_respect_nullsResample" "anyLast_respect_nullsSimpleState" "anyLast_respect_nullsState" -"anyLastSimpleState" -"anyLastState" "anyMap" "anyMerge" "anyNull" "anyOrDefault" "anyOrNull" "anyResample" +"anySimpleState" +"anyState" "any_respect_nulls" "any_respect_nullsArgMax" "any_respect_nullsArgMin" @@ -174,8 +176,6 @@ "any_respect_nullsResample" "any_respect_nullsSimpleState" "any_respect_nullsState" -"anySimpleState" -"anyState" "any_value" "any_valueArgMax" "any_valueArgMin" @@ -189,6 +189,8 @@ "any_valueOrDefault" "any_valueOrNull" "any_valueResample" +"any_valueSimpleState" +"any_valueState" "any_value_respect_nulls" "any_value_respect_nullsArgMax" "any_value_respect_nullsArgMin" @@ -204,12 +206,10 @@ "any_value_respect_nullsResample" "any_value_respect_nullsSimpleState" "any_value_respect_nullsState" -"any_valueSimpleState" -"any_valueState" "APPEND" "appendTrailingCharIfAbsent" -"APPLY" "APPLY DELETED MASK" +"APPLY" "approx_top_count" "approx_top_countArgMax" "approx_top_countArgMin" @@ -285,43 +285,14 @@ "argMinResample" "argMinSimpleState" "argMinState" -"array" +"ARRAY JOIN" "Array" -"array_agg" -"array_aggArgMax" -"array_aggArgMin" -"array_aggArray" -"array_aggDistinct" -"array_aggForEach" -"array_aggIf" -"array_aggMap" -"array_aggMerge" -"array_aggNull" -"array_aggOrDefault" -"array_aggOrNull" -"array_aggResample" -"array_aggSimpleState" -"array_aggState" +"array" "arrayAll" "arrayAUC" "arrayAvg" "arrayCompact" "arrayConcat" -"array_concat_agg" -"array_concat_aggArgMax" -"array_concat_aggArgMin" -"array_concat_aggArray" -"array_concat_aggDistinct" -"array_concat_aggForEach" -"array_concat_aggIf" -"array_concat_aggMap" -"array_concat_aggMerge" -"array_concat_aggNull" -"array_concat_aggOrDefault" -"array_concat_aggOrNull" -"array_concat_aggResample" -"array_concat_aggSimpleState" -"array_concat_aggState" "arrayCount" "arrayCumSum" "arrayCumSumNonNegative" @@ -345,7 +316,6 @@ "arrayIntersect" "arrayJaccardIndex" "arrayJoin" -"ARRAY JOIN" "arrayLast" "arrayLastIndex" "arrayLastOrNull" @@ -382,6 +352,36 @@ "arrayUniq" "arrayWithConstant" "arrayZip" +"array_agg" +"array_aggArgMax" +"array_aggArgMin" +"array_aggArray" +"array_aggDistinct" +"array_aggForEach" +"array_aggIf" +"array_aggMap" +"array_aggMerge" +"array_aggNull" +"array_aggOrDefault" +"array_aggOrNull" +"array_aggResample" +"array_aggSimpleState" +"array_aggState" +"array_concat_agg" +"array_concat_aggArgMax" +"array_concat_aggArgMin" +"array_concat_aggArray" +"array_concat_aggDistinct" +"array_concat_aggForEach" +"array_concat_aggIf" +"array_concat_aggMap" +"array_concat_aggMerge" +"array_concat_aggNull" +"array_concat_aggOrDefault" +"array_concat_aggOrNull" +"array_concat_aggResample" +"array_concat_aggSimpleState" +"array_concat_aggState" "AS" "ASC" "ASCENDING" @@ -396,7 +396,6 @@ "atan" "atan2" "atanh" -"ATTACH" "ATTACH PART" "ATTACH PARTITION" "ATTACH POLICY" @@ -406,6 +405,7 @@ "ATTACH ROW POLICY" "ATTACH SETTINGS PROFILE" "ATTACH USER" +"ATTACH" "AUTO_INCREMENT" "avg" "avgArgMax" @@ -445,39 +445,17 @@ "base58Encode" "base64Decode" "base64Encode" -"base_backup" +"base64URLDecode" +"base64URLEncode" "basename" +"base_backup" "BCRYPT_HASH" "BCRYPT_PASSWORD" "BEGIN TRANSACTION" "BETWEEN" "BIDIRECTIONAL" -"BIGINT" -"BIGINT SIGNED" -"BIGINT UNSIGNED" "bin" -"BINARY" -"BINARY LARGE OBJECT" -"BINARY VARYING" -"BIT" "bitAnd" -"BIT_AND" -"BIT_ANDArgMax" -"BIT_ANDArgMin" -"BIT_ANDArray" -"BIT_ANDDistinct" -"BIT_ANDForEach" -"BIT_ANDIf" -"BIT_ANDMap" -"BIT_ANDMerge" -"BIT_ANDNull" -"BIT_ANDOrDefault" -"BIT_ANDOrNull" -"BIT_ANDResample" -"BIT_ANDSimpleState" -"BIT_ANDState" -"__bitBoolMaskAnd" -"__bitBoolMaskOr" "bitCount" "bitHammingDistance" "bitmapAnd" @@ -503,6 +481,31 @@ "bitmaskToList" "bitNot" "bitOr" +"bitPositionsToArray" +"bitRotateLeft" +"bitRotateRight" +"bitShiftLeft" +"bitShiftRight" +"bitSlice" +"bitTest" +"bitTestAll" +"bitTestAny" +"bitXor" +"BIT_AND" +"BIT_ANDArgMax" +"BIT_ANDArgMin" +"BIT_ANDArray" +"BIT_ANDDistinct" +"BIT_ANDForEach" +"BIT_ANDIf" +"BIT_ANDMap" +"BIT_ANDMerge" +"BIT_ANDNull" +"BIT_ANDOrDefault" +"BIT_ANDOrNull" +"BIT_ANDResample" +"BIT_ANDSimpleState" +"BIT_ANDState" "BIT_OR" "BIT_ORArgMax" "BIT_ORArgMin" @@ -518,18 +521,6 @@ "BIT_ORResample" "BIT_ORSimpleState" "BIT_ORState" -"bitPositionsToArray" -"bitRotateLeft" -"bitRotateRight" -"bitShiftLeft" -"bitShiftRight" -"bitSlice" -"__bitSwapLastTwo" -"bitTest" -"bitTestAll" -"bitTestAny" -"__bitWrapperFunc" -"bitXor" "BIT_XOR" "BIT_XORArgMax" "BIT_XORArgMin" @@ -546,13 +537,9 @@ "BIT_XORSimpleState" "BIT_XORState" "BLAKE3" -"BLOB" "blockNumber" "blockSerializedSize" "blockSize" -"bool" -"Bool" -"boolean" "BOTH" "boundingRatio" "boundingRatioArgMax" @@ -571,8 +558,6 @@ "boundingRatioState" "buildId" "BY" -"BYTE" -"BYTEA" "byteHammingDistance" "byteSize" "byteSlice" @@ -583,7 +568,6 @@ "caseWithExpression" "caseWithoutExpr" "caseWithoutExpression" -"_CAST" "CAST" "catboostEvaluate" "categoricalInformationValue" @@ -607,36 +591,41 @@ "CHANGE" "CHANGEABLE_IN_READONLY" "CHANGED" -"char" -"CHAR" -"CHARACTER" -"CHARACTER LARGE OBJECT" -"CHARACTER_LENGTH" -"CHARACTER VARYING" -"CHAR LARGE OBJECT" -"CHAR_LENGTH" +"changeDay" +"changeHour" +"changeMinute" +"changeMonth" +"changeSecond" +"changeYear" "CHAR VARYING" -"CHECK" +"CHAR" +"char" +"CHARACTER LARGE OBJECT" +"CHARACTER VARYING" +"CHARACTER" +"CHARACTER_LENGTH" +"CHAR_LENGTH" "CHECK ALL TABLES" "CHECK TABLE" +"CHECK" "cityHash64" +"clamp" "CLEANUP" "CLEAR COLUMN" "CLEAR INDEX" "CLEAR PROJECTION" -"CLEAR STATISTIC" -"CLOB" +"CLEAR STATISTICS" "CLUSTER" -"cluster_host_ids" "CLUSTERS" +"cluster_host_ids" "CN" "coalesce" "CODEC" "COLLATE" "COLUMN" "COLUMNS" -"COMMENT" "COMMENT COLUMN" +"COMMENT" "COMMIT" "COMPRESSION" "concat" @@ -644,8 +633,8 @@ "concatWithSeparator" "concatWithSeparatorAssumeInjective" "concat_ws" -"connection_id" "connectionId" +"connection_id" "CONST" "CONSTRAINT" "contingency" @@ -735,21 +724,13 @@ "countSubstringsCaseInsensitive" "countSubstringsCaseInsensitiveUTF8" "covarPop" -"COVAR_POP" "covarPopArgMax" -"COVAR_POPArgMax" "covarPopArgMin" -"COVAR_POPArgMin" "covarPopArray" -"COVAR_POPArray" "covarPopDistinct" -"COVAR_POPDistinct" "covarPopForEach" -"COVAR_POPForEach" "covarPopIf" -"COVAR_POPIf" "covarPopMap" -"COVAR_POPMap" "covarPopMatrix" "covarPopMatrixArgMax" "covarPopMatrixArgMin" @@ -766,17 +747,11 @@ "covarPopMatrixSimpleState" "covarPopMatrixState" "covarPopMerge" -"COVAR_POPMerge" "covarPopNull" -"COVAR_POPNull" "covarPopOrDefault" -"COVAR_POPOrDefault" "covarPopOrNull" -"COVAR_POPOrNull" "covarPopResample" -"COVAR_POPResample" "covarPopSimpleState" -"COVAR_POPSimpleState" "covarPopStable" "covarPopStableArgMax" "covarPopStableArgMin" @@ -793,23 +768,14 @@ "covarPopStableSimpleState" "covarPopStableState" "covarPopState" -"COVAR_POPState" "covarSamp" -"COVAR_SAMP" "covarSampArgMax" -"COVAR_SAMPArgMax" "covarSampArgMin" -"COVAR_SAMPArgMin" "covarSampArray" -"COVAR_SAMPArray" "covarSampDistinct" -"COVAR_SAMPDistinct" "covarSampForEach" -"COVAR_SAMPForEach" "covarSampIf" -"COVAR_SAMPIf" "covarSampMap" -"COVAR_SAMPMap" "covarSampMatrix" "covarSampMatrixArgMax" "covarSampMatrixArgMin" @@ -826,17 +792,11 @@ "covarSampMatrixSimpleState" "covarSampMatrixState" "covarSampMerge" -"COVAR_SAMPMerge" "covarSampNull" -"COVAR_SAMPNull" "covarSampOrDefault" -"COVAR_SAMPOrDefault" "covarSampOrNull" -"COVAR_SAMPOrNull" "covarSampResample" -"COVAR_SAMPResample" "covarSampSimpleState" -"COVAR_SAMPSimpleState" "covarSampStable" "covarSampStableArgMax" "covarSampStableArgMin" @@ -853,6 +813,35 @@ "covarSampStableSimpleState" "covarSampStableState" "covarSampState" +"COVAR_POP" +"COVAR_POPArgMax" +"COVAR_POPArgMin" +"COVAR_POPArray" +"COVAR_POPDistinct" +"COVAR_POPForEach" +"COVAR_POPIf" +"COVAR_POPMap" +"COVAR_POPMerge" +"COVAR_POPNull" +"COVAR_POPOrDefault" +"COVAR_POPOrNull" +"COVAR_POPResample" +"COVAR_POPSimpleState" +"COVAR_POPState" +"COVAR_SAMP" +"COVAR_SAMPArgMax" +"COVAR_SAMPArgMin" +"COVAR_SAMPArray" +"COVAR_SAMPDistinct" +"COVAR_SAMPForEach" +"COVAR_SAMPIf" +"COVAR_SAMPMap" +"COVAR_SAMPMerge" +"COVAR_SAMPNull" +"COVAR_SAMPOrDefault" +"COVAR_SAMPOrNull" +"COVAR_SAMPResample" +"COVAR_SAMPSimpleState" "COVAR_SAMPState" "cramersV" "cramersVArgMax" @@ -887,7 +876,6 @@ "CRC32" "CRC32IEEE" "CRC64" -"CREATE" "CREATE POLICY" "CREATE PROFILE" "CREATE QUOTA" @@ -897,25 +885,27 @@ "CREATE TABLE" "CREATE TEMPORARY TABLE" "CREATE USER" +"CREATE" "CROSS" "CUBE" "curdate" -"current_database" -"currentDatabase" -"current_date" "CURRENT GRANTS" -"currentProfiles" "CURRENT QUOTA" -"currentRoles" "CURRENT ROLES" "CURRENT ROW" -"current_schemas" -"currentSchemas" -"current_timestamp" "CURRENT TRANSACTION" -"currentUser" -"CURRENT_USER" +"currentDatabase" +"currentProfiles" +"currentRoles" +"currentSchemas" "CURRENTUSER" +"currentUser" +"current_database" +"current_date" +"current_schemas" +"current_timestamp" +"CURRENT_USER" +"current_user" "cutFragment" "cutIPv6" "cutQueryString" @@ -932,27 +922,29 @@ "cutWWW" "D" "damerauLevenshteinDistance" +"DATA INNER UUID" +"DATA" "DATABASE" "DATABASES" -"Date" "DATE" -"Date32" -"DATE_ADD" +"Date" "DATEADD" -"date_diff" -"dateDiff" -"DATE_DIFF" "DATEDIFF" -"DATE_FORMAT" +"dateDiff" "dateName" -"DATE_SUB" "DATESUB" "DateTime" -"DateTime32" "DateTime64" "dateTime64ToSnowflake" +"dateTime64ToSnowflakeID" "dateTimeToSnowflake" +"dateTimeToSnowflakeID" "dateTrunc" +"DATE_ADD" +"DATE_DIFF" +"date_diff" +"DATE_FORMAT" +"DATE_SUB" "DATE_TRUNC" "DAY" "DAYOFMONTH" @@ -960,10 +952,8 @@ "DAYOFYEAR" "DAYS" "DD" -"DEC" "Decimal" "Decimal128" -"Decimal256" "Decimal32" "Decimal64" "decodeHTMLComponent" @@ -972,17 +962,17 @@ "decodeXMLComponent" "decrypt" "DEDUPLICATE" -"DEFAULT" "DEFAULT DATABASE" -"defaultProfiles" "DEFAULT ROLE" +"DEFAULT" +"defaultProfiles" "defaultRoles" "defaultValueOfArgumentType" "defaultValueOfTypeName" "DEFINER" "degrees" -"DELETE" "DELETE WHERE" +"DELETE" "deltaSum" "deltaSumArgMax" "deltaSumArgMin" @@ -1014,6 +1004,21 @@ "deltaSumTimestampSimpleState" "deltaSumTimestampState" "demangle" +"denseRank" +"denseRankArgMax" +"denseRankArgMin" +"denseRankArray" +"denseRankDistinct" +"denseRankForEach" +"denseRankIf" +"denseRankMap" +"denseRankMerge" +"denseRankNull" +"denseRankOrDefault" +"denseRankOrNull" +"denseRankResample" +"denseRankSimpleState" +"denseRankState" "dense_rank" "dense_rankArgMax" "dense_rankArgMin" @@ -1033,9 +1038,9 @@ "DESC" "DESCENDING" "DESCRIBE" -"DETACH" "DETACH PART" "DETACH PARTITION" +"DETACH" "detectCharset" "detectLanguage" "detectLanguageMixed" @@ -1092,8 +1097,8 @@ "distanceL2Squared" "distanceLinf" "distanceLp" -"DISTINCT" "DISTINCT ON" +"DISTINCT" "DIV" "divide" "divideDecimal" @@ -1102,11 +1107,8 @@ "domainWithoutWWW" "domainWithoutWWWRFC" "dotProduct" -"DOUBLE" -"DOUBLE PRECISION" "DOUBLE_SHA1_HASH" "DOUBLE_SHA1_PASSWORD" -"DROP" "DROP COLUMN" "DROP CONSTRAINT" "DROP DEFAULT" @@ -1116,15 +1118,20 @@ "DROP PART" "DROP PARTITION" "DROP PROJECTION" -"DROP STATISTIC" +"DROP STATISTICS" "DROP TABLE" "DROP TEMPORARY TABLE" +"DROP" "dumpColumnStructure" +"dynamicElement" +"dynamicType" "e" "editDistance" +"editDistanceUTF8" "ELSE" -"empty" +"EMPTY AS" "EMPTY" +"empty" "emptyArrayDate" "emptyArrayDateTime" "emptyArrayFloat32" @@ -1139,10 +1146,9 @@ "emptyArrayUInt32" "emptyArrayUInt64" "emptyArrayUInt8" -"EMPTY AS" +"ENABLED ROLES" "enabledProfiles" "enabledRoles" -"ENABLED ROLES" "encodeURLComponent" "encodeURLFormComponent" "encodeXMLComponent" @@ -1168,11 +1174,10 @@ "entropySimpleState" "entropyState" "Enum" -"ENUM" "Enum16" "Enum8" -"EPHEMERAL" "EPHEMERAL SEQUENTIAL" +"EPHEMERAL" "equals" "erf" "erfc" @@ -1182,11 +1187,11 @@ "EVENT" "EVENTS" "EVERY" -"EXCEPT" "EXCEPT DATABASE" "EXCEPT DATABASES" "EXCEPT TABLE" "EXCEPT TABLES" +"EXCEPT" "EXCHANGE DICTIONARIES" "EXCHANGE TABLES" "EXISTS" @@ -1272,8 +1277,8 @@ "EXPRESSION" "EXTENDED" "EXTERNAL DDL FROM" -"extract" "EXTRACT" +"extract" "extractAll" "extractAllGroups" "extractAllGroupsHorizontal" @@ -1289,15 +1294,15 @@ "FALSE" "farmFingerprint64" "farmHash64" -"FETCH" "FETCH PART" "FETCH PARTITION" +"FETCH" "FIELDS" -"file" "FILE" -"filesystemAvailable" +"file" "FILESYSTEM CACHE" "FILESYSTEM CACHES" +"filesystemAvailable" "filesystemCapacity" "filesystemUnreserved" "FILTER" @@ -1322,6 +1327,8 @@ "first_valueOrDefault" "first_valueOrNull" "first_valueResample" +"first_valueSimpleState" +"first_valueState" "first_value_respect_nulls" "first_value_respect_nullsArgMax" "first_value_respect_nullsArgMin" @@ -1337,9 +1344,6 @@ "first_value_respect_nullsResample" "first_value_respect_nullsSimpleState" "first_value_respect_nullsState" -"first_valueSimpleState" -"first_valueState" -"FIXED" "FixedString" "flameGraph" "flameGraphArgMax" @@ -1358,19 +1362,17 @@ "flameGraphState" "flatten" "flattenTuple" -"FLOAT" "Float32" "Float64" "floor" "FOLLOWING" "FOR" "ForEach" -"FOREIGN" "FOREIGN KEY" +"FOREIGN" "FORGET PARTITION" -"format" "FORMAT" -"FORMAT_BYTES" +"format" "formatDateTime" "formatDateTimeInJodaSyntax" "formatQuery" @@ -1383,26 +1385,27 @@ "formatReadableTimeDelta" "formatRow" "formatRowNoNewline" +"FORMAT_BYTES" "FQDN" "fragment" "FREEZE" +"FROM INFILE" +"FROM SHARD" "FROM" -"FROM_BASE64" -"FROM_DAYS" "fromDaysSinceYearZero" "fromDaysSinceYearZero32" -"FROM INFILE" "fromModifiedJulianDay" "fromModifiedJulianDayOrNull" -"FROM SHARD" -"FROM_UNIXTIME" "fromUnixTimestamp" "fromUnixTimestamp64Micro" "fromUnixTimestamp64Milli" "fromUnixTimestamp64Nano" "fromUnixTimestampInJodaSyntax" -"from_utc_timestamp" "fromUTCTimestamp" +"FROM_BASE64" +"FROM_DAYS" +"FROM_UNIXTIME" +"from_utc_timestamp" "FULL" "fullHostName" "FULLTEXT" @@ -1411,44 +1414,46 @@ "gccMurmurHash" "gcd" "generateRandomStructure" +"generateSnowflakeID" "generateULID" "generateUUIDv4" +"generateUUIDv7" "geoDistance" "geohashDecode" "geohashEncode" "geohashesInBox" -"GEOMETRY" "geoToH3" "geoToS2" +"getClientHTTPHeader" "getMacro" "getOSKernelVersion" -"__getScalar" "getServerPort" "getSetting" "getSizeOfEnumType" "getSubcolumn" "getTypeSerializationStreams" +"GLOBAL IN" +"GLOBAL NOT IN" "GLOBAL" "globalIn" -"GLOBAL IN" "globalInIgnoreSet" "globalNotIn" -"GLOBAL NOT IN" "globalNotInIgnoreSet" "globalNotNullIn" "globalNotNullInIgnoreSet" "globalNullIn" "globalNullInIgnoreSet" "globalVariable" +"GRANT OPTION FOR" "GRANT" "GRANTEES" -"GRANT OPTION FOR" "GRANULARITY" "greatCircleAngle" "greatCircleDistance" "greater" "greaterOrEquals" "greatest" +"GROUP BY" "groupArray" "groupArrayArgMax" "groupArrayArgMin" @@ -1673,7 +1678,21 @@ "groupBitXorResample" "groupBitXorSimpleState" "groupBitXorState" -"GROUP BY" +"groupConcat" +"groupConcatArgMax" +"groupConcatArgMin" +"groupConcatArray" +"groupConcatDistinct" +"groupConcatForEach" +"groupConcatIf" +"groupConcatMap" +"groupConcatMerge" +"groupConcatNull" +"groupConcatOrDefault" +"groupConcatOrNull" +"groupConcatResample" +"groupConcatSimpleState" +"groupConcatState" "GROUPING SETS" "GROUPS" "groupUniqArray" @@ -1691,6 +1710,21 @@ "groupUniqArrayResample" "groupUniqArraySimpleState" "groupUniqArrayState" +"group_concat" +"group_concatArgMax" +"group_concatArgMin" +"group_concatArray" +"group_concatDistinct" +"group_concatForEach" +"group_concatIf" +"group_concatMap" +"group_concatMerge" +"group_concatNull" +"group_concatOrDefault" +"group_concatOrNull" +"group_concatResample" +"group_concatSimpleState" +"group_concatState" "H" "h3CellAreaM2" "h3CellAreaRads2" @@ -1753,6 +1787,8 @@ "hex" "HH" "HIERARCHICAL" +"hilbertDecode" +"hilbertEncode" "histogram" "histogramArgMax" "histogramArgMin" @@ -1773,8 +1809,8 @@ "hopEnd" "hopStart" "HOST" -"hostname" "hostName" +"hostname" "HOUR" "HOURS" "HTTP" @@ -1784,25 +1820,24 @@ "identity" "idnaDecode" "idnaEncode" -"if" "IF EMPTY" "IF EXISTS" "IF NOT EXISTS" +"if" "ifNotFinite" "ifNull" -"ignore" "IGNORE NULLS" -"ilike" +"ignore" "ILIKE" -"in" +"ilike" +"IN PARTITION" "IN" +"in" "INDEX" "INDEXES" "indexHint" "indexOf" "INDICES" -"INET4" -"INET6" "INET6_ATON" "INET6_NTOA" "INET_ATON" @@ -1812,28 +1847,18 @@ "initcap" "initcapUTF8" "initializeAggregation" -"initial_query_id" "initialQueryID" +"initial_query_id" "INJECTIVE" "INNER" -"IN PARTITION" "INSERT INTO" "instr" -"INT" -"INT1" -"Int128" "Int16" -"INT1 SIGNED" -"INT1 UNSIGNED" -"Int256" "Int32" "Int64" "Int8" "intDiv" "intDivOrZero" -"INTEGER" -"INTEGER SIGNED" -"INTEGER UNSIGNED" "INTERPOLATE" "INTERSECT" "INTERVAL" @@ -1854,11 +1879,8 @@ "intervalLengthSumResample" "intervalLengthSumSimpleState" "intervalLengthSumState" -"IntervalMicrosecond" -"IntervalMillisecond" "IntervalMinute" "IntervalMonth" -"IntervalNanosecond" "IntervalQuarter" "IntervalSecond" "IntervalWeek" @@ -1868,12 +1890,9 @@ "intHash32" "intHash64" "INTO OUTFILE" -"INT SIGNED" -"INT UNSIGNED" "INVISIBLE" "INVOKER" "IP" -"IPv4" "IPv4CIDRToRange" "IPv4NumToString" "IPv4NumToStringClassC" @@ -1881,12 +1900,14 @@ "IPv4StringToNumOrDefault" "IPv4StringToNumOrNull" "IPv4ToIPv6" -"IPv6" "IPv6CIDRToRange" "IPv6NumToString" "IPv6StringToNum" "IPv6StringToNumOrDefault" "IPv6StringToNumOrNull" +"IS NOT DISTINCT FROM" +"IS NOT NULL" +"IS NULL" "isConstant" "isDecimalOverflow" "isFinite" @@ -1896,16 +1917,13 @@ "isIPv6String" "isNaN" "isNotDistinctFrom" -"IS NOT DISTINCT FROM" "isNotNull" -"IS NOT NULL" "isNull" -"IS NULL" "isNullable" -"IS_OBJECT_ID" "isValidJSON" "isValidUTF8" "isZeroOrNull" +"IS_OBJECT_ID" "jaroSimilarity" "jaroWinklerSimilarity" "javaHash" @@ -1913,10 +1931,7 @@ "JOIN" "joinGet" "joinGetOrNull" -"JSON" "JSONArrayLength" -"JSON_ARRAY_LENGTH" -"JSON_EXISTS" "JSONExtract" "JSONExtractArrayRaw" "JSONExtractBool" @@ -1931,15 +1946,19 @@ "JSONHas" "JSONKey" "JSONLength" +"JSONMergePatch" "jsonMergePatch" -"JSON_QUERY" "JSONType" +"JSON_ARRAY_LENGTH" +"JSON_EXISTS" +"JSON_QUERY" "JSON_VALUE" "jumpConsistentHash" +"JWT" "kafkaMurmurHash" "KERBEROS" -"KEY" "KEY BY" +"KEY" "KEYED BY" "KEYS" "KILL" @@ -2046,6 +2065,8 @@ "last_valueOrDefault" "last_valueOrNull" "last_valueResample" +"last_valueSimpleState" +"last_valueState" "last_value_respect_nulls" "last_value_respect_nullsArgMax" "last_value_respect_nullsArgMin" @@ -2061,8 +2082,6 @@ "last_value_respect_nullsResample" "last_value_respect_nullsSimpleState" "last_value_respect_nullsState" -"last_valueSimpleState" -"last_valueState" "LAYOUT" "lcase" "lcm" @@ -2084,25 +2103,26 @@ "leadInFrameState" "LEADING" "least" -"left" -"LEFT" "LEFT ARRAY JOIN" +"LEFT" +"left" "leftPad" "leftPadUTF8" "leftUTF8" "lemmatize" "length" "lengthUTF8" +"LESS THAN" "less" "lessOrEquals" -"LESS THAN" "LEVEL" "levenshteinDistance" +"levenshteinDistanceUTF8" "lgamma" "LIFETIME" "LIGHTWEIGHT" -"like" "LIKE" +"like" "LIMIT" "LINEAR" "LinfDistance" @@ -2118,8 +2138,6 @@ "log1p" "log2" "logTrace" -"LONGBLOB" -"LONGTEXT" "LowCardinality" "lowCardinalityIndices" "lowCardinalityKeys" @@ -2129,8 +2147,8 @@ "LpDistance" "LpNorm" "LpNormalize" -"ltrim" "LTRIM" +"ltrim" "lttb" "lttbArgMax" "lttbArgMin" @@ -2170,7 +2188,6 @@ "mannWhitneyUTestSimpleState" "mannWhitneyUTestState" "map" -"Map" "mapAdd" "mapAll" "mapApply" @@ -2181,7 +2198,6 @@ "mapExtractKeyLike" "mapFilter" "mapFromArrays" -"MAP_FROM_ARRAYS" "mapFromString" "mapKeys" "mapPartialReverseSort" @@ -2192,18 +2208,19 @@ "mapSubtract" "mapUpdate" "mapValues" -"match" +"MAP_FROM_ARRAYS" "MATCH" -"materialize" -"MATERIALIZE" +"match" "MATERIALIZE COLUMN" -"MATERIALIZED" "MATERIALIZE INDEX" "MATERIALIZE PROJECTION" -"MATERIALIZE STATISTIC" +"MATERIALIZE STATISTICS" "MATERIALIZE TTL" -"max" +"MATERIALIZE" +"materialize" +"MATERIALIZED" "MAX" +"max" "max2" "maxArgMax" "maxArgMin" @@ -2507,14 +2524,11 @@ "medianTimingWeightedResample" "medianTimingWeightedSimpleState" "medianTimingWeightedState" -"MEDIUMBLOB" -"MEDIUMINT" -"MEDIUMINT SIGNED" -"MEDIUMINT UNSIGNED" -"MEDIUMTEXT" "MEMORY" "Merge" "MERGES" +"METRICS INNER UUID" +"METRICS" "metroHash64" "MI" "MICROSECOND" @@ -2522,8 +2536,8 @@ "mid" "MILLISECOND" "MILLISECONDS" -"min" "MIN" +"min" "min2" "minArgMax" "minArgMin" @@ -2562,18 +2576,20 @@ "MINUTES" "mismatches" "MM" -"mod" "MOD" -"MODIFY" +"mod" "MODIFY COLUMN" "MODIFY COMMENT" +"MODIFY DEFINER" "MODIFY ORDER BY" "MODIFY QUERY" "MODIFY REFRESH" "MODIFY SAMPLE BY" "MODIFY SETTING" "MODIFY SQL SECURITY" +"MODIFY STATISTICS" "MODIFY TTL" +"MODIFY" "modulo" "moduloLegacy" "moduloOrZero" @@ -2582,9 +2598,9 @@ "MONTHS" "mortonDecode" "mortonEncode" -"MOVE" "MOVE PART" "MOVE PARTITION" +"MOVE" "movingXXX" "MS" "multiFuzzyMatchAllIndices" @@ -2596,7 +2612,6 @@ "multiMatchAnyIndex" "multiply" "multiplyDecimal" -"MultiPolygon" "multiSearchAllPositions" "multiSearchAllPositionsCaseInsensitive" "multiSearchAllPositionsCaseInsensitiveUTF8" @@ -2624,18 +2639,10 @@ "NAMED COLLECTION" "NANOSECOND" "NANOSECONDS" -"NATIONAL CHAR" -"NATIONAL CHARACTER" -"NATIONAL CHARACTER LARGE OBJECT" -"NATIONAL CHARACTER VARYING" -"NATIONAL CHAR VARYING" -"NCHAR" -"NCHAR LARGE OBJECT" -"NCHAR VARYING" "negate" "neighbor" -"nested" "Nested" +"nested" "netloc" "NEXT" "ngramDistance" @@ -2678,7 +2685,6 @@ "nonNegativeDerivativeResample" "nonNegativeDerivativeSimpleState" "nonNegativeDerivativeState" -"NO_PASSWORD" "normalizedQueryHash" "normalizedQueryHashKeepNames" "normalizeL1" @@ -2696,13 +2702,19 @@ "normL2Squared" "normLinf" "normLp" -"not" -"NOT" "NOT BETWEEN" +"NOT IDENTIFIED" +"NOT ILIKE" +"NOT IN" +"NOT KEYED" +"NOT LIKE" +"NOT OVERRIDABLE" +"NOT" +"not" "notEmpty" "notEquals" -"nothing" "Nothing" +"nothing" "nothingArgMax" "nothingArgMin" "nothingArray" @@ -2746,21 +2758,16 @@ "nothingUInt64Resample" "nothingUInt64SimpleState" "nothingUInt64State" -"NOT IDENTIFIED" "notILike" -"NOT ILIKE" "notIn" -"NOT IN" "notInIgnoreSet" -"NOT KEYED" "notLike" -"NOT LIKE" "notNullIn" "notNullInIgnoreSet" -"NOT OVERRIDABLE" "now" "now64" "nowInBlock" +"NO_PASSWORD" "NS" "nth_value" "nth_valueArgMax" @@ -2792,28 +2799,25 @@ "ntileResample" "ntileSimpleState" "ntileState" -"Null" "NULL" +"Null" "Nullable" "nullIf" "nullIn" "nullInIgnoreSet" "NULLS" -"NUMERIC" -"NVARCHAR" -"Object" "OCTET_LENGTH" "OFFSET" -"ON" "ON DELETE" -"ONLY" "ON UPDATE" "ON VOLUME" +"ON" +"ONLY" "OPTIMIZE TABLE" -"or" -"OR" -"ORDER BY" "OR REPLACE" +"OR" +"or" +"ORDER BY" "OUTER" "OVER" "OVERRIDABLE" @@ -2838,32 +2842,64 @@ "parseDateTimeInJodaSyntaxOrZero" "parseDateTimeOrNull" "parseDateTimeOrZero" +"parseReadableSize" +"parseReadableSizeOrNull" +"parseReadableSizeOrZero" "parseTimeDelta" "PART" "PARTIAL" -"PARTITION" "PARTITION BY" +"PARTITION" +"partitionID" "partitionId" "PARTITIONS" "PART_MOVE_TO_SHARD" "PASTE" "path" "pathFull" +"percentRank" +"percentRankArgMax" +"percentRankArgMin" +"percentRankArray" +"percentRankDistinct" +"percentRankForEach" +"percentRankIf" +"percentRankMap" +"percentRankMerge" +"percentRankNull" +"percentRankOrDefault" +"percentRankOrNull" +"percentRankResample" +"percentRankSimpleState" +"percentRankState" +"percent_rank" +"percent_rankArgMax" +"percent_rankArgMin" +"percent_rankArray" +"percent_rankDistinct" +"percent_rankForEach" +"percent_rankIf" +"percent_rankMap" +"percent_rankMerge" +"percent_rankNull" +"percent_rankOrDefault" +"percent_rankOrNull" +"percent_rankResample" +"percent_rankSimpleState" +"percent_rankState" "PERIODIC REFRESH" "PERMANENTLY" "PERMISSIVE" -"PERSISTENT" "PERSISTENT SEQUENTIAL" +"PERSISTENT" "pi" "PIPELINE" "PLAINTEXT_PASSWORD" "PLAN" "plus" "pmod" -"Point" "pointInEllipses" "pointInPolygon" -"Polygon" "polygonAreaCartesian" "polygonAreaSpherical" "polygonConvexHullCartesian" @@ -2887,15 +2923,16 @@ "positionCaseInsensitive" "positionCaseInsensitiveUTF8" "positionUTF8" -"positive_modulo" "positiveModulo" +"positive_modulo" "pow" "power" "PRECEDING" "PRECISION" "PREWHERE" -"PRIMARY" "PRIMARY KEY" +"PRIMARY" +"printf" "PROFILE" "PROJECTION" "proportionsZTest" @@ -2906,6 +2943,7 @@ "punycodeEncode" "Q" "QQ" +"QUALIFY" "quantile" "quantileArgMax" "quantileArgMin" @@ -3418,12 +3456,12 @@ "quantileTimingWeightedState" "QUARTER" "QUARTERS" +"QUERY TREE" "QUERY" -"query_id" "queryID" "queryString" "queryStringAndFragment" -"QUERY TREE" +"query_id" "QUOTA" "radians" "rand" @@ -3440,16 +3478,16 @@ "randNegativeBinomial" "randNormal" "randomFixedString" -"RANDOMIZED" "RANDOMIZE FOR" +"RANDOMIZED" "randomPrintableASCII" "randomString" "randomStringUTF8" "randPoisson" "randStudentT" "randUniform" -"range" "RANGE" +"range" "rank" "rankArgMax" "rankArgMin" @@ -3481,20 +3519,22 @@ "rankSimpleState" "rankState" "READONLY" +"readWKTLineString" +"readWKTMultiLineString" "readWKTMultiPolygon" "readWKTPoint" "readWKTPolygon" "readWKTRing" -"REAL" "REALM" "RECOMPRESS" +"RECURSIVE" "REFERENCES" "REFRESH" "REGEXP" "regexpExtract" +"regexpQuoteMeta" "REGEXP_EXTRACT" "REGEXP_MATCHES" -"regexpQuoteMeta" "REGEXP_REPLACE" "regionHierarchy" "regionIn" @@ -3526,21 +3566,21 @@ "reinterpretAsUInt64" "reinterpretAsUInt8" "reinterpretAsUUID" -"REMOVE" "REMOVE SAMPLE BY" "REMOVE TTL" -"RENAME" +"REMOVE" "RENAME COLUMN" "RENAME DATABASE" "RENAME DICTIONARY" "RENAME TABLE" "RENAME TO" +"RENAME" "repeat" -"replace" +"REPLACE PARTITION" "REPLACE" +"replace" "replaceAll" "replaceOne" -"REPLACE PARTITION" "replaceRegexpAll" "replaceRegexpOne" "replicate" @@ -3570,12 +3610,11 @@ "reverseUTF8" "revision" "REVOKE" -"right" "RIGHT" +"right" "rightPad" "rightPadUTF8" "rightUTF8" -"Ring" "ROLLBACK" "ROLLUP" "round" @@ -3585,6 +3624,9 @@ "roundDuration" "roundToExp2" "ROW" +"rowNumberInAllBlocks" +"rowNumberInBlock" +"ROWS" "row_number" "row_numberArgMax" "row_numberArgMin" @@ -3592,8 +3634,6 @@ "row_numberDistinct" "row_numberForEach" "row_numberIf" -"rowNumberInAllBlocks" -"rowNumberInBlock" "row_numberMap" "row_numberMerge" "row_numberNull" @@ -3602,10 +3642,9 @@ "row_numberResample" "row_numberSimpleState" "row_numberState" -"ROWS" "rpad" -"rtrim" "RTRIM" +"rtrim" "runningAccumulate" "runningConcurrency" "runningDifference" @@ -3622,10 +3661,10 @@ "s2ToGeo" "S3" "SALT" -"SAMPLE" "SAMPLE BY" +"SAMPLE" +"SAN" "scalarProduct" -"__scalarSubqueryResult" "SCHEMA" "SCHEME" "SECOND" @@ -3681,18 +3720,18 @@ "seriesOutliersDetectTukey" "seriesPeriodDetectFFT" "SERVER" -"serverTimezone" "serverTimeZone" +"serverTimezone" "serverUUID" -"SET" -"SET DEFAULT" "SET DEFAULT ROLE" +"SET DEFAULT" "SET FAKE TIME" "SET NULL" -"SET ROLE" "SET ROLE DEFAULT" -"SETTINGS" +"SET ROLE" "SET TRANSACTION SNAPSHOT" +"SET" +"SETTINGS" "SHA1" "SHA224" "SHA256" @@ -3703,9 +3742,7 @@ "SHA512_256" "shardCount" "shardNum" -"SHOW" "SHOW ACCESS" -"showCertificate" "SHOW CREATE" "SHOW ENGINES" "SHOW FUNCTIONS" @@ -3713,11 +3750,12 @@ "SHOW PRIVILEGES" "SHOW PROCESSLIST" "SHOW SETTING" +"SHOW" +"showCertificate" "sigmoid" "sign" "SIGNED" "SIMPLE" -"SimpleAggregateFunction" "simpleJSONExtractBool" "simpleJSONExtractFloat" "simpleJSONExtractInt" @@ -3741,7 +3779,6 @@ "simpleLinearRegressionSimpleState" "simpleLinearRegressionState" "sin" -"SINGLE" "singleValueOrNull" "singleValueOrNullArgMax" "singleValueOrNullArgMin" @@ -3796,44 +3833,43 @@ "skewSampState" "sleep" "sleepEachRow" -"SMALLINT" -"SMALLINT SIGNED" -"SMALLINT UNSIGNED" +"snowflakeIDToDateTime" +"snowflakeIDToDateTime64" "snowflakeToDateTime" "snowflakeToDateTime64" "soundex" "SOURCE" "space" -"sparkbar" "sparkBar" -"sparkbarArgMax" +"sparkbar" "sparkBarArgMax" -"sparkbarArgMin" +"sparkbarArgMax" "sparkBarArgMin" -"sparkbarArray" +"sparkbarArgMin" "sparkBarArray" -"sparkbarDistinct" +"sparkbarArray" "sparkBarDistinct" -"sparkbarForEach" +"sparkbarDistinct" "sparkBarForEach" -"sparkbarIf" +"sparkbarForEach" "sparkBarIf" -"sparkbarMap" +"sparkbarIf" "sparkBarMap" -"sparkbarMerge" +"sparkbarMap" "sparkBarMerge" -"sparkbarNull" +"sparkbarMerge" "sparkBarNull" -"sparkbarOrDefault" +"sparkbarNull" "sparkBarOrDefault" -"sparkbarOrNull" +"sparkbarOrDefault" "sparkBarOrNull" -"sparkbarResample" +"sparkbarOrNull" "sparkBarResample" -"sparkbarSimpleState" +"sparkbarResample" "sparkBarSimpleState" -"sparkbarState" +"sparkbarSimpleState" "sparkBarState" +"sparkbarState" "SPATIAL" "splitByAlpha" "splitByChar" @@ -3860,42 +3896,29 @@ "SS" "SSH_KEY" "SSL_CERTIFICATE" +"START TRANSACTION" "startsWith" "startsWithUTF8" "State" -"STATISTIC" +"STATISTICS" "STD" "STDArgMax" "STDArgMin" "STDArray" "stddevPop" -"STDDEV_POP" "stddevPopArgMax" -"STDDEV_POPArgMax" "stddevPopArgMin" -"STDDEV_POPArgMin" "stddevPopArray" -"STDDEV_POPArray" "stddevPopDistinct" -"STDDEV_POPDistinct" "stddevPopForEach" -"STDDEV_POPForEach" "stddevPopIf" -"STDDEV_POPIf" "stddevPopMap" -"STDDEV_POPMap" "stddevPopMerge" -"STDDEV_POPMerge" "stddevPopNull" -"STDDEV_POPNull" "stddevPopOrDefault" -"STDDEV_POPOrDefault" "stddevPopOrNull" -"STDDEV_POPOrNull" "stddevPopResample" -"STDDEV_POPResample" "stddevPopSimpleState" -"STDDEV_POPSimpleState" "stddevPopStable" "stddevPopStableArgMax" "stddevPopStableArgMin" @@ -3912,35 +3935,20 @@ "stddevPopStableSimpleState" "stddevPopStableState" "stddevPopState" -"STDDEV_POPState" "stddevSamp" -"STDDEV_SAMP" "stddevSampArgMax" -"STDDEV_SAMPArgMax" "stddevSampArgMin" -"STDDEV_SAMPArgMin" "stddevSampArray" -"STDDEV_SAMPArray" "stddevSampDistinct" -"STDDEV_SAMPDistinct" "stddevSampForEach" -"STDDEV_SAMPForEach" "stddevSampIf" -"STDDEV_SAMPIf" "stddevSampMap" -"STDDEV_SAMPMap" "stddevSampMerge" -"STDDEV_SAMPMerge" "stddevSampNull" -"STDDEV_SAMPNull" "stddevSampOrDefault" -"STDDEV_SAMPOrDefault" "stddevSampOrNull" -"STDDEV_SAMPOrNull" "stddevSampResample" -"STDDEV_SAMPResample" "stddevSampSimpleState" -"STDDEV_SAMPSimpleState" "stddevSampStable" "stddevSampStableArgMax" "stddevSampStableArgMin" @@ -3957,6 +3965,35 @@ "stddevSampStableSimpleState" "stddevSampStableState" "stddevSampState" +"STDDEV_POP" +"STDDEV_POPArgMax" +"STDDEV_POPArgMin" +"STDDEV_POPArray" +"STDDEV_POPDistinct" +"STDDEV_POPForEach" +"STDDEV_POPIf" +"STDDEV_POPMap" +"STDDEV_POPMerge" +"STDDEV_POPNull" +"STDDEV_POPOrDefault" +"STDDEV_POPOrNull" +"STDDEV_POPResample" +"STDDEV_POPSimpleState" +"STDDEV_POPState" +"STDDEV_SAMP" +"STDDEV_SAMPArgMax" +"STDDEV_SAMPArgMin" +"STDDEV_SAMPArray" +"STDDEV_SAMPDistinct" +"STDDEV_SAMPForEach" +"STDDEV_SAMPIf" +"STDDEV_SAMPMap" +"STDDEV_SAMPMerge" +"STDDEV_SAMPNull" +"STDDEV_SAMPOrDefault" +"STDDEV_SAMPOrNull" +"STDDEV_SAMPResample" +"STDDEV_SAMPSimpleState" "STDDEV_SAMPState" "STDDistinct" "STDForEach" @@ -4008,10 +4045,10 @@ "stringJaccardIndex" "stringJaccardIndexUTF8" "stringToH3" -"str_to_date" -"str_to_map" "structureToCapnProtoSchema" "structureToProtobufSchema" +"str_to_date" +"str_to_map" "studentTTest" "studentTTestArgMax" "studentTTestArgMin" @@ -4029,16 +4066,16 @@ "studentTTestState" "subBitmap" "subDate" -"SUBPARTITION" "SUBPARTITION BY" +"SUBPARTITION" "SUBPARTITIONS" "substr" -"substring" "SUBSTRING" +"substring" "substringIndex" -"SUBSTRING_INDEX" "substringIndexUTF8" "substringUTF8" +"SUBSTRING_INDEX" "subtractDays" "subtractHours" "subtractInterval" @@ -4173,22 +4210,23 @@ "sumWithOverflowSimpleState" "sumWithOverflowState" "SUSPEND" -"svg" "SVG" +"svg" "SYNC" "synonyms" "SYNTAX" "SYSTEM" -"TABLE" "TABLE OVERRIDE" +"TABLE" "TABLES" +"TAGS INNER UUID" +"TAGS" "tan" "tanh" "tcpPort" -"TEMPORARY" "TEMPORARY TABLE" +"TEMPORARY" "TEST" -"TEXT" "tgamma" "theilsU" "theilsUArgMax" @@ -4208,33 +4246,31 @@ "THEN" "throwIf" "tid" -"TIME" "timeDiff" "timeSlot" "timeSlots" -"timestamp" "TIMESTAMP" -"TIMESTAMP_ADD" +"timestamp" "TIMESTAMPADD" -"timestamp_diff" -"timestampDiff" -"TIMESTAMP_DIFF" "TIMESTAMPDIFF" -"TIMESTAMP_SUB" +"timestampDiff" "TIMESTAMPSUB" -"timezone" +"TIMESTAMP_ADD" +"TIMESTAMP_DIFF" +"timestamp_diff" +"TIMESTAMP_SUB" "timeZone" -"timezoneOf" +"timezone" "timeZoneOf" -"timezoneOffset" +"timezoneOf" "timeZoneOffset" -"TINYBLOB" -"TINYINT" -"TINYINT SIGNED" -"TINYINT UNSIGNED" -"TINYTEXT" +"timezoneOffset" +"TO DISK" +"TO INNER UUID" +"TO SHARD" +"TO TABLE" +"TO VOLUME" "TO" -"TO_BASE64" "toBool" "toColumnTypeName" "toDate" @@ -4258,7 +4294,6 @@ "toDayOfMonth" "toDayOfWeek" "toDayOfYear" -"TO_DAYS" "toDaysSinceYearZero" "toDecimal128" "toDecimal128OrDefault" @@ -4277,7 +4312,6 @@ "toDecimal64OrNull" "toDecimal64OrZero" "toDecimalString" -"TO DISK" "toFixedString" "toFloat32" "toFloat32OrDefault" @@ -4288,7 +4322,6 @@ "toFloat64OrNull" "toFloat64OrZero" "toHour" -"TO INNER UUID" "toInt128" "toInt128OrDefault" "toInt128OrNull" @@ -4389,7 +4422,6 @@ "toRelativeWeekNum" "toRelativeYearNum" "toSecond" -"TO SHARD" "toStartOfDay" "toStartOfFifteenMinutes" "toStartOfFiveMinute" @@ -4409,11 +4441,10 @@ "toStartOfYear" "toString" "toStringCutToZero" -"TO TABLE" "TOTALS" "toTime" -"toTimezone" "toTimeZone" +"toTimezone" "toTypeName" "toUInt128" "toUInt128OrDefault" @@ -4439,25 +4470,26 @@ "toUInt8OrDefault" "toUInt8OrNull" "toUInt8OrZero" -"TO_UNIXTIME" "toUnixTimestamp" "toUnixTimestamp64Micro" "toUnixTimestamp64Milli" "toUnixTimestamp64Nano" -"to_utc_timestamp" "toUTCTimestamp" "toUUID" "toUUIDOrDefault" "toUUIDOrNull" "toUUIDOrZero" "toValidUTF8" -"TO VOLUME" "toWeek" "toYear" "toYearWeek" "toYYYYMM" "toYYYYMMDD" "toYYYYMMDDhhmmss" +"TO_BASE64" +"TO_DAYS" +"TO_UNIXTIME" +"to_utc_timestamp" "TRACKING ONLY" "TRAILING" "TRANSACTION" @@ -4468,17 +4500,18 @@ "translate" "translateUTF8" "TRIGGER" -"trim" "TRIM" +"trim" "trimBoth" "trimLeft" "trimRight" "TRUE" "trunc" -"truncate" "TRUNCATE" +"truncate" "tryBase58Decode" "tryBase64Decode" +"tryBase64URLDecode" "tryDecrypt" "tryIdnaEncode" "tryPunycodeDecode" @@ -4486,8 +4519,8 @@ "tumble" "tumbleEnd" "tumbleStart" -"tuple" "Tuple" +"tuple" "tupleConcat" "tupleDivide" "tupleDivideByNumber" @@ -4502,15 +4535,14 @@ "tupleModuloByNumber" "tupleMultiply" "tupleMultiplyByNumber" +"tupleNames" "tupleNegate" "tuplePlus" "tupleToNameValuePairs" "TYPE" "TYPEOF" "ucase" -"UInt128" "UInt16" -"UInt256" "UInt32" "UInt64" "UInt8" @@ -4643,48 +4675,32 @@ "USE" "user" "USING" -"UTC_timestamp" "UTCTimestamp" +"UTC_timestamp" "UUID" "UUIDNumToString" "UUIDStringToNum" -"validateNestedArraySizes" +"UUIDToNum" +"UUIDv7ToDateTime" "VALID UNTIL" +"validateNestedArraySizes" "VALUES" -"VARBINARY" -"VARCHAR" -"VARCHAR2" -"Variant" "variantElement" "variantType" "varPop" -"VAR_POP" "varPopArgMax" -"VAR_POPArgMax" "varPopArgMin" -"VAR_POPArgMin" "varPopArray" -"VAR_POPArray" "varPopDistinct" -"VAR_POPDistinct" "varPopForEach" -"VAR_POPForEach" "varPopIf" -"VAR_POPIf" "varPopMap" -"VAR_POPMap" "varPopMerge" -"VAR_POPMerge" "varPopNull" -"VAR_POPNull" "varPopOrDefault" -"VAR_POPOrDefault" "varPopOrNull" -"VAR_POPOrNull" "varPopResample" -"VAR_POPResample" "varPopSimpleState" -"VAR_POPSimpleState" "varPopStable" "varPopStableArgMax" "varPopStableArgMin" @@ -4701,35 +4717,20 @@ "varPopStableSimpleState" "varPopStableState" "varPopState" -"VAR_POPState" "varSamp" -"VAR_SAMP" "varSampArgMax" -"VAR_SAMPArgMax" "varSampArgMin" -"VAR_SAMPArgMin" "varSampArray" -"VAR_SAMPArray" "varSampDistinct" -"VAR_SAMPDistinct" "varSampForEach" -"VAR_SAMPForEach" "varSampIf" -"VAR_SAMPIf" "varSampMap" -"VAR_SAMPMap" "varSampMerge" -"VAR_SAMPMerge" "varSampNull" -"VAR_SAMPNull" "varSampOrDefault" -"VAR_SAMPOrDefault" "varSampOrNull" -"VAR_SAMPOrNull" "varSampResample" -"VAR_SAMPResample" "varSampSimpleState" -"VAR_SAMPSimpleState" "varSampStable" "varSampStableArgMax" "varSampStableArgMin" @@ -4746,8 +4747,37 @@ "varSampStableSimpleState" "varSampStableState" "varSampState" -"VAR_SAMPState" "VARYING" +"VAR_POP" +"VAR_POPArgMax" +"VAR_POPArgMin" +"VAR_POPArray" +"VAR_POPDistinct" +"VAR_POPForEach" +"VAR_POPIf" +"VAR_POPMap" +"VAR_POPMerge" +"VAR_POPNull" +"VAR_POPOrDefault" +"VAR_POPOrNull" +"VAR_POPResample" +"VAR_POPSimpleState" +"VAR_POPState" +"VAR_SAMP" +"VAR_SAMPArgMax" +"VAR_SAMPArgMin" +"VAR_SAMPArray" +"VAR_SAMPDistinct" +"VAR_SAMPForEach" +"VAR_SAMPIf" +"VAR_SAMPMap" +"VAR_SAMPMerge" +"VAR_SAMPNull" +"VAR_SAMPOrDefault" +"VAR_SAMPOrNull" +"VAR_SAMPResample" +"VAR_SAMPSimpleState" +"VAR_SAMPState" "vectorDifference" "vectorSum" "version" @@ -4763,8 +4793,8 @@ "visitParamHas" "WATCH" "WATERMARK" -"week" "WEEK" +"week" "WEEKS" "welchTTest" "welchTTestArgMax" @@ -4783,8 +4813,8 @@ "welchTTestState" "WHEN" "WHERE" -"width_bucket" "widthBucket" +"width_bucket" "WINDOW" "windowFunnel" "windowFunnelArgMax" @@ -4802,15 +4832,15 @@ "windowFunnelSimpleState" "windowFunnelState" "windowID" -"WITH" "WITH ADMIN OPTION" "WITH CHECK" "WITH FILL" "WITH GRANT OPTION" -"with_itemindex" "WITH NAME" "WITH REPLACE OPTION" "WITH TIES" +"WITH" +"WITH_ITEMINDEX" "WK" "wkt" "wordShingleMinHash" @@ -4845,3 +4875,11 @@ "YYYYMMDDToDate32" "ZKPATH" "zookeeperSessionUptime" +"_CAST" +"__actionName" +"__bitBoolMaskAnd" +"__bitBoolMaskOr" +"__bitSwapLastTwo" +"__bitWrapperFunc" +"__getScalar" +"__scalarSubqueryResult" diff --git a/tests/fuzz/dictionaries/datatypes.dict b/tests/fuzz/dictionaries/datatypes.dict index a01a94fd3e3..e562595fb67 100644 --- a/tests/fuzz/dictionaries/datatypes.dict +++ b/tests/fuzz/dictionaries/datatypes.dict @@ -1,135 +1,4283 @@ -"AggregateFunction" -"Array" -"BIGINT" -"BIGINT SIGNED" -"BIGINT UNSIGNED" -"BINARY" -"BINARY LARGE OBJECT" -"BINARY VARYING" -"BIT" -"BLOB" -"BYTE" -"BYTEA" -"Bool" -"CHAR" -"CHAR LARGE OBJECT" -"CHAR VARYING" -"CHARACTER" -"CHARACTER LARGE OBJECT" -"CHARACTER VARYING" -"CLOB" -"DEC" -"DOUBLE" -"DOUBLE PRECISION" -"Date" -"Date32" -"DateTime" -"DateTime32" -"DateTime64" -"Decimal" -"Decimal128" -"Decimal256" -"Decimal32" -"Decimal64" -"ENUM" -"Enum" -"Enum16" -"Enum8" -"FIXED" -"FLOAT" -"FixedString" -"Float32" -"Float64" -"GEOMETRY" -"INET4" -"INET6" -"INT" -"INT SIGNED" -"INT UNSIGNED" -"INT1" -"INT1 SIGNED" -"INT1 UNSIGNED" -"INTEGER" -"INTEGER SIGNED" -"INTEGER UNSIGNED" -"IPv4" -"IPv6" -"Int128" -"Int16" -"Int256" -"Int32" -"Int64" -"Int8" -"IntervalDay" -"IntervalHour" -"IntervalMicrosecond" -"IntervalMillisecond" -"IntervalMinute" -"IntervalMonth" -"IntervalNanosecond" -"IntervalQuarter" -"IntervalSecond" -"IntervalWeek" -"IntervalYear" -"JSON" -"LONGBLOB" -"LONGTEXT" -"LowCardinality" -"MEDIUMBLOB" -"MEDIUMINT" -"MEDIUMINT SIGNED" -"MEDIUMINT UNSIGNED" -"MEDIUMTEXT" -"Map" -"MultiPolygon" -"NATIONAL CHAR" -"NATIONAL CHAR VARYING" -"NATIONAL CHARACTER" -"NATIONAL CHARACTER LARGE OBJECT" -"NATIONAL CHARACTER VARYING" -"NCHAR" -"NCHAR LARGE OBJECT" -"NCHAR VARYING" -"NUMERIC" -"NVARCHAR" -"Nested" -"Nothing" -"Nullable" -"Object" -"Point" -"Polygon" -"REAL" -"Ring" -"SET" -"SIGNED" -"SINGLE" -"SMALLINT" -"SMALLINT SIGNED" -"SMALLINT UNSIGNED" -"SimpleAggregateFunction" -"String" -"TEXT" -"TIME" -"TIMESTAMP" -"TINYBLOB" -"TINYINT" -"TINYINT SIGNED" -"TINYINT UNSIGNED" -"TINYTEXT" -"Tuple" -"UInt128" -"UInt16" -"UInt256" -"UInt32" -"UInt64" -"UInt8" -"UNSIGNED" -"UUID" -"VARBINARY" -"VARCHAR" -"VARCHAR2" -"Variant" +"BIT_AND" +"BIT_ANDArgMax" +"BIT_ANDArgMin" +"BIT_ANDArray" +"BIT_ANDDistinct" +"BIT_ANDForEach" +"BIT_ANDIf" +"BIT_ANDMap" +"BIT_ANDMerge" +"BIT_ANDNull" +"BIT_ANDOrDefault" +"BIT_ANDOrNull" +"BIT_ANDResample" +"BIT_ANDSimpleState" +"BIT_ANDState" +"BIT_OR" +"BIT_ORArgMax" +"BIT_ORArgMin" +"BIT_ORArray" +"BIT_ORDistinct" +"BIT_ORForEach" +"BIT_ORIf" +"BIT_ORMap" +"BIT_ORMerge" +"BIT_ORNull" +"BIT_OROrDefault" +"BIT_OROrNull" +"BIT_ORResample" +"BIT_ORSimpleState" +"BIT_ORState" +"BIT_XOR" +"BIT_XORArgMax" +"BIT_XORArgMin" +"BIT_XORArray" +"BIT_XORDistinct" +"BIT_XORForEach" +"BIT_XORIf" +"BIT_XORMap" +"BIT_XORMerge" +"BIT_XORNull" +"BIT_XOROrDefault" +"BIT_XOROrNull" +"BIT_XORResample" +"BIT_XORSimpleState" +"BIT_XORState" +"BLAKE3" +"CAST" +"CHARACTER_LENGTH" +"CHAR_LENGTH" +"COVAR_POP" +"COVAR_POPArgMax" +"COVAR_POPArgMin" +"COVAR_POPArray" +"COVAR_POPDistinct" +"COVAR_POPForEach" +"COVAR_POPIf" +"COVAR_POPMap" +"COVAR_POPMerge" +"COVAR_POPNull" +"COVAR_POPOrDefault" +"COVAR_POPOrNull" +"COVAR_POPResample" +"COVAR_POPSimpleState" +"COVAR_POPState" +"COVAR_SAMP" +"COVAR_SAMPArgMax" +"COVAR_SAMPArgMin" +"COVAR_SAMPArray" +"COVAR_SAMPDistinct" +"COVAR_SAMPForEach" +"COVAR_SAMPIf" +"COVAR_SAMPMap" +"COVAR_SAMPMerge" +"COVAR_SAMPNull" +"COVAR_SAMPOrDefault" +"COVAR_SAMPOrNull" +"COVAR_SAMPResample" +"COVAR_SAMPSimpleState" +"COVAR_SAMPState" +"CRC32" +"CRC32IEEE" +"CRC64" +"DATABASE" +"DATE" +"DATE_DIFF" +"DATE_FORMAT" +"DATE_TRUNC" +"DAY" +"DAYOFMONTH" +"DAYOFWEEK" +"DAYOFYEAR" +"FORMAT_BYTES" +"FQDN" +"FROM_BASE64" +"FROM_DAYS" +"FROM_UNIXTIME" +"HOUR" +"INET6_ATON" +"INET6_NTOA" +"INET_ATON" +"INET_NTOA" +"IPv4CIDRToRange" +"IPv4NumToString" +"IPv4NumToStringClassC" +"IPv4StringToNum" +"IPv4StringToNumOrDefault" +"IPv4StringToNumOrNull" +"IPv4ToIPv6" +"IPv6CIDRToRange" +"IPv6NumToString" +"IPv6StringToNum" +"IPv6StringToNumOrDefault" +"IPv6StringToNumOrNull" +"JSONArrayLength" +"JSONExtract" +"JSONExtractArrayRaw" +"JSONExtractBool" +"JSONExtractFloat" +"JSONExtractInt" +"JSONExtractKeys" +"JSONExtractKeysAndValues" +"JSONExtractKeysAndValuesRaw" +"JSONExtractRaw" +"JSONExtractString" +"JSONExtractUInt" +"JSONHas" +"JSONKey" +"JSONLength" +"JSONMergePatch" +"JSONType" +"JSON_ARRAY_LENGTH" +"JSON_EXISTS" +"JSON_QUERY" +"JSON_VALUE" +"L1Distance" +"L1Norm" +"L1Normalize" +"L2Distance" +"L2Norm" +"L2Normalize" +"L2SquaredDistance" +"L2SquaredNorm" +"LAST_DAY" +"LinfDistance" +"LinfNorm" +"LinfNormalize" +"LpDistance" +"LpNorm" +"LpNormalize" +"MACNumToString" +"MACStringToNum" +"MACStringToOUI" +"MAP_FROM_ARRAYS" +"MD4" +"MD5" +"MILLISECOND" +"MINUTE" +"MONTH" +"OCTET_LENGTH" +"QUARTER" +"REGEXP_EXTRACT" +"REGEXP_MATCHES" +"REGEXP_REPLACE" +"SCHEMA" +"SECOND" +"SHA1" +"SHA224" +"SHA256" +"SHA384" +"SHA512" +"SHA512_256" +"STD" +"STDArgMax" +"STDArgMin" +"STDArray" +"STDDEV_POP" +"STDDEV_POPArgMax" +"STDDEV_POPArgMin" +"STDDEV_POPArray" +"STDDEV_POPDistinct" +"STDDEV_POPForEach" +"STDDEV_POPIf" +"STDDEV_POPMap" +"STDDEV_POPMerge" +"STDDEV_POPNull" +"STDDEV_POPOrDefault" +"STDDEV_POPOrNull" +"STDDEV_POPResample" +"STDDEV_POPSimpleState" +"STDDEV_POPState" +"STDDEV_SAMP" +"STDDEV_SAMPArgMax" +"STDDEV_SAMPArgMin" +"STDDEV_SAMPArray" +"STDDEV_SAMPDistinct" +"STDDEV_SAMPForEach" +"STDDEV_SAMPIf" +"STDDEV_SAMPMap" +"STDDEV_SAMPMerge" +"STDDEV_SAMPNull" +"STDDEV_SAMPOrDefault" +"STDDEV_SAMPOrNull" +"STDDEV_SAMPResample" +"STDDEV_SAMPSimpleState" +"STDDEV_SAMPState" +"STDDistinct" +"STDForEach" +"STDIf" +"STDMap" +"STDMerge" +"STDNull" +"STDOrDefault" +"STDOrNull" +"STDResample" +"STDSimpleState" +"STDState" +"SUBSTRING_INDEX" +"SVG" +"TIMESTAMP_DIFF" +"TO_BASE64" +"TO_DAYS" +"TO_UNIXTIME" +"ULIDStringToDateTime" +"URLHash" +"URLHierarchy" +"URLPathHierarchy" +"UTCTimestamp" +"UTC_timestamp" +"UUIDNumToString" +"UUIDStringToNum" +"UUIDToNum" +"UUIDv7ToDateTime" +"VAR_POP" +"VAR_POPArgMax" +"VAR_POPArgMin" +"VAR_POPArray" +"VAR_POPDistinct" +"VAR_POPForEach" +"VAR_POPIf" +"VAR_POPMap" +"VAR_POPMerge" +"VAR_POPNull" +"VAR_POPOrDefault" +"VAR_POPOrNull" +"VAR_POPResample" +"VAR_POPSimpleState" +"VAR_POPState" +"VAR_SAMP" +"VAR_SAMPArgMax" +"VAR_SAMPArgMin" +"VAR_SAMPArray" +"VAR_SAMPDistinct" +"VAR_SAMPForEach" +"VAR_SAMPIf" +"VAR_SAMPMap" +"VAR_SAMPMerge" +"VAR_SAMPNull" +"VAR_SAMPOrDefault" +"VAR_SAMPOrNull" +"VAR_SAMPResample" +"VAR_SAMPSimpleState" +"VAR_SAMPState" "YEAR" -"bool" -"boolean" -"Dynamic" +"YYYYMMDDToDate" +"YYYYMMDDToDate32" +"YYYYMMDDhhmmssToDateTime" +"YYYYMMDDhhmmssToDateTime64" +"_CAST" +"__actionName" +"__bitBoolMaskAnd" +"__bitBoolMaskOr" +"__bitSwapLastTwo" +"__bitWrapperFunc" +"__getScalar" +"__scalarSubqueryResult" +"abs" +"accurateCast" +"accurateCastOrDefault" +"accurateCastOrNull" +"acos" +"acosh" +"addDate" +"addDays" +"addHours" +"addInterval" +"addMicroseconds" +"addMilliseconds" +"addMinutes" +"addMonths" +"addNanoseconds" +"addQuarters" +"addSeconds" +"addTupleOfIntervals" +"addWeeks" +"addYears" +"addressToLine" +"addressToLineWithInlines" +"addressToSymbol" +"aes_decrypt_mysql" +"aes_encrypt_mysql" +"age" +"aggThrow" +"aggThrowArgMax" +"aggThrowArgMin" +"aggThrowArray" +"aggThrowDistinct" +"aggThrowForEach" +"aggThrowIf" +"aggThrowMap" +"aggThrowMerge" +"aggThrowNull" +"aggThrowOrDefault" +"aggThrowOrNull" +"aggThrowResample" +"aggThrowSimpleState" +"aggThrowState" +"alphaTokens" +"analysisOfVariance" +"analysisOfVarianceArgMax" +"analysisOfVarianceArgMin" +"analysisOfVarianceArray" +"analysisOfVarianceDistinct" +"analysisOfVarianceForEach" +"analysisOfVarianceIf" +"analysisOfVarianceMap" +"analysisOfVarianceMerge" +"analysisOfVarianceNull" +"analysisOfVarianceOrDefault" +"analysisOfVarianceOrNull" +"analysisOfVarianceResample" +"analysisOfVarianceSimpleState" +"analysisOfVarianceState" +"and" +"anova" +"anovaArgMax" +"anovaArgMin" +"anovaArray" +"anovaDistinct" +"anovaForEach" +"anovaIf" +"anovaMap" +"anovaMerge" +"anovaNull" +"anovaOrDefault" +"anovaOrNull" +"anovaResample" +"anovaSimpleState" +"anovaState" +"any" +"anyArgMax" +"anyArgMin" +"anyArray" +"anyDistinct" +"anyForEach" +"anyHeavy" +"anyHeavyArgMax" +"anyHeavyArgMin" +"anyHeavyArray" +"anyHeavyDistinct" +"anyHeavyForEach" +"anyHeavyIf" +"anyHeavyMap" +"anyHeavyMerge" +"anyHeavyNull" +"anyHeavyOrDefault" +"anyHeavyOrNull" +"anyHeavyResample" +"anyHeavySimpleState" +"anyHeavyState" +"anyIf" +"anyLast" +"anyLastArgMax" +"anyLastArgMin" +"anyLastArray" +"anyLastDistinct" +"anyLastForEach" +"anyLastIf" +"anyLastMap" +"anyLastMerge" +"anyLastNull" +"anyLastOrDefault" +"anyLastOrNull" +"anyLastResample" +"anyLastSimpleState" +"anyLastState" +"anyLast_respect_nulls" +"anyLast_respect_nullsArgMax" +"anyLast_respect_nullsArgMin" +"anyLast_respect_nullsArray" +"anyLast_respect_nullsDistinct" +"anyLast_respect_nullsForEach" +"anyLast_respect_nullsIf" +"anyLast_respect_nullsMap" +"anyLast_respect_nullsMerge" +"anyLast_respect_nullsNull" +"anyLast_respect_nullsOrDefault" +"anyLast_respect_nullsOrNull" +"anyLast_respect_nullsResample" +"anyLast_respect_nullsSimpleState" +"anyLast_respect_nullsState" +"anyMap" +"anyMerge" +"anyNull" +"anyOrDefault" +"anyOrNull" +"anyResample" +"anySimpleState" +"anyState" +"any_respect_nulls" +"any_respect_nullsArgMax" +"any_respect_nullsArgMin" +"any_respect_nullsArray" +"any_respect_nullsDistinct" +"any_respect_nullsForEach" +"any_respect_nullsIf" +"any_respect_nullsMap" +"any_respect_nullsMerge" +"any_respect_nullsNull" +"any_respect_nullsOrDefault" +"any_respect_nullsOrNull" +"any_respect_nullsResample" +"any_respect_nullsSimpleState" +"any_respect_nullsState" +"any_value" +"any_valueArgMax" +"any_valueArgMin" +"any_valueArray" +"any_valueDistinct" +"any_valueForEach" +"any_valueIf" +"any_valueMap" +"any_valueMerge" +"any_valueNull" +"any_valueOrDefault" +"any_valueOrNull" +"any_valueResample" +"any_valueSimpleState" +"any_valueState" +"any_value_respect_nulls" +"any_value_respect_nullsArgMax" +"any_value_respect_nullsArgMin" +"any_value_respect_nullsArray" +"any_value_respect_nullsDistinct" +"any_value_respect_nullsForEach" +"any_value_respect_nullsIf" +"any_value_respect_nullsMap" +"any_value_respect_nullsMerge" +"any_value_respect_nullsNull" +"any_value_respect_nullsOrDefault" +"any_value_respect_nullsOrNull" +"any_value_respect_nullsResample" +"any_value_respect_nullsSimpleState" +"any_value_respect_nullsState" +"appendTrailingCharIfAbsent" +"approx_top_count" +"approx_top_countArgMax" +"approx_top_countArgMin" +"approx_top_countArray" +"approx_top_countDistinct" +"approx_top_countForEach" +"approx_top_countIf" +"approx_top_countMap" +"approx_top_countMerge" +"approx_top_countNull" +"approx_top_countOrDefault" +"approx_top_countOrNull" +"approx_top_countResample" +"approx_top_countSimpleState" +"approx_top_countState" +"approx_top_k" +"approx_top_kArgMax" +"approx_top_kArgMin" +"approx_top_kArray" +"approx_top_kDistinct" +"approx_top_kForEach" +"approx_top_kIf" +"approx_top_kMap" +"approx_top_kMerge" +"approx_top_kNull" +"approx_top_kOrDefault" +"approx_top_kOrNull" +"approx_top_kResample" +"approx_top_kSimpleState" +"approx_top_kState" +"approx_top_sum" +"approx_top_sumArgMax" +"approx_top_sumArgMin" +"approx_top_sumArray" +"approx_top_sumDistinct" +"approx_top_sumForEach" +"approx_top_sumIf" +"approx_top_sumMap" +"approx_top_sumMerge" +"approx_top_sumNull" +"approx_top_sumOrDefault" +"approx_top_sumOrNull" +"approx_top_sumResample" +"approx_top_sumSimpleState" +"approx_top_sumState" +"argMax" +"argMaxArgMax" +"argMaxArgMin" +"argMaxArray" +"argMaxDistinct" +"argMaxForEach" +"argMaxIf" +"argMaxMap" +"argMaxMerge" +"argMaxNull" +"argMaxOrDefault" +"argMaxOrNull" +"argMaxResample" +"argMaxSimpleState" +"argMaxState" +"argMin" +"argMinArgMax" +"argMinArgMin" +"argMinArray" +"argMinDistinct" +"argMinForEach" +"argMinIf" +"argMinMap" +"argMinMerge" +"argMinNull" +"argMinOrDefault" +"argMinOrNull" +"argMinResample" +"argMinSimpleState" +"argMinState" +"array" +"arrayAUC" +"arrayAll" +"arrayAvg" +"arrayCompact" +"arrayConcat" +"arrayCount" +"arrayCumSum" +"arrayCumSumNonNegative" +"arrayDifference" +"arrayDistinct" +"arrayDotProduct" +"arrayElement" +"arrayEnumerate" +"arrayEnumerateDense" +"arrayEnumerateDenseRanked" +"arrayEnumerateUniq" +"arrayEnumerateUniqRanked" +"arrayExists" +"arrayFill" +"arrayFilter" +"arrayFirst" +"arrayFirstIndex" +"arrayFirstOrNull" +"arrayFlatten" +"arrayFold" +"arrayIntersect" +"arrayJaccardIndex" +"arrayJoin" +"arrayLast" +"arrayLastIndex" +"arrayLastOrNull" +"arrayMap" +"arrayMax" +"arrayMin" +"arrayPartialReverseSort" +"arrayPartialShuffle" +"arrayPartialSort" +"arrayPopBack" +"arrayPopFront" +"arrayProduct" +"arrayPushBack" +"arrayPushFront" +"arrayRandomSample" +"arrayReduce" +"arrayReduceInRanges" +"arrayResize" +"arrayReverse" +"arrayReverseFill" +"arrayReverseSort" +"arrayReverseSplit" +"arrayRotateLeft" +"arrayRotateRight" +"arrayShiftLeft" +"arrayShiftRight" +"arrayShingles" +"arrayShuffle" +"arraySlice" +"arraySort" +"arraySplit" +"arrayStringConcat" +"arraySum" +"arrayUniq" +"arrayWithConstant" +"arrayZip" +"array_agg" +"array_aggArgMax" +"array_aggArgMin" +"array_aggArray" +"array_aggDistinct" +"array_aggForEach" +"array_aggIf" +"array_aggMap" +"array_aggMerge" +"array_aggNull" +"array_aggOrDefault" +"array_aggOrNull" +"array_aggResample" +"array_aggSimpleState" +"array_aggState" +"array_concat_agg" +"array_concat_aggArgMax" +"array_concat_aggArgMin" +"array_concat_aggArray" +"array_concat_aggDistinct" +"array_concat_aggForEach" +"array_concat_aggIf" +"array_concat_aggMap" +"array_concat_aggMerge" +"array_concat_aggNull" +"array_concat_aggOrDefault" +"array_concat_aggOrNull" +"array_concat_aggResample" +"array_concat_aggSimpleState" +"array_concat_aggState" +"ascii" +"asin" +"asinh" +"assumeNotNull" +"atan" +"atan2" +"atanh" +"avg" +"avgArgMax" +"avgArgMin" +"avgArray" +"avgDistinct" +"avgForEach" +"avgIf" +"avgMap" +"avgMerge" +"avgNull" +"avgOrDefault" +"avgOrNull" +"avgResample" +"avgSimpleState" +"avgState" +"avgWeighted" +"avgWeightedArgMax" +"avgWeightedArgMin" +"avgWeightedArray" +"avgWeightedDistinct" +"avgWeightedForEach" +"avgWeightedIf" +"avgWeightedMap" +"avgWeightedMerge" +"avgWeightedNull" +"avgWeightedOrDefault" +"avgWeightedOrNull" +"avgWeightedResample" +"avgWeightedSimpleState" +"avgWeightedState" +"bar" +"base58Decode" +"base58Encode" +"base64Decode" +"base64Encode" +"base64URLDecode" +"base64URLEncode" +"basename" +"bin" +"bitAnd" +"bitCount" +"bitHammingDistance" +"bitNot" +"bitOr" +"bitPositionsToArray" +"bitRotateLeft" +"bitRotateRight" +"bitShiftLeft" +"bitShiftRight" +"bitSlice" +"bitTest" +"bitTestAll" +"bitTestAny" +"bitXor" +"bitmapAnd" +"bitmapAndCardinality" +"bitmapAndnot" +"bitmapAndnotCardinality" +"bitmapBuild" +"bitmapCardinality" +"bitmapContains" +"bitmapHasAll" +"bitmapHasAny" +"bitmapMax" +"bitmapMin" +"bitmapOr" +"bitmapOrCardinality" +"bitmapSubsetInRange" +"bitmapSubsetLimit" +"bitmapToArray" +"bitmapTransform" +"bitmapXor" +"bitmapXorCardinality" +"bitmaskToArray" +"bitmaskToList" +"blockNumber" +"blockSerializedSize" +"blockSize" +"boundingRatio" +"boundingRatioArgMax" +"boundingRatioArgMin" +"boundingRatioArray" +"boundingRatioDistinct" +"boundingRatioForEach" +"boundingRatioIf" +"boundingRatioMap" +"boundingRatioMerge" +"boundingRatioNull" +"boundingRatioOrDefault" +"boundingRatioOrNull" +"boundingRatioResample" +"boundingRatioSimpleState" +"boundingRatioState" +"buildId" +"byteHammingDistance" +"byteSize" +"byteSlice" +"byteSwap" +"caseWithExpr" +"caseWithExpression" +"caseWithoutExpr" +"caseWithoutExpression" +"catboostEvaluate" +"categoricalInformationValue" +"categoricalInformationValueArgMax" +"categoricalInformationValueArgMin" +"categoricalInformationValueArray" +"categoricalInformationValueDistinct" +"categoricalInformationValueForEach" +"categoricalInformationValueIf" +"categoricalInformationValueMap" +"categoricalInformationValueMerge" +"categoricalInformationValueNull" +"categoricalInformationValueOrDefault" +"categoricalInformationValueOrNull" +"categoricalInformationValueResample" +"categoricalInformationValueSimpleState" +"categoricalInformationValueState" +"cbrt" +"ceil" +"ceiling" +"changeDay" +"changeHour" +"changeMinute" +"changeMonth" +"changeSecond" +"changeYear" +"char" +"cityHash64" +"clamp" +"coalesce" +"concat" +"concatAssumeInjective" +"concatWithSeparator" +"concatWithSeparatorAssumeInjective" +"concat_ws" +"connectionId" +"connection_id" +"contingency" +"contingencyArgMax" +"contingencyArgMin" +"contingencyArray" +"contingencyDistinct" +"contingencyForEach" +"contingencyIf" +"contingencyMap" +"contingencyMerge" +"contingencyNull" +"contingencyOrDefault" +"contingencyOrNull" +"contingencyResample" +"contingencySimpleState" +"contingencyState" +"convertCharset" +"corr" +"corrArgMax" +"corrArgMin" +"corrArray" +"corrDistinct" +"corrForEach" +"corrIf" +"corrMap" +"corrMatrix" +"corrMatrixArgMax" +"corrMatrixArgMin" +"corrMatrixArray" +"corrMatrixDistinct" +"corrMatrixForEach" +"corrMatrixIf" +"corrMatrixMap" +"corrMatrixMerge" +"corrMatrixNull" +"corrMatrixOrDefault" +"corrMatrixOrNull" +"corrMatrixResample" +"corrMatrixSimpleState" +"corrMatrixState" +"corrMerge" +"corrNull" +"corrOrDefault" +"corrOrNull" +"corrResample" +"corrSimpleState" +"corrStable" +"corrStableArgMax" +"corrStableArgMin" +"corrStableArray" +"corrStableDistinct" +"corrStableForEach" +"corrStableIf" +"corrStableMap" +"corrStableMerge" +"corrStableNull" +"corrStableOrDefault" +"corrStableOrNull" +"corrStableResample" +"corrStableSimpleState" +"corrStableState" +"corrState" +"cos" +"cosh" +"cosineDistance" +"count" +"countArgMax" +"countArgMin" +"countArray" +"countDigits" +"countDistinct" +"countEqual" +"countForEach" +"countIf" +"countMap" +"countMatches" +"countMatchesCaseInsensitive" +"countMerge" +"countNull" +"countOrDefault" +"countOrNull" +"countResample" +"countSimpleState" +"countState" +"countSubstrings" +"countSubstringsCaseInsensitive" +"countSubstringsCaseInsensitiveUTF8" +"covarPop" +"covarPopArgMax" +"covarPopArgMin" +"covarPopArray" +"covarPopDistinct" +"covarPopForEach" +"covarPopIf" +"covarPopMap" +"covarPopMatrix" +"covarPopMatrixArgMax" +"covarPopMatrixArgMin" +"covarPopMatrixArray" +"covarPopMatrixDistinct" +"covarPopMatrixForEach" +"covarPopMatrixIf" +"covarPopMatrixMap" +"covarPopMatrixMerge" +"covarPopMatrixNull" +"covarPopMatrixOrDefault" +"covarPopMatrixOrNull" +"covarPopMatrixResample" +"covarPopMatrixSimpleState" +"covarPopMatrixState" +"covarPopMerge" +"covarPopNull" +"covarPopOrDefault" +"covarPopOrNull" +"covarPopResample" +"covarPopSimpleState" +"covarPopStable" +"covarPopStableArgMax" +"covarPopStableArgMin" +"covarPopStableArray" +"covarPopStableDistinct" +"covarPopStableForEach" +"covarPopStableIf" +"covarPopStableMap" +"covarPopStableMerge" +"covarPopStableNull" +"covarPopStableOrDefault" +"covarPopStableOrNull" +"covarPopStableResample" +"covarPopStableSimpleState" +"covarPopStableState" +"covarPopState" +"covarSamp" +"covarSampArgMax" +"covarSampArgMin" +"covarSampArray" +"covarSampDistinct" +"covarSampForEach" +"covarSampIf" +"covarSampMap" +"covarSampMatrix" +"covarSampMatrixArgMax" +"covarSampMatrixArgMin" +"covarSampMatrixArray" +"covarSampMatrixDistinct" +"covarSampMatrixForEach" +"covarSampMatrixIf" +"covarSampMatrixMap" +"covarSampMatrixMerge" +"covarSampMatrixNull" +"covarSampMatrixOrDefault" +"covarSampMatrixOrNull" +"covarSampMatrixResample" +"covarSampMatrixSimpleState" +"covarSampMatrixState" +"covarSampMerge" +"covarSampNull" +"covarSampOrDefault" +"covarSampOrNull" +"covarSampResample" +"covarSampSimpleState" +"covarSampStable" +"covarSampStableArgMax" +"covarSampStableArgMin" +"covarSampStableArray" +"covarSampStableDistinct" +"covarSampStableForEach" +"covarSampStableIf" +"covarSampStableMap" +"covarSampStableMerge" +"covarSampStableNull" +"covarSampStableOrDefault" +"covarSampStableOrNull" +"covarSampStableResample" +"covarSampStableSimpleState" +"covarSampStableState" +"covarSampState" +"cramersV" +"cramersVArgMax" +"cramersVArgMin" +"cramersVArray" +"cramersVBiasCorrected" +"cramersVBiasCorrectedArgMax" +"cramersVBiasCorrectedArgMin" +"cramersVBiasCorrectedArray" +"cramersVBiasCorrectedDistinct" +"cramersVBiasCorrectedForEach" +"cramersVBiasCorrectedIf" +"cramersVBiasCorrectedMap" +"cramersVBiasCorrectedMerge" +"cramersVBiasCorrectedNull" +"cramersVBiasCorrectedOrDefault" +"cramersVBiasCorrectedOrNull" +"cramersVBiasCorrectedResample" +"cramersVBiasCorrectedSimpleState" +"cramersVBiasCorrectedState" +"cramersVDistinct" +"cramersVForEach" +"cramersVIf" +"cramersVMap" +"cramersVMerge" +"cramersVNull" +"cramersVOrDefault" +"cramersVOrNull" +"cramersVResample" +"cramersVSimpleState" +"cramersVState" +"curdate" +"currentDatabase" +"currentProfiles" +"currentRoles" +"currentSchemas" +"currentUser" +"current_database" +"current_date" +"current_schemas" +"current_timestamp" +"current_user" +"cutFragment" +"cutIPv6" +"cutQueryString" +"cutQueryStringAndFragment" +"cutToFirstSignificantSubdomain" +"cutToFirstSignificantSubdomainCustom" +"cutToFirstSignificantSubdomainCustomRFC" +"cutToFirstSignificantSubdomainCustomWithWWW" +"cutToFirstSignificantSubdomainCustomWithWWWRFC" +"cutToFirstSignificantSubdomainRFC" +"cutToFirstSignificantSubdomainWithWWW" +"cutToFirstSignificantSubdomainWithWWWRFC" +"cutURLParameter" +"cutWWW" +"damerauLevenshteinDistance" +"dateDiff" +"dateName" +"dateTime64ToSnowflake" +"dateTime64ToSnowflakeID" +"dateTimeToSnowflake" +"dateTimeToSnowflakeID" +"dateTrunc" +"date_diff" +"decodeHTMLComponent" +"decodeURLComponent" +"decodeURLFormComponent" +"decodeXMLComponent" +"decrypt" +"defaultProfiles" +"defaultRoles" +"defaultValueOfArgumentType" +"defaultValueOfTypeName" +"degrees" +"deltaSum" +"deltaSumArgMax" +"deltaSumArgMin" +"deltaSumArray" +"deltaSumDistinct" +"deltaSumForEach" +"deltaSumIf" +"deltaSumMap" +"deltaSumMerge" +"deltaSumNull" +"deltaSumOrDefault" +"deltaSumOrNull" +"deltaSumResample" +"deltaSumSimpleState" +"deltaSumState" +"deltaSumTimestamp" +"deltaSumTimestampArgMax" +"deltaSumTimestampArgMin" +"deltaSumTimestampArray" +"deltaSumTimestampDistinct" +"deltaSumTimestampForEach" +"deltaSumTimestampIf" +"deltaSumTimestampMap" +"deltaSumTimestampMerge" +"deltaSumTimestampNull" +"deltaSumTimestampOrDefault" +"deltaSumTimestampOrNull" +"deltaSumTimestampResample" +"deltaSumTimestampSimpleState" +"deltaSumTimestampState" +"demangle" +"denseRank" +"denseRankArgMax" +"denseRankArgMin" +"denseRankArray" +"denseRankDistinct" +"denseRankForEach" +"denseRankIf" +"denseRankMap" +"denseRankMerge" +"denseRankNull" +"denseRankOrDefault" +"denseRankOrNull" +"denseRankResample" +"denseRankSimpleState" +"denseRankState" +"dense_rank" +"dense_rankArgMax" +"dense_rankArgMin" +"dense_rankArray" +"dense_rankDistinct" +"dense_rankForEach" +"dense_rankIf" +"dense_rankMap" +"dense_rankMerge" +"dense_rankNull" +"dense_rankOrDefault" +"dense_rankOrNull" +"dense_rankResample" +"dense_rankSimpleState" +"dense_rankState" +"detectCharset" +"detectLanguage" +"detectLanguageMixed" +"detectLanguageUnknown" +"detectProgrammingLanguage" +"detectTonality" +"dictGet" +"dictGetAll" +"dictGetChildren" +"dictGetDate" +"dictGetDateOrDefault" +"dictGetDateTime" +"dictGetDateTimeOrDefault" +"dictGetDescendants" +"dictGetFloat32" +"dictGetFloat32OrDefault" +"dictGetFloat64" +"dictGetFloat64OrDefault" +"dictGetHierarchy" +"dictGetIPv4" +"dictGetIPv4OrDefault" +"dictGetIPv6" +"dictGetIPv6OrDefault" +"dictGetInt16" +"dictGetInt16OrDefault" +"dictGetInt32" +"dictGetInt32OrDefault" +"dictGetInt64" +"dictGetInt64OrDefault" +"dictGetInt8" +"dictGetInt8OrDefault" +"dictGetOrDefault" +"dictGetOrNull" +"dictGetString" +"dictGetStringOrDefault" +"dictGetUInt16" +"dictGetUInt16OrDefault" +"dictGetUInt32" +"dictGetUInt32OrDefault" +"dictGetUInt64" +"dictGetUInt64OrDefault" +"dictGetUInt8" +"dictGetUInt8OrDefault" +"dictGetUUID" +"dictGetUUIDOrDefault" +"dictHas" +"dictIsIn" +"displayName" +"distanceL1" +"distanceL2" +"distanceL2Squared" +"distanceLinf" +"distanceLp" +"divide" +"divideDecimal" +"domain" +"domainRFC" +"domainWithoutWWW" +"domainWithoutWWWRFC" +"dotProduct" +"dumpColumnStructure" +"dynamicElement" +"dynamicType" +"e" +"editDistance" +"editDistanceUTF8" +"empty" +"emptyArrayDate" +"emptyArrayDateTime" +"emptyArrayFloat32" +"emptyArrayFloat64" +"emptyArrayInt16" +"emptyArrayInt32" +"emptyArrayInt64" +"emptyArrayInt8" +"emptyArrayString" +"emptyArrayToSingle" +"emptyArrayUInt16" +"emptyArrayUInt32" +"emptyArrayUInt64" +"emptyArrayUInt8" +"enabledProfiles" +"enabledRoles" +"encodeURLComponent" +"encodeURLFormComponent" +"encodeXMLComponent" +"encrypt" +"endsWith" +"endsWithUTF8" +"entropy" +"entropyArgMax" +"entropyArgMin" +"entropyArray" +"entropyDistinct" +"entropyForEach" +"entropyIf" +"entropyMap" +"entropyMerge" +"entropyNull" +"entropyOrDefault" +"entropyOrNull" +"entropyResample" +"entropySimpleState" +"entropyState" +"equals" +"erf" +"erfc" +"errorCodeToName" +"evalMLMethod" +"exp" +"exp10" +"exp2" +"exponentialMovingAverage" +"exponentialMovingAverageArgMax" +"exponentialMovingAverageArgMin" +"exponentialMovingAverageArray" +"exponentialMovingAverageDistinct" +"exponentialMovingAverageForEach" +"exponentialMovingAverageIf" +"exponentialMovingAverageMap" +"exponentialMovingAverageMerge" +"exponentialMovingAverageNull" +"exponentialMovingAverageOrDefault" +"exponentialMovingAverageOrNull" +"exponentialMovingAverageResample" +"exponentialMovingAverageSimpleState" +"exponentialMovingAverageState" +"exponentialTimeDecayedAvg" +"exponentialTimeDecayedAvgArgMax" +"exponentialTimeDecayedAvgArgMin" +"exponentialTimeDecayedAvgArray" +"exponentialTimeDecayedAvgDistinct" +"exponentialTimeDecayedAvgForEach" +"exponentialTimeDecayedAvgIf" +"exponentialTimeDecayedAvgMap" +"exponentialTimeDecayedAvgMerge" +"exponentialTimeDecayedAvgNull" +"exponentialTimeDecayedAvgOrDefault" +"exponentialTimeDecayedAvgOrNull" +"exponentialTimeDecayedAvgResample" +"exponentialTimeDecayedAvgSimpleState" +"exponentialTimeDecayedAvgState" +"exponentialTimeDecayedCount" +"exponentialTimeDecayedCountArgMax" +"exponentialTimeDecayedCountArgMin" +"exponentialTimeDecayedCountArray" +"exponentialTimeDecayedCountDistinct" +"exponentialTimeDecayedCountForEach" +"exponentialTimeDecayedCountIf" +"exponentialTimeDecayedCountMap" +"exponentialTimeDecayedCountMerge" +"exponentialTimeDecayedCountNull" +"exponentialTimeDecayedCountOrDefault" +"exponentialTimeDecayedCountOrNull" +"exponentialTimeDecayedCountResample" +"exponentialTimeDecayedCountSimpleState" +"exponentialTimeDecayedCountState" +"exponentialTimeDecayedMax" +"exponentialTimeDecayedMaxArgMax" +"exponentialTimeDecayedMaxArgMin" +"exponentialTimeDecayedMaxArray" +"exponentialTimeDecayedMaxDistinct" +"exponentialTimeDecayedMaxForEach" +"exponentialTimeDecayedMaxIf" +"exponentialTimeDecayedMaxMap" +"exponentialTimeDecayedMaxMerge" +"exponentialTimeDecayedMaxNull" +"exponentialTimeDecayedMaxOrDefault" +"exponentialTimeDecayedMaxOrNull" +"exponentialTimeDecayedMaxResample" +"exponentialTimeDecayedMaxSimpleState" +"exponentialTimeDecayedMaxState" +"exponentialTimeDecayedSum" +"exponentialTimeDecayedSumArgMax" +"exponentialTimeDecayedSumArgMin" +"exponentialTimeDecayedSumArray" +"exponentialTimeDecayedSumDistinct" +"exponentialTimeDecayedSumForEach" +"exponentialTimeDecayedSumIf" +"exponentialTimeDecayedSumMap" +"exponentialTimeDecayedSumMerge" +"exponentialTimeDecayedSumNull" +"exponentialTimeDecayedSumOrDefault" +"exponentialTimeDecayedSumOrNull" +"exponentialTimeDecayedSumResample" +"exponentialTimeDecayedSumSimpleState" +"exponentialTimeDecayedSumState" +"extract" +"extractAll" +"extractAllGroups" +"extractAllGroupsHorizontal" +"extractAllGroupsVertical" +"extractGroups" +"extractKeyValuePairs" +"extractKeyValuePairsWithEscaping" +"extractTextFromHTML" +"extractURLParameter" +"extractURLParameterNames" +"extractURLParameters" +"factorial" +"farmFingerprint64" +"farmHash64" +"file" +"filesystemAvailable" +"filesystemCapacity" +"filesystemUnreserved" +"finalizeAggregation" +"firstLine" +"firstSignificantSubdomain" +"firstSignificantSubdomainCustom" +"firstSignificantSubdomainCustomRFC" +"firstSignificantSubdomainRFC" +"first_value" +"first_valueArgMax" +"first_valueArgMin" +"first_valueArray" +"first_valueDistinct" +"first_valueForEach" +"first_valueIf" +"first_valueMap" +"first_valueMerge" +"first_valueNull" +"first_valueOrDefault" +"first_valueOrNull" +"first_valueResample" +"first_valueSimpleState" +"first_valueState" +"first_value_respect_nulls" +"first_value_respect_nullsArgMax" +"first_value_respect_nullsArgMin" +"first_value_respect_nullsArray" +"first_value_respect_nullsDistinct" +"first_value_respect_nullsForEach" +"first_value_respect_nullsIf" +"first_value_respect_nullsMap" +"first_value_respect_nullsMerge" +"first_value_respect_nullsNull" +"first_value_respect_nullsOrDefault" +"first_value_respect_nullsOrNull" +"first_value_respect_nullsResample" +"first_value_respect_nullsSimpleState" +"first_value_respect_nullsState" +"flameGraph" +"flameGraphArgMax" +"flameGraphArgMin" +"flameGraphArray" +"flameGraphDistinct" +"flameGraphForEach" +"flameGraphIf" +"flameGraphMap" +"flameGraphMerge" +"flameGraphNull" +"flameGraphOrDefault" +"flameGraphOrNull" +"flameGraphResample" +"flameGraphSimpleState" +"flameGraphState" +"flatten" +"flattenTuple" +"floor" +"format" +"formatDateTime" +"formatDateTimeInJodaSyntax" +"formatQuery" +"formatQueryOrNull" +"formatQuerySingleLine" +"formatQuerySingleLineOrNull" +"formatReadableDecimalSize" +"formatReadableQuantity" +"formatReadableSize" +"formatReadableTimeDelta" +"formatRow" +"formatRowNoNewline" +"fragment" +"fromDaysSinceYearZero" +"fromDaysSinceYearZero32" +"fromModifiedJulianDay" +"fromModifiedJulianDayOrNull" +"fromUTCTimestamp" +"fromUnixTimestamp" +"fromUnixTimestamp64Micro" +"fromUnixTimestamp64Milli" +"fromUnixTimestamp64Nano" +"fromUnixTimestampInJodaSyntax" +"from_utc_timestamp" +"fullHostName" +"fuzzBits" +"gccMurmurHash" +"gcd" +"generateRandomStructure" +"generateSnowflakeID" +"generateULID" +"generateUUIDv4" +"generateUUIDv7" +"geoDistance" +"geoToH3" +"geoToS2" +"geohashDecode" +"geohashEncode" +"geohashesInBox" +"getClientHTTPHeader" +"getMacro" +"getOSKernelVersion" +"getServerPort" +"getSetting" +"getSizeOfEnumType" +"getSubcolumn" +"getTypeSerializationStreams" +"globalIn" +"globalInIgnoreSet" +"globalNotIn" +"globalNotInIgnoreSet" +"globalNotNullIn" +"globalNotNullInIgnoreSet" +"globalNullIn" +"globalNullInIgnoreSet" +"globalVariable" +"greatCircleAngle" +"greatCircleDistance" +"greater" +"greaterOrEquals" +"greatest" +"groupArray" +"groupArrayArgMax" +"groupArrayArgMin" +"groupArrayArray" +"groupArrayDistinct" +"groupArrayForEach" +"groupArrayIf" +"groupArrayInsertAt" +"groupArrayInsertAtArgMax" +"groupArrayInsertAtArgMin" +"groupArrayInsertAtArray" +"groupArrayInsertAtDistinct" +"groupArrayInsertAtForEach" +"groupArrayInsertAtIf" +"groupArrayInsertAtMap" +"groupArrayInsertAtMerge" +"groupArrayInsertAtNull" +"groupArrayInsertAtOrDefault" +"groupArrayInsertAtOrNull" +"groupArrayInsertAtResample" +"groupArrayInsertAtSimpleState" +"groupArrayInsertAtState" +"groupArrayIntersect" +"groupArrayIntersectArgMax" +"groupArrayIntersectArgMin" +"groupArrayIntersectArray" +"groupArrayIntersectDistinct" +"groupArrayIntersectForEach" +"groupArrayIntersectIf" +"groupArrayIntersectMap" +"groupArrayIntersectMerge" +"groupArrayIntersectNull" +"groupArrayIntersectOrDefault" +"groupArrayIntersectOrNull" +"groupArrayIntersectResample" +"groupArrayIntersectSimpleState" +"groupArrayIntersectState" +"groupArrayLast" +"groupArrayLastArgMax" +"groupArrayLastArgMin" +"groupArrayLastArray" +"groupArrayLastDistinct" +"groupArrayLastForEach" +"groupArrayLastIf" +"groupArrayLastMap" +"groupArrayLastMerge" +"groupArrayLastNull" +"groupArrayLastOrDefault" +"groupArrayLastOrNull" +"groupArrayLastResample" +"groupArrayLastSimpleState" +"groupArrayLastState" +"groupArrayMap" +"groupArrayMerge" +"groupArrayMovingAvg" +"groupArrayMovingAvgArgMax" +"groupArrayMovingAvgArgMin" +"groupArrayMovingAvgArray" +"groupArrayMovingAvgDistinct" +"groupArrayMovingAvgForEach" +"groupArrayMovingAvgIf" +"groupArrayMovingAvgMap" +"groupArrayMovingAvgMerge" +"groupArrayMovingAvgNull" +"groupArrayMovingAvgOrDefault" +"groupArrayMovingAvgOrNull" +"groupArrayMovingAvgResample" +"groupArrayMovingAvgSimpleState" +"groupArrayMovingAvgState" +"groupArrayMovingSum" +"groupArrayMovingSumArgMax" +"groupArrayMovingSumArgMin" +"groupArrayMovingSumArray" +"groupArrayMovingSumDistinct" +"groupArrayMovingSumForEach" +"groupArrayMovingSumIf" +"groupArrayMovingSumMap" +"groupArrayMovingSumMerge" +"groupArrayMovingSumNull" +"groupArrayMovingSumOrDefault" +"groupArrayMovingSumOrNull" +"groupArrayMovingSumResample" +"groupArrayMovingSumSimpleState" +"groupArrayMovingSumState" +"groupArrayNull" +"groupArrayOrDefault" +"groupArrayOrNull" +"groupArrayResample" +"groupArraySample" +"groupArraySampleArgMax" +"groupArraySampleArgMin" +"groupArraySampleArray" +"groupArraySampleDistinct" +"groupArraySampleForEach" +"groupArraySampleIf" +"groupArraySampleMap" +"groupArraySampleMerge" +"groupArraySampleNull" +"groupArraySampleOrDefault" +"groupArraySampleOrNull" +"groupArraySampleResample" +"groupArraySampleSimpleState" +"groupArraySampleState" +"groupArraySimpleState" +"groupArraySorted" +"groupArraySortedArgMax" +"groupArraySortedArgMin" +"groupArraySortedArray" +"groupArraySortedDistinct" +"groupArraySortedForEach" +"groupArraySortedIf" +"groupArraySortedMap" +"groupArraySortedMerge" +"groupArraySortedNull" +"groupArraySortedOrDefault" +"groupArraySortedOrNull" +"groupArraySortedResample" +"groupArraySortedSimpleState" +"groupArraySortedState" +"groupArrayState" +"groupBitAnd" +"groupBitAndArgMax" +"groupBitAndArgMin" +"groupBitAndArray" +"groupBitAndDistinct" +"groupBitAndForEach" +"groupBitAndIf" +"groupBitAndMap" +"groupBitAndMerge" +"groupBitAndNull" +"groupBitAndOrDefault" +"groupBitAndOrNull" +"groupBitAndResample" +"groupBitAndSimpleState" +"groupBitAndState" +"groupBitOr" +"groupBitOrArgMax" +"groupBitOrArgMin" +"groupBitOrArray" +"groupBitOrDistinct" +"groupBitOrForEach" +"groupBitOrIf" +"groupBitOrMap" +"groupBitOrMerge" +"groupBitOrNull" +"groupBitOrOrDefault" +"groupBitOrOrNull" +"groupBitOrResample" +"groupBitOrSimpleState" +"groupBitOrState" +"groupBitXor" +"groupBitXorArgMax" +"groupBitXorArgMin" +"groupBitXorArray" +"groupBitXorDistinct" +"groupBitXorForEach" +"groupBitXorIf" +"groupBitXorMap" +"groupBitXorMerge" +"groupBitXorNull" +"groupBitXorOrDefault" +"groupBitXorOrNull" +"groupBitXorResample" +"groupBitXorSimpleState" +"groupBitXorState" +"groupBitmap" +"groupBitmapAnd" +"groupBitmapAndArgMax" +"groupBitmapAndArgMin" +"groupBitmapAndArray" +"groupBitmapAndDistinct" +"groupBitmapAndForEach" +"groupBitmapAndIf" +"groupBitmapAndMap" +"groupBitmapAndMerge" +"groupBitmapAndNull" +"groupBitmapAndOrDefault" +"groupBitmapAndOrNull" +"groupBitmapAndResample" +"groupBitmapAndSimpleState" +"groupBitmapAndState" +"groupBitmapArgMax" +"groupBitmapArgMin" +"groupBitmapArray" +"groupBitmapDistinct" +"groupBitmapForEach" +"groupBitmapIf" +"groupBitmapMap" +"groupBitmapMerge" +"groupBitmapNull" +"groupBitmapOr" +"groupBitmapOrArgMax" +"groupBitmapOrArgMin" +"groupBitmapOrArray" +"groupBitmapOrDefault" +"groupBitmapOrDistinct" +"groupBitmapOrForEach" +"groupBitmapOrIf" +"groupBitmapOrMap" +"groupBitmapOrMerge" +"groupBitmapOrNull" +"groupBitmapOrNull" +"groupBitmapOrOrDefault" +"groupBitmapOrOrNull" +"groupBitmapOrResample" +"groupBitmapOrSimpleState" +"groupBitmapOrState" +"groupBitmapResample" +"groupBitmapSimpleState" +"groupBitmapState" +"groupBitmapXor" +"groupBitmapXorArgMax" +"groupBitmapXorArgMin" +"groupBitmapXorArray" +"groupBitmapXorDistinct" +"groupBitmapXorForEach" +"groupBitmapXorIf" +"groupBitmapXorMap" +"groupBitmapXorMerge" +"groupBitmapXorNull" +"groupBitmapXorOrDefault" +"groupBitmapXorOrNull" +"groupBitmapXorResample" +"groupBitmapXorSimpleState" +"groupBitmapXorState" +"groupConcat" +"groupConcatArgMax" +"groupConcatArgMin" +"groupConcatArray" +"groupConcatDistinct" +"groupConcatForEach" +"groupConcatIf" +"groupConcatMap" +"groupConcatMerge" +"groupConcatNull" +"groupConcatOrDefault" +"groupConcatOrNull" +"groupConcatResample" +"groupConcatSimpleState" +"groupConcatState" +"groupUniqArray" +"groupUniqArrayArgMax" +"groupUniqArrayArgMin" +"groupUniqArrayArray" +"groupUniqArrayDistinct" +"groupUniqArrayForEach" +"groupUniqArrayIf" +"groupUniqArrayMap" +"groupUniqArrayMerge" +"groupUniqArrayNull" +"groupUniqArrayOrDefault" +"groupUniqArrayOrNull" +"groupUniqArrayResample" +"groupUniqArraySimpleState" +"groupUniqArrayState" +"group_concat" +"group_concatArgMax" +"group_concatArgMin" +"group_concatArray" +"group_concatDistinct" +"group_concatForEach" +"group_concatIf" +"group_concatMap" +"group_concatMerge" +"group_concatNull" +"group_concatOrDefault" +"group_concatOrNull" +"group_concatResample" +"group_concatSimpleState" +"group_concatState" +"h3CellAreaM2" +"h3CellAreaRads2" +"h3Distance" +"h3EdgeAngle" +"h3EdgeLengthKm" +"h3EdgeLengthM" +"h3ExactEdgeLengthKm" +"h3ExactEdgeLengthM" +"h3ExactEdgeLengthRads" +"h3GetBaseCell" +"h3GetDestinationIndexFromUnidirectionalEdge" +"h3GetFaces" +"h3GetIndexesFromUnidirectionalEdge" +"h3GetOriginIndexFromUnidirectionalEdge" +"h3GetPentagonIndexes" +"h3GetRes0Indexes" +"h3GetResolution" +"h3GetUnidirectionalEdge" +"h3GetUnidirectionalEdgeBoundary" +"h3GetUnidirectionalEdgesFromHexagon" +"h3HexAreaKm2" +"h3HexAreaM2" +"h3HexRing" +"h3IndexesAreNeighbors" +"h3IsPentagon" +"h3IsResClassIII" +"h3IsValid" +"h3Line" +"h3NumHexagons" +"h3PointDistKm" +"h3PointDistM" +"h3PointDistRads" +"h3ToCenterChild" +"h3ToChildren" +"h3ToGeo" +"h3ToGeoBoundary" +"h3ToParent" +"h3ToString" +"h3UnidirectionalEdgeIsValid" +"h3kRing" +"halfMD5" +"has" +"hasAll" +"hasAny" +"hasColumnInTable" +"hasSubsequence" +"hasSubsequenceCaseInsensitive" +"hasSubsequenceCaseInsensitiveUTF8" +"hasSubsequenceUTF8" +"hasSubstr" +"hasThreadFuzzer" +"hasToken" +"hasTokenCaseInsensitive" +"hasTokenCaseInsensitiveOrNull" +"hasTokenOrNull" +"hex" +"hilbertDecode" +"hilbertEncode" +"histogram" +"histogramArgMax" +"histogramArgMin" +"histogramArray" +"histogramDistinct" +"histogramForEach" +"histogramIf" +"histogramMap" +"histogramMerge" +"histogramNull" +"histogramOrDefault" +"histogramOrNull" +"histogramResample" +"histogramSimpleState" +"histogramState" +"hiveHash" +"hop" +"hopEnd" +"hopStart" +"hostName" +"hostname" +"hypot" +"identity" +"idnaDecode" +"idnaEncode" +"if" +"ifNotFinite" +"ifNull" +"ignore" +"ilike" +"in" +"inIgnoreSet" +"indexHint" +"indexOf" +"initcap" +"initcapUTF8" +"initialQueryID" +"initial_query_id" +"initializeAggregation" +"instr" +"intDiv" +"intDivOrZero" +"intExp10" +"intExp2" +"intHash32" +"intHash64" +"intervalLengthSum" +"intervalLengthSumArgMax" +"intervalLengthSumArgMin" +"intervalLengthSumArray" +"intervalLengthSumDistinct" +"intervalLengthSumForEach" +"intervalLengthSumIf" +"intervalLengthSumMap" +"intervalLengthSumMerge" +"intervalLengthSumNull" +"intervalLengthSumOrDefault" +"intervalLengthSumOrNull" +"intervalLengthSumResample" +"intervalLengthSumSimpleState" +"intervalLengthSumState" +"isConstant" +"isDecimalOverflow" +"isFinite" +"isIPAddressInRange" +"isIPv4String" +"isIPv6String" +"isInfinite" +"isNaN" +"isNotDistinctFrom" +"isNotNull" +"isNull" +"isNullable" +"isValidJSON" +"isValidUTF8" +"isZeroOrNull" +"jaroSimilarity" +"jaroWinklerSimilarity" +"javaHash" +"javaHashUTF16LE" +"joinGet" +"joinGetOrNull" +"jsonMergePatch" +"jumpConsistentHash" +"kafkaMurmurHash" +"kolmogorovSmirnovTest" +"kolmogorovSmirnovTestArgMax" +"kolmogorovSmirnovTestArgMin" +"kolmogorovSmirnovTestArray" +"kolmogorovSmirnovTestDistinct" +"kolmogorovSmirnovTestForEach" +"kolmogorovSmirnovTestIf" +"kolmogorovSmirnovTestMap" +"kolmogorovSmirnovTestMerge" +"kolmogorovSmirnovTestNull" +"kolmogorovSmirnovTestOrDefault" +"kolmogorovSmirnovTestOrNull" +"kolmogorovSmirnovTestResample" +"kolmogorovSmirnovTestSimpleState" +"kolmogorovSmirnovTestState" +"kostikConsistentHash" +"kql_array_sort_asc" +"kql_array_sort_desc" +"kurtPop" +"kurtPopArgMax" +"kurtPopArgMin" +"kurtPopArray" +"kurtPopDistinct" +"kurtPopForEach" +"kurtPopIf" +"kurtPopMap" +"kurtPopMerge" +"kurtPopNull" +"kurtPopOrDefault" +"kurtPopOrNull" +"kurtPopResample" +"kurtPopSimpleState" +"kurtPopState" +"kurtSamp" +"kurtSampArgMax" +"kurtSampArgMin" +"kurtSampArray" +"kurtSampDistinct" +"kurtSampForEach" +"kurtSampIf" +"kurtSampMap" +"kurtSampMerge" +"kurtSampNull" +"kurtSampOrDefault" +"kurtSampOrNull" +"kurtSampResample" +"kurtSampSimpleState" +"kurtSampState" +"lagInFrame" +"lagInFrameArgMax" +"lagInFrameArgMin" +"lagInFrameArray" +"lagInFrameDistinct" +"lagInFrameForEach" +"lagInFrameIf" +"lagInFrameMap" +"lagInFrameMerge" +"lagInFrameNull" +"lagInFrameOrDefault" +"lagInFrameOrNull" +"lagInFrameResample" +"lagInFrameSimpleState" +"lagInFrameState" +"largestTriangleThreeBuckets" +"largestTriangleThreeBucketsArgMax" +"largestTriangleThreeBucketsArgMin" +"largestTriangleThreeBucketsArray" +"largestTriangleThreeBucketsDistinct" +"largestTriangleThreeBucketsForEach" +"largestTriangleThreeBucketsIf" +"largestTriangleThreeBucketsMap" +"largestTriangleThreeBucketsMerge" +"largestTriangleThreeBucketsNull" +"largestTriangleThreeBucketsOrDefault" +"largestTriangleThreeBucketsOrNull" +"largestTriangleThreeBucketsResample" +"largestTriangleThreeBucketsSimpleState" +"largestTriangleThreeBucketsState" +"last_value" +"last_valueArgMax" +"last_valueArgMin" +"last_valueArray" +"last_valueDistinct" +"last_valueForEach" +"last_valueIf" +"last_valueMap" +"last_valueMerge" +"last_valueNull" +"last_valueOrDefault" +"last_valueOrNull" +"last_valueResample" +"last_valueSimpleState" +"last_valueState" +"last_value_respect_nulls" +"last_value_respect_nullsArgMax" +"last_value_respect_nullsArgMin" +"last_value_respect_nullsArray" +"last_value_respect_nullsDistinct" +"last_value_respect_nullsForEach" +"last_value_respect_nullsIf" +"last_value_respect_nullsMap" +"last_value_respect_nullsMerge" +"last_value_respect_nullsNull" +"last_value_respect_nullsOrDefault" +"last_value_respect_nullsOrNull" +"last_value_respect_nullsResample" +"last_value_respect_nullsSimpleState" +"last_value_respect_nullsState" +"lcase" +"lcm" +"leadInFrame" +"leadInFrameArgMax" +"leadInFrameArgMin" +"leadInFrameArray" +"leadInFrameDistinct" +"leadInFrameForEach" +"leadInFrameIf" +"leadInFrameMap" +"leadInFrameMerge" +"leadInFrameNull" +"leadInFrameOrDefault" +"leadInFrameOrNull" +"leadInFrameResample" +"leadInFrameSimpleState" +"leadInFrameState" +"least" +"left" +"leftPad" +"leftPadUTF8" +"leftUTF8" +"lemmatize" +"length" +"lengthUTF8" +"less" +"lessOrEquals" +"levenshteinDistance" +"levenshteinDistanceUTF8" +"lgamma" +"like" +"ln" +"locate" +"log" +"log10" +"log1p" +"log2" +"logTrace" +"lowCardinalityIndices" +"lowCardinalityKeys" +"lower" +"lowerUTF8" +"lpad" +"ltrim" +"lttb" +"lttbArgMax" +"lttbArgMin" +"lttbArray" +"lttbDistinct" +"lttbForEach" +"lttbIf" +"lttbMap" +"lttbMerge" +"lttbNull" +"lttbOrDefault" +"lttbOrNull" +"lttbResample" +"lttbSimpleState" +"lttbState" +"makeDate" +"makeDate32" +"makeDateTime" +"makeDateTime64" +"mannWhitneyUTest" +"mannWhitneyUTestArgMax" +"mannWhitneyUTestArgMin" +"mannWhitneyUTestArray" +"mannWhitneyUTestDistinct" +"mannWhitneyUTestForEach" +"mannWhitneyUTestIf" +"mannWhitneyUTestMap" +"mannWhitneyUTestMerge" +"mannWhitneyUTestNull" +"mannWhitneyUTestOrDefault" +"mannWhitneyUTestOrNull" +"mannWhitneyUTestResample" +"mannWhitneyUTestSimpleState" +"mannWhitneyUTestState" +"map" +"mapAdd" +"mapAll" +"mapApply" +"mapConcat" +"mapContains" +"mapContainsKeyLike" +"mapExists" +"mapExtractKeyLike" +"mapFilter" +"mapFromArrays" +"mapFromString" +"mapKeys" +"mapPartialReverseSort" +"mapPartialSort" +"mapPopulateSeries" +"mapReverseSort" +"mapSort" +"mapSubtract" +"mapUpdate" +"mapValues" +"match" +"materialize" +"max" +"max2" +"maxArgMax" +"maxArgMin" +"maxArray" +"maxDistinct" +"maxForEach" +"maxIf" +"maxIntersections" +"maxIntersectionsArgMax" +"maxIntersectionsArgMin" +"maxIntersectionsArray" +"maxIntersectionsDistinct" +"maxIntersectionsForEach" +"maxIntersectionsIf" +"maxIntersectionsMap" +"maxIntersectionsMerge" +"maxIntersectionsNull" +"maxIntersectionsOrDefault" +"maxIntersectionsOrNull" +"maxIntersectionsPosition" +"maxIntersectionsPositionArgMax" +"maxIntersectionsPositionArgMin" +"maxIntersectionsPositionArray" +"maxIntersectionsPositionDistinct" +"maxIntersectionsPositionForEach" +"maxIntersectionsPositionIf" +"maxIntersectionsPositionMap" +"maxIntersectionsPositionMerge" +"maxIntersectionsPositionNull" +"maxIntersectionsPositionOrDefault" +"maxIntersectionsPositionOrNull" +"maxIntersectionsPositionResample" +"maxIntersectionsPositionSimpleState" +"maxIntersectionsPositionState" +"maxIntersectionsResample" +"maxIntersectionsSimpleState" +"maxIntersectionsState" +"maxMap" +"maxMappedArrays" +"maxMappedArraysArgMax" +"maxMappedArraysArgMin" +"maxMappedArraysArray" +"maxMappedArraysDistinct" +"maxMappedArraysForEach" +"maxMappedArraysIf" +"maxMappedArraysMap" +"maxMappedArraysMerge" +"maxMappedArraysNull" +"maxMappedArraysOrDefault" +"maxMappedArraysOrNull" +"maxMappedArraysResample" +"maxMappedArraysSimpleState" +"maxMappedArraysState" +"maxMerge" +"maxNull" +"maxOrDefault" +"maxOrNull" +"maxResample" +"maxSimpleState" +"maxState" +"meanZTest" +"meanZTestArgMax" +"meanZTestArgMin" +"meanZTestArray" +"meanZTestDistinct" +"meanZTestForEach" +"meanZTestIf" +"meanZTestMap" +"meanZTestMerge" +"meanZTestNull" +"meanZTestOrDefault" +"meanZTestOrNull" +"meanZTestResample" +"meanZTestSimpleState" +"meanZTestState" +"median" +"medianArgMax" +"medianArgMin" +"medianArray" +"medianBFloat16" +"medianBFloat16ArgMax" +"medianBFloat16ArgMin" +"medianBFloat16Array" +"medianBFloat16Distinct" +"medianBFloat16ForEach" +"medianBFloat16If" +"medianBFloat16Map" +"medianBFloat16Merge" +"medianBFloat16Null" +"medianBFloat16OrDefault" +"medianBFloat16OrNull" +"medianBFloat16Resample" +"medianBFloat16SimpleState" +"medianBFloat16State" +"medianBFloat16Weighted" +"medianBFloat16WeightedArgMax" +"medianBFloat16WeightedArgMin" +"medianBFloat16WeightedArray" +"medianBFloat16WeightedDistinct" +"medianBFloat16WeightedForEach" +"medianBFloat16WeightedIf" +"medianBFloat16WeightedMap" +"medianBFloat16WeightedMerge" +"medianBFloat16WeightedNull" +"medianBFloat16WeightedOrDefault" +"medianBFloat16WeightedOrNull" +"medianBFloat16WeightedResample" +"medianBFloat16WeightedSimpleState" +"medianBFloat16WeightedState" +"medianDD" +"medianDDArgMax" +"medianDDArgMin" +"medianDDArray" +"medianDDDistinct" +"medianDDForEach" +"medianDDIf" +"medianDDMap" +"medianDDMerge" +"medianDDNull" +"medianDDOrDefault" +"medianDDOrNull" +"medianDDResample" +"medianDDSimpleState" +"medianDDState" +"medianDeterministic" +"medianDeterministicArgMax" +"medianDeterministicArgMin" +"medianDeterministicArray" +"medianDeterministicDistinct" +"medianDeterministicForEach" +"medianDeterministicIf" +"medianDeterministicMap" +"medianDeterministicMerge" +"medianDeterministicNull" +"medianDeterministicOrDefault" +"medianDeterministicOrNull" +"medianDeterministicResample" +"medianDeterministicSimpleState" +"medianDeterministicState" +"medianDistinct" +"medianExact" +"medianExactArgMax" +"medianExactArgMin" +"medianExactArray" +"medianExactDistinct" +"medianExactForEach" +"medianExactHigh" +"medianExactHighArgMax" +"medianExactHighArgMin" +"medianExactHighArray" +"medianExactHighDistinct" +"medianExactHighForEach" +"medianExactHighIf" +"medianExactHighMap" +"medianExactHighMerge" +"medianExactHighNull" +"medianExactHighOrDefault" +"medianExactHighOrNull" +"medianExactHighResample" +"medianExactHighSimpleState" +"medianExactHighState" +"medianExactIf" +"medianExactLow" +"medianExactLowArgMax" +"medianExactLowArgMin" +"medianExactLowArray" +"medianExactLowDistinct" +"medianExactLowForEach" +"medianExactLowIf" +"medianExactLowMap" +"medianExactLowMerge" +"medianExactLowNull" +"medianExactLowOrDefault" +"medianExactLowOrNull" +"medianExactLowResample" +"medianExactLowSimpleState" +"medianExactLowState" +"medianExactMap" +"medianExactMerge" +"medianExactNull" +"medianExactOrDefault" +"medianExactOrNull" +"medianExactResample" +"medianExactSimpleState" +"medianExactState" +"medianExactWeighted" +"medianExactWeightedArgMax" +"medianExactWeightedArgMin" +"medianExactWeightedArray" +"medianExactWeightedDistinct" +"medianExactWeightedForEach" +"medianExactWeightedIf" +"medianExactWeightedMap" +"medianExactWeightedMerge" +"medianExactWeightedNull" +"medianExactWeightedOrDefault" +"medianExactWeightedOrNull" +"medianExactWeightedResample" +"medianExactWeightedSimpleState" +"medianExactWeightedState" +"medianForEach" +"medianGK" +"medianGKArgMax" +"medianGKArgMin" +"medianGKArray" +"medianGKDistinct" +"medianGKForEach" +"medianGKIf" +"medianGKMap" +"medianGKMerge" +"medianGKNull" +"medianGKOrDefault" +"medianGKOrNull" +"medianGKResample" +"medianGKSimpleState" +"medianGKState" +"medianIf" +"medianInterpolatedWeighted" +"medianInterpolatedWeightedArgMax" +"medianInterpolatedWeightedArgMin" +"medianInterpolatedWeightedArray" +"medianInterpolatedWeightedDistinct" +"medianInterpolatedWeightedForEach" +"medianInterpolatedWeightedIf" +"medianInterpolatedWeightedMap" +"medianInterpolatedWeightedMerge" +"medianInterpolatedWeightedNull" +"medianInterpolatedWeightedOrDefault" +"medianInterpolatedWeightedOrNull" +"medianInterpolatedWeightedResample" +"medianInterpolatedWeightedSimpleState" +"medianInterpolatedWeightedState" +"medianMap" +"medianMerge" +"medianNull" +"medianOrDefault" +"medianOrNull" +"medianResample" +"medianSimpleState" +"medianState" +"medianTDigest" +"medianTDigestArgMax" +"medianTDigestArgMin" +"medianTDigestArray" +"medianTDigestDistinct" +"medianTDigestForEach" +"medianTDigestIf" +"medianTDigestMap" +"medianTDigestMerge" +"medianTDigestNull" +"medianTDigestOrDefault" +"medianTDigestOrNull" +"medianTDigestResample" +"medianTDigestSimpleState" +"medianTDigestState" +"medianTDigestWeighted" +"medianTDigestWeightedArgMax" +"medianTDigestWeightedArgMin" +"medianTDigestWeightedArray" +"medianTDigestWeightedDistinct" +"medianTDigestWeightedForEach" +"medianTDigestWeightedIf" +"medianTDigestWeightedMap" +"medianTDigestWeightedMerge" +"medianTDigestWeightedNull" +"medianTDigestWeightedOrDefault" +"medianTDigestWeightedOrNull" +"medianTDigestWeightedResample" +"medianTDigestWeightedSimpleState" +"medianTDigestWeightedState" +"medianTiming" +"medianTimingArgMax" +"medianTimingArgMin" +"medianTimingArray" +"medianTimingDistinct" +"medianTimingForEach" +"medianTimingIf" +"medianTimingMap" +"medianTimingMerge" +"medianTimingNull" +"medianTimingOrDefault" +"medianTimingOrNull" +"medianTimingResample" +"medianTimingSimpleState" +"medianTimingState" +"medianTimingWeighted" +"medianTimingWeightedArgMax" +"medianTimingWeightedArgMin" +"medianTimingWeightedArray" +"medianTimingWeightedDistinct" +"medianTimingWeightedForEach" +"medianTimingWeightedIf" +"medianTimingWeightedMap" +"medianTimingWeightedMerge" +"medianTimingWeightedNull" +"medianTimingWeightedOrDefault" +"medianTimingWeightedOrNull" +"medianTimingWeightedResample" +"medianTimingWeightedSimpleState" +"medianTimingWeightedState" +"metroHash64" +"mid" +"min" +"min2" +"minArgMax" +"minArgMin" +"minArray" +"minDistinct" +"minForEach" +"minIf" +"minMap" +"minMappedArrays" +"minMappedArraysArgMax" +"minMappedArraysArgMin" +"minMappedArraysArray" +"minMappedArraysDistinct" +"minMappedArraysForEach" +"minMappedArraysIf" +"minMappedArraysMap" +"minMappedArraysMerge" +"minMappedArraysNull" +"minMappedArraysOrDefault" +"minMappedArraysOrNull" +"minMappedArraysResample" +"minMappedArraysSimpleState" +"minMappedArraysState" +"minMerge" +"minNull" +"minOrDefault" +"minOrNull" +"minResample" +"minSampleSizeContinous" +"minSampleSizeContinuous" +"minSampleSizeConversion" +"minSimpleState" +"minState" +"minus" +"mismatches" +"mod" +"modulo" +"moduloLegacy" +"moduloOrZero" +"monthName" +"mortonDecode" +"mortonEncode" +"multiFuzzyMatchAllIndices" +"multiFuzzyMatchAny" +"multiFuzzyMatchAnyIndex" +"multiIf" +"multiMatchAllIndices" +"multiMatchAny" +"multiMatchAnyIndex" +"multiSearchAllPositions" +"multiSearchAllPositionsCaseInsensitive" +"multiSearchAllPositionsCaseInsensitiveUTF8" +"multiSearchAllPositionsUTF8" +"multiSearchAny" +"multiSearchAnyCaseInsensitive" +"multiSearchAnyCaseInsensitiveUTF8" +"multiSearchAnyUTF8" +"multiSearchFirstIndex" +"multiSearchFirstIndexCaseInsensitive" +"multiSearchFirstIndexCaseInsensitiveUTF8" +"multiSearchFirstIndexUTF8" +"multiSearchFirstPosition" +"multiSearchFirstPositionCaseInsensitive" +"multiSearchFirstPositionCaseInsensitiveUTF8" +"multiSearchFirstPositionUTF8" +"multiply" +"multiplyDecimal" +"murmurHash2_32" +"murmurHash2_64" +"murmurHash3_128" +"murmurHash3_32" +"murmurHash3_64" +"negate" +"neighbor" +"nested" +"netloc" +"ngramDistance" +"ngramDistanceCaseInsensitive" +"ngramDistanceCaseInsensitiveUTF8" +"ngramDistanceUTF8" +"ngramMinHash" +"ngramMinHashArg" +"ngramMinHashArgCaseInsensitive" +"ngramMinHashArgCaseInsensitiveUTF8" +"ngramMinHashArgUTF8" +"ngramMinHashCaseInsensitive" +"ngramMinHashCaseInsensitiveUTF8" +"ngramMinHashUTF8" +"ngramSearch" +"ngramSearchCaseInsensitive" +"ngramSearchCaseInsensitiveUTF8" +"ngramSearchUTF8" +"ngramSimHash" +"ngramSimHashCaseInsensitive" +"ngramSimHashCaseInsensitiveUTF8" +"ngramSimHashUTF8" +"ngrams" +"nonNegativeDerivative" +"nonNegativeDerivativeArgMax" +"nonNegativeDerivativeArgMin" +"nonNegativeDerivativeArray" +"nonNegativeDerivativeDistinct" +"nonNegativeDerivativeForEach" +"nonNegativeDerivativeIf" +"nonNegativeDerivativeMap" +"nonNegativeDerivativeMerge" +"nonNegativeDerivativeNull" +"nonNegativeDerivativeOrDefault" +"nonNegativeDerivativeOrNull" +"nonNegativeDerivativeResample" +"nonNegativeDerivativeSimpleState" +"nonNegativeDerivativeState" +"normL1" +"normL2" +"normL2Squared" +"normLinf" +"normLp" +"normalizeL1" +"normalizeL2" +"normalizeLinf" +"normalizeLp" +"normalizeQuery" +"normalizeQueryKeepNames" +"normalizeUTF8NFC" +"normalizeUTF8NFD" +"normalizeUTF8NFKC" +"normalizeUTF8NFKD" +"normalizedQueryHash" +"normalizedQueryHashKeepNames" +"not" +"notEmpty" +"notEquals" +"notILike" +"notIn" +"notInIgnoreSet" +"notLike" +"notNullIn" +"notNullInIgnoreSet" +"nothing" +"nothingArgMax" +"nothingArgMin" +"nothingArray" +"nothingDistinct" +"nothingForEach" +"nothingIf" +"nothingMap" +"nothingMerge" +"nothingNull" +"nothingNull" +"nothingNullArgMax" +"nothingNullArgMin" +"nothingNullArray" +"nothingNullDistinct" +"nothingNullForEach" +"nothingNullIf" +"nothingNullMap" +"nothingNullMerge" +"nothingNullNull" +"nothingNullOrDefault" +"nothingNullOrNull" +"nothingNullResample" +"nothingNullSimpleState" +"nothingNullState" +"nothingOrDefault" +"nothingOrNull" +"nothingResample" +"nothingSimpleState" +"nothingState" +"nothingUInt64" +"nothingUInt64ArgMax" +"nothingUInt64ArgMin" +"nothingUInt64Array" +"nothingUInt64Distinct" +"nothingUInt64ForEach" +"nothingUInt64If" +"nothingUInt64Map" +"nothingUInt64Merge" +"nothingUInt64Null" +"nothingUInt64OrDefault" +"nothingUInt64OrNull" +"nothingUInt64Resample" +"nothingUInt64SimpleState" +"nothingUInt64State" +"now" +"now64" +"nowInBlock" +"nth_value" +"nth_valueArgMax" +"nth_valueArgMin" +"nth_valueArray" +"nth_valueDistinct" +"nth_valueForEach" +"nth_valueIf" +"nth_valueMap" +"nth_valueMerge" +"nth_valueNull" +"nth_valueOrDefault" +"nth_valueOrNull" +"nth_valueResample" +"nth_valueSimpleState" +"nth_valueState" +"ntile" +"ntileArgMax" +"ntileArgMin" +"ntileArray" +"ntileDistinct" +"ntileForEach" +"ntileIf" +"ntileMap" +"ntileMerge" +"ntileNull" +"ntileOrDefault" +"ntileOrNull" +"ntileResample" +"ntileSimpleState" +"ntileState" +"nullIf" +"nullIn" +"nullInIgnoreSet" +"or" +"parseDateTime" +"parseDateTime32BestEffort" +"parseDateTime32BestEffortOrNull" +"parseDateTime32BestEffortOrZero" +"parseDateTime64BestEffort" +"parseDateTime64BestEffortOrNull" +"parseDateTime64BestEffortOrZero" +"parseDateTime64BestEffortUS" +"parseDateTime64BestEffortUSOrNull" +"parseDateTime64BestEffortUSOrZero" +"parseDateTimeBestEffort" +"parseDateTimeBestEffortOrNull" +"parseDateTimeBestEffortOrZero" +"parseDateTimeBestEffortUS" +"parseDateTimeBestEffortUSOrNull" +"parseDateTimeBestEffortUSOrZero" +"parseDateTimeInJodaSyntax" +"parseDateTimeInJodaSyntaxOrNull" +"parseDateTimeInJodaSyntaxOrZero" +"parseDateTimeOrNull" +"parseDateTimeOrZero" +"parseReadableSize" +"parseReadableSizeOrNull" +"parseReadableSizeOrZero" +"parseTimeDelta" +"partitionID" +"partitionId" +"path" +"pathFull" +"percentRank" +"percentRankArgMax" +"percentRankArgMin" +"percentRankArray" +"percentRankDistinct" +"percentRankForEach" +"percentRankIf" +"percentRankMap" +"percentRankMerge" +"percentRankNull" +"percentRankOrDefault" +"percentRankOrNull" +"percentRankResample" +"percentRankSimpleState" +"percentRankState" +"percent_rank" +"percent_rankArgMax" +"percent_rankArgMin" +"percent_rankArray" +"percent_rankDistinct" +"percent_rankForEach" +"percent_rankIf" +"percent_rankMap" +"percent_rankMerge" +"percent_rankNull" +"percent_rankOrDefault" +"percent_rankOrNull" +"percent_rankResample" +"percent_rankSimpleState" +"percent_rankState" +"pi" +"plus" +"pmod" +"pointInEllipses" +"pointInPolygon" +"polygonAreaCartesian" +"polygonAreaSpherical" +"polygonConvexHullCartesian" +"polygonPerimeterCartesian" +"polygonPerimeterSpherical" +"polygonsDistanceCartesian" +"polygonsDistanceSpherical" +"polygonsEqualsCartesian" +"polygonsIntersectionCartesian" +"polygonsIntersectionSpherical" +"polygonsSymDifferenceCartesian" +"polygonsSymDifferenceSpherical" +"polygonsUnionCartesian" +"polygonsUnionSpherical" +"polygonsWithinCartesian" +"polygonsWithinSpherical" +"port" +"portRFC" +"position" +"positionCaseInsensitive" +"positionCaseInsensitiveUTF8" +"positionUTF8" +"positiveModulo" +"positive_modulo" +"pow" +"power" +"printf" +"proportionsZTest" +"protocol" +"punycodeDecode" +"punycodeEncode" +"quantile" +"quantileArgMax" +"quantileArgMin" +"quantileArray" +"quantileBFloat16" +"quantileBFloat16ArgMax" +"quantileBFloat16ArgMin" +"quantileBFloat16Array" +"quantileBFloat16Distinct" +"quantileBFloat16ForEach" +"quantileBFloat16If" +"quantileBFloat16Map" +"quantileBFloat16Merge" +"quantileBFloat16Null" +"quantileBFloat16OrDefault" +"quantileBFloat16OrNull" +"quantileBFloat16Resample" +"quantileBFloat16SimpleState" +"quantileBFloat16State" +"quantileBFloat16Weighted" +"quantileBFloat16WeightedArgMax" +"quantileBFloat16WeightedArgMin" +"quantileBFloat16WeightedArray" +"quantileBFloat16WeightedDistinct" +"quantileBFloat16WeightedForEach" +"quantileBFloat16WeightedIf" +"quantileBFloat16WeightedMap" +"quantileBFloat16WeightedMerge" +"quantileBFloat16WeightedNull" +"quantileBFloat16WeightedOrDefault" +"quantileBFloat16WeightedOrNull" +"quantileBFloat16WeightedResample" +"quantileBFloat16WeightedSimpleState" +"quantileBFloat16WeightedState" +"quantileDD" +"quantileDDArgMax" +"quantileDDArgMin" +"quantileDDArray" +"quantileDDDistinct" +"quantileDDForEach" +"quantileDDIf" +"quantileDDMap" +"quantileDDMerge" +"quantileDDNull" +"quantileDDOrDefault" +"quantileDDOrNull" +"quantileDDResample" +"quantileDDSimpleState" +"quantileDDState" +"quantileDeterministic" +"quantileDeterministicArgMax" +"quantileDeterministicArgMin" +"quantileDeterministicArray" +"quantileDeterministicDistinct" +"quantileDeterministicForEach" +"quantileDeterministicIf" +"quantileDeterministicMap" +"quantileDeterministicMerge" +"quantileDeterministicNull" +"quantileDeterministicOrDefault" +"quantileDeterministicOrNull" +"quantileDeterministicResample" +"quantileDeterministicSimpleState" +"quantileDeterministicState" +"quantileDistinct" +"quantileExact" +"quantileExactArgMax" +"quantileExactArgMin" +"quantileExactArray" +"quantileExactDistinct" +"quantileExactExclusive" +"quantileExactExclusiveArgMax" +"quantileExactExclusiveArgMin" +"quantileExactExclusiveArray" +"quantileExactExclusiveDistinct" +"quantileExactExclusiveForEach" +"quantileExactExclusiveIf" +"quantileExactExclusiveMap" +"quantileExactExclusiveMerge" +"quantileExactExclusiveNull" +"quantileExactExclusiveOrDefault" +"quantileExactExclusiveOrNull" +"quantileExactExclusiveResample" +"quantileExactExclusiveSimpleState" +"quantileExactExclusiveState" +"quantileExactForEach" +"quantileExactHigh" +"quantileExactHighArgMax" +"quantileExactHighArgMin" +"quantileExactHighArray" +"quantileExactHighDistinct" +"quantileExactHighForEach" +"quantileExactHighIf" +"quantileExactHighMap" +"quantileExactHighMerge" +"quantileExactHighNull" +"quantileExactHighOrDefault" +"quantileExactHighOrNull" +"quantileExactHighResample" +"quantileExactHighSimpleState" +"quantileExactHighState" +"quantileExactIf" +"quantileExactInclusive" +"quantileExactInclusiveArgMax" +"quantileExactInclusiveArgMin" +"quantileExactInclusiveArray" +"quantileExactInclusiveDistinct" +"quantileExactInclusiveForEach" +"quantileExactInclusiveIf" +"quantileExactInclusiveMap" +"quantileExactInclusiveMerge" +"quantileExactInclusiveNull" +"quantileExactInclusiveOrDefault" +"quantileExactInclusiveOrNull" +"quantileExactInclusiveResample" +"quantileExactInclusiveSimpleState" +"quantileExactInclusiveState" +"quantileExactLow" +"quantileExactLowArgMax" +"quantileExactLowArgMin" +"quantileExactLowArray" +"quantileExactLowDistinct" +"quantileExactLowForEach" +"quantileExactLowIf" +"quantileExactLowMap" +"quantileExactLowMerge" +"quantileExactLowNull" +"quantileExactLowOrDefault" +"quantileExactLowOrNull" +"quantileExactLowResample" +"quantileExactLowSimpleState" +"quantileExactLowState" +"quantileExactMap" +"quantileExactMerge" +"quantileExactNull" +"quantileExactOrDefault" +"quantileExactOrNull" +"quantileExactResample" +"quantileExactSimpleState" +"quantileExactState" +"quantileExactWeighted" +"quantileExactWeightedArgMax" +"quantileExactWeightedArgMin" +"quantileExactWeightedArray" +"quantileExactWeightedDistinct" +"quantileExactWeightedForEach" +"quantileExactWeightedIf" +"quantileExactWeightedMap" +"quantileExactWeightedMerge" +"quantileExactWeightedNull" +"quantileExactWeightedOrDefault" +"quantileExactWeightedOrNull" +"quantileExactWeightedResample" +"quantileExactWeightedSimpleState" +"quantileExactWeightedState" +"quantileForEach" +"quantileGK" +"quantileGKArgMax" +"quantileGKArgMin" +"quantileGKArray" +"quantileGKDistinct" +"quantileGKForEach" +"quantileGKIf" +"quantileGKMap" +"quantileGKMerge" +"quantileGKNull" +"quantileGKOrDefault" +"quantileGKOrNull" +"quantileGKResample" +"quantileGKSimpleState" +"quantileGKState" +"quantileIf" +"quantileInterpolatedWeighted" +"quantileInterpolatedWeightedArgMax" +"quantileInterpolatedWeightedArgMin" +"quantileInterpolatedWeightedArray" +"quantileInterpolatedWeightedDistinct" +"quantileInterpolatedWeightedForEach" +"quantileInterpolatedWeightedIf" +"quantileInterpolatedWeightedMap" +"quantileInterpolatedWeightedMerge" +"quantileInterpolatedWeightedNull" +"quantileInterpolatedWeightedOrDefault" +"quantileInterpolatedWeightedOrNull" +"quantileInterpolatedWeightedResample" +"quantileInterpolatedWeightedSimpleState" +"quantileInterpolatedWeightedState" +"quantileMap" +"quantileMerge" +"quantileNull" +"quantileOrDefault" +"quantileOrNull" +"quantileResample" +"quantileSimpleState" +"quantileState" +"quantileTDigest" +"quantileTDigestArgMax" +"quantileTDigestArgMin" +"quantileTDigestArray" +"quantileTDigestDistinct" +"quantileTDigestForEach" +"quantileTDigestIf" +"quantileTDigestMap" +"quantileTDigestMerge" +"quantileTDigestNull" +"quantileTDigestOrDefault" +"quantileTDigestOrNull" +"quantileTDigestResample" +"quantileTDigestSimpleState" +"quantileTDigestState" +"quantileTDigestWeighted" +"quantileTDigestWeightedArgMax" +"quantileTDigestWeightedArgMin" +"quantileTDigestWeightedArray" +"quantileTDigestWeightedDistinct" +"quantileTDigestWeightedForEach" +"quantileTDigestWeightedIf" +"quantileTDigestWeightedMap" +"quantileTDigestWeightedMerge" +"quantileTDigestWeightedNull" +"quantileTDigestWeightedOrDefault" +"quantileTDigestWeightedOrNull" +"quantileTDigestWeightedResample" +"quantileTDigestWeightedSimpleState" +"quantileTDigestWeightedState" +"quantileTiming" +"quantileTimingArgMax" +"quantileTimingArgMin" +"quantileTimingArray" +"quantileTimingDistinct" +"quantileTimingForEach" +"quantileTimingIf" +"quantileTimingMap" +"quantileTimingMerge" +"quantileTimingNull" +"quantileTimingOrDefault" +"quantileTimingOrNull" +"quantileTimingResample" +"quantileTimingSimpleState" +"quantileTimingState" +"quantileTimingWeighted" +"quantileTimingWeightedArgMax" +"quantileTimingWeightedArgMin" +"quantileTimingWeightedArray" +"quantileTimingWeightedDistinct" +"quantileTimingWeightedForEach" +"quantileTimingWeightedIf" +"quantileTimingWeightedMap" +"quantileTimingWeightedMerge" +"quantileTimingWeightedNull" +"quantileTimingWeightedOrDefault" +"quantileTimingWeightedOrNull" +"quantileTimingWeightedResample" +"quantileTimingWeightedSimpleState" +"quantileTimingWeightedState" +"quantiles" +"quantilesArgMax" +"quantilesArgMin" +"quantilesArray" +"quantilesBFloat16" +"quantilesBFloat16ArgMax" +"quantilesBFloat16ArgMin" +"quantilesBFloat16Array" +"quantilesBFloat16Distinct" +"quantilesBFloat16ForEach" +"quantilesBFloat16If" +"quantilesBFloat16Map" +"quantilesBFloat16Merge" +"quantilesBFloat16Null" +"quantilesBFloat16OrDefault" +"quantilesBFloat16OrNull" +"quantilesBFloat16Resample" +"quantilesBFloat16SimpleState" +"quantilesBFloat16State" +"quantilesBFloat16Weighted" +"quantilesBFloat16WeightedArgMax" +"quantilesBFloat16WeightedArgMin" +"quantilesBFloat16WeightedArray" +"quantilesBFloat16WeightedDistinct" +"quantilesBFloat16WeightedForEach" +"quantilesBFloat16WeightedIf" +"quantilesBFloat16WeightedMap" +"quantilesBFloat16WeightedMerge" +"quantilesBFloat16WeightedNull" +"quantilesBFloat16WeightedOrDefault" +"quantilesBFloat16WeightedOrNull" +"quantilesBFloat16WeightedResample" +"quantilesBFloat16WeightedSimpleState" +"quantilesBFloat16WeightedState" +"quantilesDD" +"quantilesDDArgMax" +"quantilesDDArgMin" +"quantilesDDArray" +"quantilesDDDistinct" +"quantilesDDForEach" +"quantilesDDIf" +"quantilesDDMap" +"quantilesDDMerge" +"quantilesDDNull" +"quantilesDDOrDefault" +"quantilesDDOrNull" +"quantilesDDResample" +"quantilesDDSimpleState" +"quantilesDDState" +"quantilesDeterministic" +"quantilesDeterministicArgMax" +"quantilesDeterministicArgMin" +"quantilesDeterministicArray" +"quantilesDeterministicDistinct" +"quantilesDeterministicForEach" +"quantilesDeterministicIf" +"quantilesDeterministicMap" +"quantilesDeterministicMerge" +"quantilesDeterministicNull" +"quantilesDeterministicOrDefault" +"quantilesDeterministicOrNull" +"quantilesDeterministicResample" +"quantilesDeterministicSimpleState" +"quantilesDeterministicState" +"quantilesDistinct" +"quantilesExact" +"quantilesExactArgMax" +"quantilesExactArgMin" +"quantilesExactArray" +"quantilesExactDistinct" +"quantilesExactExclusive" +"quantilesExactExclusiveArgMax" +"quantilesExactExclusiveArgMin" +"quantilesExactExclusiveArray" +"quantilesExactExclusiveDistinct" +"quantilesExactExclusiveForEach" +"quantilesExactExclusiveIf" +"quantilesExactExclusiveMap" +"quantilesExactExclusiveMerge" +"quantilesExactExclusiveNull" +"quantilesExactExclusiveOrDefault" +"quantilesExactExclusiveOrNull" +"quantilesExactExclusiveResample" +"quantilesExactExclusiveSimpleState" +"quantilesExactExclusiveState" +"quantilesExactForEach" +"quantilesExactHigh" +"quantilesExactHighArgMax" +"quantilesExactHighArgMin" +"quantilesExactHighArray" +"quantilesExactHighDistinct" +"quantilesExactHighForEach" +"quantilesExactHighIf" +"quantilesExactHighMap" +"quantilesExactHighMerge" +"quantilesExactHighNull" +"quantilesExactHighOrDefault" +"quantilesExactHighOrNull" +"quantilesExactHighResample" +"quantilesExactHighSimpleState" +"quantilesExactHighState" +"quantilesExactIf" +"quantilesExactInclusive" +"quantilesExactInclusiveArgMax" +"quantilesExactInclusiveArgMin" +"quantilesExactInclusiveArray" +"quantilesExactInclusiveDistinct" +"quantilesExactInclusiveForEach" +"quantilesExactInclusiveIf" +"quantilesExactInclusiveMap" +"quantilesExactInclusiveMerge" +"quantilesExactInclusiveNull" +"quantilesExactInclusiveOrDefault" +"quantilesExactInclusiveOrNull" +"quantilesExactInclusiveResample" +"quantilesExactInclusiveSimpleState" +"quantilesExactInclusiveState" +"quantilesExactLow" +"quantilesExactLowArgMax" +"quantilesExactLowArgMin" +"quantilesExactLowArray" +"quantilesExactLowDistinct" +"quantilesExactLowForEach" +"quantilesExactLowIf" +"quantilesExactLowMap" +"quantilesExactLowMerge" +"quantilesExactLowNull" +"quantilesExactLowOrDefault" +"quantilesExactLowOrNull" +"quantilesExactLowResample" +"quantilesExactLowSimpleState" +"quantilesExactLowState" +"quantilesExactMap" +"quantilesExactMerge" +"quantilesExactNull" +"quantilesExactOrDefault" +"quantilesExactOrNull" +"quantilesExactResample" +"quantilesExactSimpleState" +"quantilesExactState" +"quantilesExactWeighted" +"quantilesExactWeightedArgMax" +"quantilesExactWeightedArgMin" +"quantilesExactWeightedArray" +"quantilesExactWeightedDistinct" +"quantilesExactWeightedForEach" +"quantilesExactWeightedIf" +"quantilesExactWeightedMap" +"quantilesExactWeightedMerge" +"quantilesExactWeightedNull" +"quantilesExactWeightedOrDefault" +"quantilesExactWeightedOrNull" +"quantilesExactWeightedResample" +"quantilesExactWeightedSimpleState" +"quantilesExactWeightedState" +"quantilesForEach" +"quantilesGK" +"quantilesGKArgMax" +"quantilesGKArgMin" +"quantilesGKArray" +"quantilesGKDistinct" +"quantilesGKForEach" +"quantilesGKIf" +"quantilesGKMap" +"quantilesGKMerge" +"quantilesGKNull" +"quantilesGKOrDefault" +"quantilesGKOrNull" +"quantilesGKResample" +"quantilesGKSimpleState" +"quantilesGKState" +"quantilesIf" +"quantilesInterpolatedWeighted" +"quantilesInterpolatedWeightedArgMax" +"quantilesInterpolatedWeightedArgMin" +"quantilesInterpolatedWeightedArray" +"quantilesInterpolatedWeightedDistinct" +"quantilesInterpolatedWeightedForEach" +"quantilesInterpolatedWeightedIf" +"quantilesInterpolatedWeightedMap" +"quantilesInterpolatedWeightedMerge" +"quantilesInterpolatedWeightedNull" +"quantilesInterpolatedWeightedOrDefault" +"quantilesInterpolatedWeightedOrNull" +"quantilesInterpolatedWeightedResample" +"quantilesInterpolatedWeightedSimpleState" +"quantilesInterpolatedWeightedState" +"quantilesMap" +"quantilesMerge" +"quantilesNull" +"quantilesOrDefault" +"quantilesOrNull" +"quantilesResample" +"quantilesSimpleState" +"quantilesState" +"quantilesTDigest" +"quantilesTDigestArgMax" +"quantilesTDigestArgMin" +"quantilesTDigestArray" +"quantilesTDigestDistinct" +"quantilesTDigestForEach" +"quantilesTDigestIf" +"quantilesTDigestMap" +"quantilesTDigestMerge" +"quantilesTDigestNull" +"quantilesTDigestOrDefault" +"quantilesTDigestOrNull" +"quantilesTDigestResample" +"quantilesTDigestSimpleState" +"quantilesTDigestState" +"quantilesTDigestWeighted" +"quantilesTDigestWeightedArgMax" +"quantilesTDigestWeightedArgMin" +"quantilesTDigestWeightedArray" +"quantilesTDigestWeightedDistinct" +"quantilesTDigestWeightedForEach" +"quantilesTDigestWeightedIf" +"quantilesTDigestWeightedMap" +"quantilesTDigestWeightedMerge" +"quantilesTDigestWeightedNull" +"quantilesTDigestWeightedOrDefault" +"quantilesTDigestWeightedOrNull" +"quantilesTDigestWeightedResample" +"quantilesTDigestWeightedSimpleState" +"quantilesTDigestWeightedState" +"quantilesTiming" +"quantilesTimingArgMax" +"quantilesTimingArgMin" +"quantilesTimingArray" +"quantilesTimingDistinct" +"quantilesTimingForEach" +"quantilesTimingIf" +"quantilesTimingMap" +"quantilesTimingMerge" +"quantilesTimingNull" +"quantilesTimingOrDefault" +"quantilesTimingOrNull" +"quantilesTimingResample" +"quantilesTimingSimpleState" +"quantilesTimingState" +"quantilesTimingWeighted" +"quantilesTimingWeightedArgMax" +"quantilesTimingWeightedArgMin" +"quantilesTimingWeightedArray" +"quantilesTimingWeightedDistinct" +"quantilesTimingWeightedForEach" +"quantilesTimingWeightedIf" +"quantilesTimingWeightedMap" +"quantilesTimingWeightedMerge" +"quantilesTimingWeightedNull" +"quantilesTimingWeightedOrDefault" +"quantilesTimingWeightedOrNull" +"quantilesTimingWeightedResample" +"quantilesTimingWeightedSimpleState" +"quantilesTimingWeightedState" +"queryID" +"queryString" +"queryStringAndFragment" +"query_id" +"radians" +"rand" +"rand32" +"rand64" +"randBernoulli" +"randBinomial" +"randCanonical" +"randChiSquared" +"randConstant" +"randExponential" +"randFisherF" +"randLogNormal" +"randNegativeBinomial" +"randNormal" +"randPoisson" +"randStudentT" +"randUniform" +"randomFixedString" +"randomPrintableASCII" +"randomString" +"randomStringUTF8" +"range" +"rank" +"rankArgMax" +"rankArgMin" +"rankArray" +"rankCorr" +"rankCorrArgMax" +"rankCorrArgMin" +"rankCorrArray" +"rankCorrDistinct" +"rankCorrForEach" +"rankCorrIf" +"rankCorrMap" +"rankCorrMerge" +"rankCorrNull" +"rankCorrOrDefault" +"rankCorrOrNull" +"rankCorrResample" +"rankCorrSimpleState" +"rankCorrState" +"rankDistinct" +"rankForEach" +"rankIf" +"rankMap" +"rankMerge" +"rankNull" +"rankOrDefault" +"rankOrNull" +"rankResample" +"rankSimpleState" +"rankState" +"readWKTLineString" +"readWKTMultiLineString" +"readWKTMultiPolygon" +"readWKTPoint" +"readWKTPolygon" +"readWKTRing" +"regexpExtract" +"regexpQuoteMeta" +"regionHierarchy" +"regionIn" +"regionToArea" +"regionToCity" +"regionToContinent" +"regionToCountry" +"regionToDistrict" +"regionToName" +"regionToPopulation" +"regionToTopContinent" +"reinterpret" +"reinterpretAsDate" +"reinterpretAsDateTime" +"reinterpretAsFixedString" +"reinterpretAsFloat32" +"reinterpretAsFloat64" +"reinterpretAsInt128" +"reinterpretAsInt16" +"reinterpretAsInt256" +"reinterpretAsInt32" +"reinterpretAsInt64" +"reinterpretAsInt8" +"reinterpretAsString" +"reinterpretAsUInt128" +"reinterpretAsUInt16" +"reinterpretAsUInt256" +"reinterpretAsUInt32" +"reinterpretAsUInt64" +"reinterpretAsUInt8" +"reinterpretAsUUID" +"repeat" +"replace" +"replaceAll" +"replaceOne" +"replaceRegexpAll" +"replaceRegexpOne" +"replicate" +"retention" +"retentionArgMax" +"retentionArgMin" +"retentionArray" +"retentionDistinct" +"retentionForEach" +"retentionIf" +"retentionMap" +"retentionMerge" +"retentionNull" +"retentionOrDefault" +"retentionOrNull" +"retentionResample" +"retentionSimpleState" +"retentionState" +"reverse" +"reverseUTF8" +"revision" +"right" +"rightPad" +"rightPadUTF8" +"rightUTF8" +"round" +"roundAge" +"roundBankers" +"roundDown" +"roundDuration" +"roundToExp2" +"rowNumberInAllBlocks" +"rowNumberInBlock" +"row_number" +"row_numberArgMax" +"row_numberArgMin" +"row_numberArray" +"row_numberDistinct" +"row_numberForEach" +"row_numberIf" +"row_numberMap" +"row_numberMerge" +"row_numberNull" +"row_numberOrDefault" +"row_numberOrNull" +"row_numberResample" +"row_numberSimpleState" +"row_numberState" +"rpad" +"rtrim" +"runningAccumulate" +"runningConcurrency" +"runningDifference" +"runningDifferenceStartingWithFirstValue" +"s2CapContains" +"s2CapUnion" +"s2CellsIntersect" +"s2GetNeighbors" +"s2RectAdd" +"s2RectContains" +"s2RectIntersection" +"s2RectUnion" +"s2ToGeo" +"scalarProduct" +"sequenceCount" +"sequenceCountArgMax" +"sequenceCountArgMin" +"sequenceCountArray" +"sequenceCountDistinct" +"sequenceCountForEach" +"sequenceCountIf" +"sequenceCountMap" +"sequenceCountMerge" +"sequenceCountNull" +"sequenceCountOrDefault" +"sequenceCountOrNull" +"sequenceCountResample" +"sequenceCountSimpleState" +"sequenceCountState" +"sequenceMatch" +"sequenceMatchArgMax" +"sequenceMatchArgMin" +"sequenceMatchArray" +"sequenceMatchDistinct" +"sequenceMatchForEach" +"sequenceMatchIf" +"sequenceMatchMap" +"sequenceMatchMerge" +"sequenceMatchNull" +"sequenceMatchOrDefault" +"sequenceMatchOrNull" +"sequenceMatchResample" +"sequenceMatchSimpleState" +"sequenceMatchState" +"sequenceNextNode" +"sequenceNextNodeArgMax" +"sequenceNextNodeArgMin" +"sequenceNextNodeArray" +"sequenceNextNodeDistinct" +"sequenceNextNodeForEach" +"sequenceNextNodeIf" +"sequenceNextNodeMap" +"sequenceNextNodeMerge" +"sequenceNextNodeNull" +"sequenceNextNodeOrDefault" +"sequenceNextNodeOrNull" +"sequenceNextNodeResample" +"sequenceNextNodeSimpleState" +"sequenceNextNodeState" +"seriesDecomposeSTL" +"seriesOutliersDetectTukey" +"seriesPeriodDetectFFT" +"serverTimeZone" +"serverTimezone" +"serverUUID" +"shardCount" +"shardNum" +"showCertificate" +"sigmoid" +"sign" +"simpleJSONExtractBool" +"simpleJSONExtractFloat" +"simpleJSONExtractInt" +"simpleJSONExtractRaw" +"simpleJSONExtractString" +"simpleJSONExtractUInt" +"simpleJSONHas" +"simpleLinearRegression" +"simpleLinearRegressionArgMax" +"simpleLinearRegressionArgMin" +"simpleLinearRegressionArray" +"simpleLinearRegressionDistinct" +"simpleLinearRegressionForEach" +"simpleLinearRegressionIf" +"simpleLinearRegressionMap" +"simpleLinearRegressionMerge" +"simpleLinearRegressionNull" +"simpleLinearRegressionOrDefault" +"simpleLinearRegressionOrNull" +"simpleLinearRegressionResample" +"simpleLinearRegressionSimpleState" +"simpleLinearRegressionState" +"sin" +"singleValueOrNull" +"singleValueOrNullArgMax" +"singleValueOrNullArgMin" +"singleValueOrNullArray" +"singleValueOrNullDistinct" +"singleValueOrNullForEach" +"singleValueOrNullIf" +"singleValueOrNullMap" +"singleValueOrNullMerge" +"singleValueOrNullNull" +"singleValueOrNullOrDefault" +"singleValueOrNullOrNull" +"singleValueOrNullResample" +"singleValueOrNullSimpleState" +"singleValueOrNullState" +"sinh" +"sipHash128" +"sipHash128Keyed" +"sipHash128Reference" +"sipHash128ReferenceKeyed" +"sipHash64" +"sipHash64Keyed" +"skewPop" +"skewPopArgMax" +"skewPopArgMin" +"skewPopArray" +"skewPopDistinct" +"skewPopForEach" +"skewPopIf" +"skewPopMap" +"skewPopMerge" +"skewPopNull" +"skewPopOrDefault" +"skewPopOrNull" +"skewPopResample" +"skewPopSimpleState" +"skewPopState" +"skewSamp" +"skewSampArgMax" +"skewSampArgMin" +"skewSampArray" +"skewSampDistinct" +"skewSampForEach" +"skewSampIf" +"skewSampMap" +"skewSampMerge" +"skewSampNull" +"skewSampOrDefault" +"skewSampOrNull" +"skewSampResample" +"skewSampSimpleState" +"skewSampState" +"sleep" +"sleepEachRow" +"snowflakeIDToDateTime" +"snowflakeIDToDateTime64" +"snowflakeToDateTime" +"snowflakeToDateTime64" +"soundex" +"space" +"sparkBar" +"sparkBarArgMax" +"sparkBarArgMin" +"sparkBarArray" +"sparkBarDistinct" +"sparkBarForEach" +"sparkBarIf" +"sparkBarMap" +"sparkBarMerge" +"sparkBarNull" +"sparkBarOrDefault" +"sparkBarOrNull" +"sparkBarResample" +"sparkBarSimpleState" +"sparkBarState" +"sparkbar" +"sparkbarArgMax" +"sparkbarArgMin" +"sparkbarArray" +"sparkbarDistinct" +"sparkbarForEach" +"sparkbarIf" +"sparkbarMap" +"sparkbarMerge" +"sparkbarNull" +"sparkbarOrDefault" +"sparkbarOrNull" +"sparkbarResample" +"sparkbarSimpleState" +"sparkbarState" +"splitByAlpha" +"splitByChar" +"splitByNonAlpha" +"splitByRegexp" +"splitByString" +"splitByWhitespace" +"sqid" +"sqidDecode" +"sqidEncode" +"sqrt" +"startsWith" +"startsWithUTF8" +"stddevPop" +"stddevPopArgMax" +"stddevPopArgMin" +"stddevPopArray" +"stddevPopDistinct" +"stddevPopForEach" +"stddevPopIf" +"stddevPopMap" +"stddevPopMerge" +"stddevPopNull" +"stddevPopOrDefault" +"stddevPopOrNull" +"stddevPopResample" +"stddevPopSimpleState" +"stddevPopStable" +"stddevPopStableArgMax" +"stddevPopStableArgMin" +"stddevPopStableArray" +"stddevPopStableDistinct" +"stddevPopStableForEach" +"stddevPopStableIf" +"stddevPopStableMap" +"stddevPopStableMerge" +"stddevPopStableNull" +"stddevPopStableOrDefault" +"stddevPopStableOrNull" +"stddevPopStableResample" +"stddevPopStableSimpleState" +"stddevPopStableState" +"stddevPopState" +"stddevSamp" +"stddevSampArgMax" +"stddevSampArgMin" +"stddevSampArray" +"stddevSampDistinct" +"stddevSampForEach" +"stddevSampIf" +"stddevSampMap" +"stddevSampMerge" +"stddevSampNull" +"stddevSampOrDefault" +"stddevSampOrNull" +"stddevSampResample" +"stddevSampSimpleState" +"stddevSampStable" +"stddevSampStableArgMax" +"stddevSampStableArgMin" +"stddevSampStableArray" +"stddevSampStableDistinct" +"stddevSampStableForEach" +"stddevSampStableIf" +"stddevSampStableMap" +"stddevSampStableMerge" +"stddevSampStableNull" +"stddevSampStableOrDefault" +"stddevSampStableOrNull" +"stddevSampStableResample" +"stddevSampStableSimpleState" +"stddevSampStableState" +"stddevSampState" +"stem" +"stochasticLinearRegression" +"stochasticLinearRegressionArgMax" +"stochasticLinearRegressionArgMin" +"stochasticLinearRegressionArray" +"stochasticLinearRegressionDistinct" +"stochasticLinearRegressionForEach" +"stochasticLinearRegressionIf" +"stochasticLinearRegressionMap" +"stochasticLinearRegressionMerge" +"stochasticLinearRegressionNull" +"stochasticLinearRegressionOrDefault" +"stochasticLinearRegressionOrNull" +"stochasticLinearRegressionResample" +"stochasticLinearRegressionSimpleState" +"stochasticLinearRegressionState" +"stochasticLogisticRegression" +"stochasticLogisticRegressionArgMax" +"stochasticLogisticRegressionArgMin" +"stochasticLogisticRegressionArray" +"stochasticLogisticRegressionDistinct" +"stochasticLogisticRegressionForEach" +"stochasticLogisticRegressionIf" +"stochasticLogisticRegressionMap" +"stochasticLogisticRegressionMerge" +"stochasticLogisticRegressionNull" +"stochasticLogisticRegressionOrDefault" +"stochasticLogisticRegressionOrNull" +"stochasticLogisticRegressionResample" +"stochasticLogisticRegressionSimpleState" +"stochasticLogisticRegressionState" +"str_to_date" +"str_to_map" +"stringJaccardIndex" +"stringJaccardIndexUTF8" +"stringToH3" +"structureToCapnProtoSchema" +"structureToProtobufSchema" +"studentTTest" +"studentTTestArgMax" +"studentTTestArgMin" +"studentTTestArray" +"studentTTestDistinct" +"studentTTestForEach" +"studentTTestIf" +"studentTTestMap" +"studentTTestMerge" +"studentTTestNull" +"studentTTestOrDefault" +"studentTTestOrNull" +"studentTTestResample" +"studentTTestSimpleState" +"studentTTestState" +"subBitmap" +"subDate" +"substr" +"substring" +"substringIndex" +"substringIndexUTF8" +"substringUTF8" +"subtractDays" +"subtractHours" +"subtractInterval" +"subtractMicroseconds" +"subtractMilliseconds" +"subtractMinutes" +"subtractMonths" +"subtractNanoseconds" +"subtractQuarters" +"subtractSeconds" +"subtractTupleOfIntervals" +"subtractWeeks" +"subtractYears" +"sum" +"sumArgMax" +"sumArgMin" +"sumArray" +"sumCount" +"sumCountArgMax" +"sumCountArgMin" +"sumCountArray" +"sumCountDistinct" +"sumCountForEach" +"sumCountIf" +"sumCountMap" +"sumCountMerge" +"sumCountNull" +"sumCountOrDefault" +"sumCountOrNull" +"sumCountResample" +"sumCountSimpleState" +"sumCountState" +"sumDistinct" +"sumForEach" +"sumIf" +"sumKahan" +"sumKahanArgMax" +"sumKahanArgMin" +"sumKahanArray" +"sumKahanDistinct" +"sumKahanForEach" +"sumKahanIf" +"sumKahanMap" +"sumKahanMerge" +"sumKahanNull" +"sumKahanOrDefault" +"sumKahanOrNull" +"sumKahanResample" +"sumKahanSimpleState" +"sumKahanState" +"sumMap" +"sumMapFiltered" +"sumMapFilteredArgMax" +"sumMapFilteredArgMin" +"sumMapFilteredArray" +"sumMapFilteredDistinct" +"sumMapFilteredForEach" +"sumMapFilteredIf" +"sumMapFilteredMap" +"sumMapFilteredMerge" +"sumMapFilteredNull" +"sumMapFilteredOrDefault" +"sumMapFilteredOrNull" +"sumMapFilteredResample" +"sumMapFilteredSimpleState" +"sumMapFilteredState" +"sumMapFilteredWithOverflow" +"sumMapFilteredWithOverflowArgMax" +"sumMapFilteredWithOverflowArgMin" +"sumMapFilteredWithOverflowArray" +"sumMapFilteredWithOverflowDistinct" +"sumMapFilteredWithOverflowForEach" +"sumMapFilteredWithOverflowIf" +"sumMapFilteredWithOverflowMap" +"sumMapFilteredWithOverflowMerge" +"sumMapFilteredWithOverflowNull" +"sumMapFilteredWithOverflowOrDefault" +"sumMapFilteredWithOverflowOrNull" +"sumMapFilteredWithOverflowResample" +"sumMapFilteredWithOverflowSimpleState" +"sumMapFilteredWithOverflowState" +"sumMapWithOverflow" +"sumMapWithOverflowArgMax" +"sumMapWithOverflowArgMin" +"sumMapWithOverflowArray" +"sumMapWithOverflowDistinct" +"sumMapWithOverflowForEach" +"sumMapWithOverflowIf" +"sumMapWithOverflowMap" +"sumMapWithOverflowMerge" +"sumMapWithOverflowNull" +"sumMapWithOverflowOrDefault" +"sumMapWithOverflowOrNull" +"sumMapWithOverflowResample" +"sumMapWithOverflowSimpleState" +"sumMapWithOverflowState" +"sumMappedArrays" +"sumMappedArraysArgMax" +"sumMappedArraysArgMin" +"sumMappedArraysArray" +"sumMappedArraysDistinct" +"sumMappedArraysForEach" +"sumMappedArraysIf" +"sumMappedArraysMap" +"sumMappedArraysMerge" +"sumMappedArraysNull" +"sumMappedArraysOrDefault" +"sumMappedArraysOrNull" +"sumMappedArraysResample" +"sumMappedArraysSimpleState" +"sumMappedArraysState" +"sumMerge" +"sumNull" +"sumOrDefault" +"sumOrNull" +"sumResample" +"sumSimpleState" +"sumState" +"sumWithOverflow" +"sumWithOverflowArgMax" +"sumWithOverflowArgMin" +"sumWithOverflowArray" +"sumWithOverflowDistinct" +"sumWithOverflowForEach" +"sumWithOverflowIf" +"sumWithOverflowMap" +"sumWithOverflowMerge" +"sumWithOverflowNull" +"sumWithOverflowOrDefault" +"sumWithOverflowOrNull" +"sumWithOverflowResample" +"sumWithOverflowSimpleState" +"sumWithOverflowState" +"svg" +"synonyms" +"tan" +"tanh" +"tcpPort" +"tgamma" +"theilsU" +"theilsUArgMax" +"theilsUArgMin" +"theilsUArray" +"theilsUDistinct" +"theilsUForEach" +"theilsUIf" +"theilsUMap" +"theilsUMerge" +"theilsUNull" +"theilsUOrDefault" +"theilsUOrNull" +"theilsUResample" +"theilsUSimpleState" +"theilsUState" +"throwIf" +"tid" +"timeDiff" +"timeSlot" +"timeSlots" +"timeZone" +"timeZoneOf" +"timeZoneOffset" +"timestamp" +"timestampDiff" +"timestamp_diff" +"timezone" +"timezoneOf" +"timezoneOffset" +"toBool" +"toColumnTypeName" +"toDate" +"toDate32" +"toDate32OrDefault" +"toDate32OrNull" +"toDate32OrZero" +"toDateOrDefault" +"toDateOrNull" +"toDateOrZero" +"toDateTime" +"toDateTime32" +"toDateTime64" +"toDateTime64OrDefault" +"toDateTime64OrNull" +"toDateTime64OrZero" +"toDateTimeOrDefault" +"toDateTimeOrNull" +"toDateTimeOrZero" +"toDayOfMonth" +"toDayOfWeek" +"toDayOfYear" +"toDaysSinceYearZero" +"toDecimal128" +"toDecimal128OrDefault" +"toDecimal128OrNull" +"toDecimal128OrZero" +"toDecimal256" +"toDecimal256OrDefault" +"toDecimal256OrNull" +"toDecimal256OrZero" +"toDecimal32" +"toDecimal32OrDefault" +"toDecimal32OrNull" +"toDecimal32OrZero" +"toDecimal64" +"toDecimal64OrDefault" +"toDecimal64OrNull" +"toDecimal64OrZero" +"toDecimalString" +"toFixedString" +"toFloat32" +"toFloat32OrDefault" +"toFloat32OrNull" +"toFloat32OrZero" +"toFloat64" +"toFloat64OrDefault" +"toFloat64OrNull" +"toFloat64OrZero" +"toHour" +"toIPv4" +"toIPv4OrDefault" +"toIPv4OrNull" +"toIPv4OrZero" +"toIPv6" +"toIPv6OrDefault" +"toIPv6OrNull" +"toIPv6OrZero" +"toISOWeek" +"toISOYear" +"toInt128" +"toInt128OrDefault" +"toInt128OrNull" +"toInt128OrZero" +"toInt16" +"toInt16OrDefault" +"toInt16OrNull" +"toInt16OrZero" +"toInt256" +"toInt256OrDefault" +"toInt256OrNull" +"toInt256OrZero" +"toInt32" +"toInt32OrDefault" +"toInt32OrNull" +"toInt32OrZero" +"toInt64" +"toInt64OrDefault" +"toInt64OrNull" +"toInt64OrZero" +"toInt8" +"toInt8OrDefault" +"toInt8OrNull" +"toInt8OrZero" +"toIntervalDay" +"toIntervalHour" +"toIntervalMicrosecond" +"toIntervalMillisecond" +"toIntervalMinute" +"toIntervalMonth" +"toIntervalNanosecond" +"toIntervalQuarter" +"toIntervalSecond" +"toIntervalWeek" +"toIntervalYear" +"toJSONString" +"toLastDayOfMonth" +"toLastDayOfWeek" +"toLowCardinality" +"toMillisecond" +"toMinute" +"toModifiedJulianDay" +"toModifiedJulianDayOrNull" +"toMonday" +"toMonth" +"toNullable" +"toQuarter" +"toRelativeDayNum" +"toRelativeHourNum" +"toRelativeMinuteNum" +"toRelativeMonthNum" +"toRelativeQuarterNum" +"toRelativeSecondNum" +"toRelativeWeekNum" +"toRelativeYearNum" +"toSecond" +"toStartOfDay" +"toStartOfFifteenMinutes" +"toStartOfFiveMinute" +"toStartOfFiveMinutes" +"toStartOfHour" +"toStartOfISOYear" +"toStartOfInterval" +"toStartOfMicrosecond" +"toStartOfMillisecond" +"toStartOfMinute" +"toStartOfMonth" +"toStartOfNanosecond" +"toStartOfQuarter" +"toStartOfSecond" +"toStartOfTenMinutes" +"toStartOfWeek" +"toStartOfYear" +"toString" +"toStringCutToZero" +"toTime" +"toTimeZone" +"toTimezone" +"toTypeName" +"toUInt128" +"toUInt128OrDefault" +"toUInt128OrNull" +"toUInt128OrZero" +"toUInt16" +"toUInt16OrDefault" +"toUInt16OrNull" +"toUInt16OrZero" +"toUInt256" +"toUInt256OrDefault" +"toUInt256OrNull" +"toUInt256OrZero" +"toUInt32" +"toUInt32OrDefault" +"toUInt32OrNull" +"toUInt32OrZero" +"toUInt64" +"toUInt64OrDefault" +"toUInt64OrNull" +"toUInt64OrZero" +"toUInt8" +"toUInt8OrDefault" +"toUInt8OrNull" +"toUInt8OrZero" +"toUTCTimestamp" +"toUUID" +"toUUIDOrDefault" +"toUUIDOrNull" +"toUUIDOrZero" +"toUnixTimestamp" +"toUnixTimestamp64Micro" +"toUnixTimestamp64Milli" +"toUnixTimestamp64Nano" +"toValidUTF8" +"toWeek" +"toYYYYMM" +"toYYYYMMDD" +"toYYYYMMDDhhmmss" +"toYear" +"toYearWeek" +"to_utc_timestamp" +"today" +"tokens" +"topK" +"topKArgMax" +"topKArgMin" +"topKArray" +"topKDistinct" +"topKForEach" +"topKIf" +"topKMap" +"topKMerge" +"topKNull" +"topKOrDefault" +"topKOrNull" +"topKResample" +"topKSimpleState" +"topKState" +"topKWeighted" +"topKWeightedArgMax" +"topKWeightedArgMin" +"topKWeightedArray" +"topKWeightedDistinct" +"topKWeightedForEach" +"topKWeightedIf" +"topKWeightedMap" +"topKWeightedMerge" +"topKWeightedNull" +"topKWeightedOrDefault" +"topKWeightedOrNull" +"topKWeightedResample" +"topKWeightedSimpleState" +"topKWeightedState" +"topLevelDomain" +"topLevelDomainRFC" +"transactionID" +"transactionLatestSnapshot" +"transactionOldestSnapshot" +"transform" +"translate" +"translateUTF8" +"trim" +"trimBoth" +"trimLeft" +"trimRight" +"trunc" +"truncate" +"tryBase58Decode" +"tryBase64Decode" +"tryBase64URLDecode" +"tryDecrypt" +"tryIdnaEncode" +"tryPunycodeDecode" +"tumble" +"tumbleEnd" +"tumbleStart" +"tuple" +"tupleConcat" +"tupleDivide" +"tupleDivideByNumber" +"tupleElement" +"tupleHammingDistance" +"tupleIntDiv" +"tupleIntDivByNumber" +"tupleIntDivOrZero" +"tupleIntDivOrZeroByNumber" +"tupleMinus" +"tupleModulo" +"tupleModuloByNumber" +"tupleMultiply" +"tupleMultiplyByNumber" +"tupleNames" +"tupleNegate" +"tuplePlus" +"tupleToNameValuePairs" +"ucase" +"unbin" +"unhex" +"uniq" +"uniqArgMax" +"uniqArgMin" +"uniqArray" +"uniqCombined" +"uniqCombined64" +"uniqCombined64ArgMax" +"uniqCombined64ArgMin" +"uniqCombined64Array" +"uniqCombined64Distinct" +"uniqCombined64ForEach" +"uniqCombined64If" +"uniqCombined64Map" +"uniqCombined64Merge" +"uniqCombined64Null" +"uniqCombined64OrDefault" +"uniqCombined64OrNull" +"uniqCombined64Resample" +"uniqCombined64SimpleState" +"uniqCombined64State" +"uniqCombinedArgMax" +"uniqCombinedArgMin" +"uniqCombinedArray" +"uniqCombinedDistinct" +"uniqCombinedForEach" +"uniqCombinedIf" +"uniqCombinedMap" +"uniqCombinedMerge" +"uniqCombinedNull" +"uniqCombinedOrDefault" +"uniqCombinedOrNull" +"uniqCombinedResample" +"uniqCombinedSimpleState" +"uniqCombinedState" +"uniqDistinct" +"uniqExact" +"uniqExactArgMax" +"uniqExactArgMin" +"uniqExactArray" +"uniqExactDistinct" +"uniqExactForEach" +"uniqExactIf" +"uniqExactMap" +"uniqExactMerge" +"uniqExactNull" +"uniqExactOrDefault" +"uniqExactOrNull" +"uniqExactResample" +"uniqExactSimpleState" +"uniqExactState" +"uniqForEach" +"uniqHLL12" +"uniqHLL12ArgMax" +"uniqHLL12ArgMin" +"uniqHLL12Array" +"uniqHLL12Distinct" +"uniqHLL12ForEach" +"uniqHLL12If" +"uniqHLL12Map" +"uniqHLL12Merge" +"uniqHLL12Null" +"uniqHLL12OrDefault" +"uniqHLL12OrNull" +"uniqHLL12Resample" +"uniqHLL12SimpleState" +"uniqHLL12State" +"uniqIf" +"uniqMap" +"uniqMerge" +"uniqNull" +"uniqOrDefault" +"uniqOrNull" +"uniqResample" +"uniqSimpleState" +"uniqState" +"uniqTheta" +"uniqThetaArgMax" +"uniqThetaArgMin" +"uniqThetaArray" +"uniqThetaDistinct" +"uniqThetaForEach" +"uniqThetaIf" +"uniqThetaIntersect" +"uniqThetaMap" +"uniqThetaMerge" +"uniqThetaNot" +"uniqThetaNull" +"uniqThetaOrDefault" +"uniqThetaOrNull" +"uniqThetaResample" +"uniqThetaSimpleState" +"uniqThetaState" +"uniqThetaUnion" +"uniqUpTo" +"uniqUpToArgMax" +"uniqUpToArgMin" +"uniqUpToArray" +"uniqUpToDistinct" +"uniqUpToForEach" +"uniqUpToIf" +"uniqUpToMap" +"uniqUpToMerge" +"uniqUpToNull" +"uniqUpToOrDefault" +"uniqUpToOrNull" +"uniqUpToResample" +"uniqUpToSimpleState" +"uniqUpToState" +"upper" +"upperUTF8" +"uptime" +"user" +"validateNestedArraySizes" +"varPop" +"varPopArgMax" +"varPopArgMin" +"varPopArray" +"varPopDistinct" +"varPopForEach" +"varPopIf" +"varPopMap" +"varPopMerge" +"varPopNull" +"varPopOrDefault" +"varPopOrNull" +"varPopResample" +"varPopSimpleState" +"varPopStable" +"varPopStableArgMax" +"varPopStableArgMin" +"varPopStableArray" +"varPopStableDistinct" +"varPopStableForEach" +"varPopStableIf" +"varPopStableMap" +"varPopStableMerge" +"varPopStableNull" +"varPopStableOrDefault" +"varPopStableOrNull" +"varPopStableResample" +"varPopStableSimpleState" +"varPopStableState" +"varPopState" +"varSamp" +"varSampArgMax" +"varSampArgMin" +"varSampArray" +"varSampDistinct" +"varSampForEach" +"varSampIf" +"varSampMap" +"varSampMerge" +"varSampNull" +"varSampOrDefault" +"varSampOrNull" +"varSampResample" +"varSampSimpleState" +"varSampStable" +"varSampStableArgMax" +"varSampStableArgMin" +"varSampStableArray" +"varSampStableDistinct" +"varSampStableForEach" +"varSampStableIf" +"varSampStableMap" +"varSampStableMerge" +"varSampStableNull" +"varSampStableOrDefault" +"varSampStableOrNull" +"varSampStableResample" +"varSampStableSimpleState" +"varSampStableState" +"varSampState" +"variantElement" +"variantType" +"vectorDifference" +"vectorSum" +"version" +"visibleWidth" +"visitParamExtractBool" +"visitParamExtractFloat" +"visitParamExtractInt" +"visitParamExtractRaw" +"visitParamExtractString" +"visitParamExtractUInt" +"visitParamHas" +"week" +"welchTTest" +"welchTTestArgMax" +"welchTTestArgMin" +"welchTTestArray" +"welchTTestDistinct" +"welchTTestForEach" +"welchTTestIf" +"welchTTestMap" +"welchTTestMerge" +"welchTTestNull" +"welchTTestOrDefault" +"welchTTestOrNull" +"welchTTestResample" +"welchTTestSimpleState" +"welchTTestState" +"widthBucket" +"width_bucket" +"windowFunnel" +"windowFunnelArgMax" +"windowFunnelArgMin" +"windowFunnelArray" +"windowFunnelDistinct" +"windowFunnelForEach" +"windowFunnelIf" +"windowFunnelMap" +"windowFunnelMerge" +"windowFunnelNull" +"windowFunnelOrDefault" +"windowFunnelOrNull" +"windowFunnelResample" +"windowFunnelSimpleState" +"windowFunnelState" +"windowID" +"wkt" +"wordShingleMinHash" +"wordShingleMinHashArg" +"wordShingleMinHashArgCaseInsensitive" +"wordShingleMinHashArgCaseInsensitiveUTF8" +"wordShingleMinHashArgUTF8" +"wordShingleMinHashCaseInsensitive" +"wordShingleMinHashCaseInsensitiveUTF8" +"wordShingleMinHashUTF8" +"wordShingleSimHash" +"wordShingleSimHashCaseInsensitive" +"wordShingleSimHashCaseInsensitiveUTF8" +"wordShingleSimHashUTF8" +"wyHash64" +"xor" +"xxHash32" +"xxHash64" +"xxh3" +"yandexConsistentHash" +"yearweek" +"yesterday" +"zookeeperSessionUptime" diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index 6f2a88c22fa..e562595fb67 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -126,6 +126,7 @@ "JSONHas" "JSONKey" "JSONLength" +"JSONMergePatch" "JSONType" "JSON_ARRAY_LENGTH" "JSON_EXISTS" @@ -227,6 +228,8 @@ "UTC_timestamp" "UUIDNumToString" "UUIDStringToNum" +"UUIDToNum" +"UUIDv7ToDateTime" "VAR_POP" "VAR_POPArgMax" "VAR_POPArgMin" @@ -263,6 +266,7 @@ "YYYYMMDDhhmmssToDateTime" "YYYYMMDDhhmmssToDateTime64" "_CAST" +"__actionName" "__bitBoolMaskAnd" "__bitBoolMaskOr" "__bitSwapLastTwo" @@ -660,6 +664,8 @@ "base58Encode" "base64Decode" "base64Encode" +"base64URLDecode" +"base64URLEncode" "basename" "bin" "bitAnd" @@ -744,8 +750,15 @@ "cbrt" "ceil" "ceiling" +"changeDay" +"changeHour" +"changeMinute" +"changeMonth" +"changeSecond" +"changeYear" "char" "cityHash64" +"clamp" "coalesce" "concat" "concatAssumeInjective" @@ -970,6 +983,7 @@ "current_date" "current_schemas" "current_timestamp" +"current_user" "cutFragment" "cutIPv6" "cutQueryString" @@ -988,7 +1002,9 @@ "dateDiff" "dateName" "dateTime64ToSnowflake" +"dateTime64ToSnowflakeID" "dateTimeToSnowflake" +"dateTimeToSnowflakeID" "dateTrunc" "date_diff" "decodeHTMLComponent" @@ -1032,6 +1048,21 @@ "deltaSumTimestampSimpleState" "deltaSumTimestampState" "demangle" +"denseRank" +"denseRankArgMax" +"denseRankArgMin" +"denseRankArray" +"denseRankDistinct" +"denseRankForEach" +"denseRankIf" +"denseRankMap" +"denseRankMerge" +"denseRankNull" +"denseRankOrDefault" +"denseRankOrNull" +"denseRankResample" +"denseRankSimpleState" +"denseRankState" "dense_rank" "dense_rankArgMax" "dense_rankArgMin" @@ -1108,8 +1139,11 @@ "domainWithoutWWWRFC" "dotProduct" "dumpColumnStructure" +"dynamicElement" +"dynamicType" "e" "editDistance" +"editDistanceUTF8" "empty" "emptyArrayDate" "emptyArrayDateTime" @@ -1334,14 +1368,17 @@ "gccMurmurHash" "gcd" "generateRandomStructure" +"generateSnowflakeID" "generateULID" "generateUUIDv4" +"generateUUIDv7" "geoDistance" "geoToH3" "geoToS2" "geohashDecode" "geohashEncode" "geohashesInBox" +"getClientHTTPHeader" "getMacro" "getOSKernelVersion" "getServerPort" @@ -1589,6 +1626,20 @@ "groupBitmapXorSimpleState" "groupBitmapXorState" "groupConcat" +"groupConcatArgMax" +"groupConcatArgMin" +"groupConcatArray" +"groupConcatDistinct" +"groupConcatForEach" +"groupConcatIf" +"groupConcatMap" +"groupConcatMerge" +"groupConcatNull" +"groupConcatOrDefault" +"groupConcatOrNull" +"groupConcatResample" +"groupConcatSimpleState" +"groupConcatState" "groupUniqArray" "groupUniqArrayArgMax" "groupUniqArrayArgMin" @@ -1604,6 +1655,21 @@ "groupUniqArrayResample" "groupUniqArraySimpleState" "groupUniqArrayState" +"group_concat" +"group_concatArgMax" +"group_concatArgMin" +"group_concatArray" +"group_concatDistinct" +"group_concatForEach" +"group_concatIf" +"group_concatMap" +"group_concatMerge" +"group_concatNull" +"group_concatOrDefault" +"group_concatOrNull" +"group_concatResample" +"group_concatSimpleState" +"group_concatState" "h3CellAreaM2" "h3CellAreaRads2" "h3Distance" @@ -1660,6 +1726,8 @@ "hasTokenCaseInsensitiveOrNull" "hasTokenOrNull" "hex" +"hilbertDecode" +"hilbertEncode" "histogram" "histogramArgMax" "histogramArgMin" @@ -1881,6 +1949,7 @@ "less" "lessOrEquals" "levenshteinDistance" +"levenshteinDistanceUTF8" "lgamma" "like" "ln" @@ -2498,10 +2567,44 @@ "parseDateTimeInJodaSyntaxOrZero" "parseDateTimeOrNull" "parseDateTimeOrZero" +"parseReadableSize" +"parseReadableSizeOrNull" +"parseReadableSizeOrZero" "parseTimeDelta" +"partitionID" "partitionId" "path" "pathFull" +"percentRank" +"percentRankArgMax" +"percentRankArgMin" +"percentRankArray" +"percentRankDistinct" +"percentRankForEach" +"percentRankIf" +"percentRankMap" +"percentRankMerge" +"percentRankNull" +"percentRankOrDefault" +"percentRankOrNull" +"percentRankResample" +"percentRankSimpleState" +"percentRankState" +"percent_rank" +"percent_rankArgMax" +"percent_rankArgMin" +"percent_rankArray" +"percent_rankDistinct" +"percent_rankForEach" +"percent_rankIf" +"percent_rankMap" +"percent_rankMerge" +"percent_rankNull" +"percent_rankOrDefault" +"percent_rankOrNull" +"percent_rankResample" +"percent_rankSimpleState" +"percent_rankState" "pi" "plus" "pmod" @@ -2533,6 +2636,7 @@ "positive_modulo" "pow" "power" +"printf" "proportionsZTest" "protocol" "punycodeDecode" @@ -3103,6 +3207,8 @@ "rankResample" "rankSimpleState" "rankState" +"readWKTLineString" +"readWKTMultiLineString" "readWKTMultiPolygon" "readWKTPoint" "readWKTPolygon" @@ -3340,6 +3446,8 @@ "skewSampState" "sleep" "sleepEachRow" +"snowflakeIDToDateTime" +"snowflakeIDToDateTime64" "snowflakeToDateTime" "snowflakeToDateTime64" "soundex" @@ -3902,6 +4010,7 @@ "truncate" "tryBase58Decode" "tryBase64Decode" +"tryBase64URLDecode" "tryDecrypt" "tryIdnaEncode" "tryPunycodeDecode" @@ -3923,6 +4032,7 @@ "tupleModuloByNumber" "tupleMultiply" "tupleMultiplyByNumber" +"tupleNames" "tupleNegate" "tuplePlus" "tupleToNameValuePairs" diff --git a/tests/fuzz/dictionaries/key_words.dict b/tests/fuzz/dictionaries/keywords.dict similarity index 95% rename from tests/fuzz/dictionaries/key_words.dict rename to tests/fuzz/dictionaries/keywords.dict index db517a2382c..abaaf9e53b5 100644 --- a/tests/fuzz/dictionaries/key_words.dict +++ b/tests/fuzz/dictionaries/keywords.dict @@ -3,7 +3,7 @@ "ADD CONSTRAINT" "ADD INDEX" "ADD PROJECTION" -"ADD STATISTIC" +"ADD STATISTICS" "ADMIN OPTION FOR" "AFTER" "ALGORITHM" @@ -76,7 +76,7 @@ "CLEAR COLUMN" "CLEAR INDEX" "CLEAR PROJECTION" -"CLEAR STATISTIC" +"CLEAR STATISTICS" "CLUSTER" "CLUSTERS" "CN" @@ -110,6 +110,8 @@ "CURRENTUSER" "CURRENT_USER" "D" +"DATA" +"DATA INNER UUID" "DATABASE" "DATABASES" "DATE" @@ -147,7 +149,7 @@ "DROP PART" "DROP PARTITION" "DROP PROJECTION" -"DROP STATISTIC" +"DROP STATISTICS" "DROP TABLE" "DROP TEMPORARY TABLE" "ELSE" @@ -247,6 +249,7 @@ "IS NULL" "IS_OBJECT_ID" "JOIN" +"JWT" "KERBEROS" "KEY" "KEY BY" @@ -277,13 +280,15 @@ "MATERIALIZE COLUMN" "MATERIALIZE INDEX" "MATERIALIZE PROJECTION" -"MATERIALIZE STATISTIC" +"MATERIALIZE STATISTICS" "MATERIALIZE TTL" "MATERIALIZED" "MAX" "MCS" "MEMORY" "MERGES" +"METRICS" +"METRICS INNER UUID" "MI" "MICROSECOND" "MICROSECONDS" @@ -297,12 +302,14 @@ "MODIFY" "MODIFY COLUMN" "MODIFY COMMENT" +"MODIFY DEFINER" "MODIFY ORDER BY" "MODIFY QUERY" "MODIFY REFRESH" "MODIFY SAMPLE BY" "MODIFY SETTING" "MODIFY SQL SECURITY" +"MODIFY STATISTICS" "MODIFY TTL" "MONTH" "MONTHS" @@ -373,6 +380,7 @@ "Protobuf" "Q" "QQ" +"QUALIFY" "QUARTER" "QUARTERS" "QUERY" @@ -384,6 +392,7 @@ "READONLY" "REALM" "RECOMPRESS" +"RECURSIVE" "REFERENCES" "REFRESH" "REGEXP" @@ -415,6 +424,7 @@ "SALT" "SAMPLE" "SAMPLE BY" +"SAN" "SCHEME" "SECOND" "SECONDS" @@ -460,7 +470,8 @@ "SS" "SSH_KEY" "SSL_CERTIFICATE" -"STATISTIC" +"START TRANSACTION" +"STATISTICS" "STEP" "STORAGE" "STRICT" @@ -475,6 +486,8 @@ "TABLE" "TABLE OVERRIDE" "TABLES" +"TAGS" +"TAGS INNER UUID" "TEMPORARY" "TEMPORARY TABLE" "TEST" @@ -529,6 +542,7 @@ "WITH NAME" "WITH REPLACE OPTION" "WITH TIES" +"WITH_ITEMINDEX" "WK" "WRITABLE" "WW" @@ -540,4 +554,3 @@ "bagexpansion" "base_backup" "cluster_host_ids" -"with_itemindex" From b5f7875f574b640dc00d42dea5e721058e9a10fc Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 13 Aug 2024 14:04:04 +0000 Subject: [PATCH 631/644] Remove odd new line --- tests/queries/0_stateless/00652_mergetree_mutations.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/00652_mergetree_mutations.sh b/tests/queries/0_stateless/00652_mergetree_mutations.sh index 3b0966dd2c3..6be0ebf882f 100755 --- a/tests/queries/0_stateless/00652_mergetree_mutations.sh +++ b/tests/queries/0_stateless/00652_mergetree_mutations.sh @@ -89,4 +89,3 @@ done ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, is_done FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner' ORDER BY mutation_id" ${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner" - From 79e055783931bd544897ec23eb07b7c7c9b09a69 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 13 Aug 2024 14:18:48 +0000 Subject: [PATCH 632/644] Write a script to automatically update the dict --- tests/fuzz/README.md | 23 - tests/fuzz/all.dict | 2377 +++++++------ tests/fuzz/dictionaries/datatypes.dict | 4418 +----------------------- tests/fuzz/dictionaries/functions.dict | 4283 ----------------------- tests/fuzz/update_dict.sh | 20 + 5 files changed, 1389 insertions(+), 9732 deletions(-) delete mode 100644 tests/fuzz/README.md create mode 100755 tests/fuzz/update_dict.sh diff --git a/tests/fuzz/README.md b/tests/fuzz/README.md deleted file mode 100644 index 576ad66ed93..00000000000 --- a/tests/fuzz/README.md +++ /dev/null @@ -1,23 +0,0 @@ -The list of functions generated via the following query - -``` - clickhouse client -q "SELECT * FROM (SELECT DISTINCT concat('\"', name, '\"') as res FROM system.functions ORDER BY name UNION ALL SELECT concat('\"', a.name, b.name, '\"') as res FROM system.functions as a CROSS JOIN system.aggregate_function_combinators as b WHERE a.is_aggregate = 1) ORDER BY res" > functions.dict -``` - -The list of datatypes generated via the following query: - -``` - clickhouse client -q "SELECT DISTINCT concat('\"', name, '\"') as res FROM system.data_type_families ORDER BY name" > datatypes.dict -``` - -The list of keywords generated via the following query: - -``` - clickhouse client -q "SELECT DISTINCT concat('\"', keyword, '\"') as res FROM system.keywords ORDER BY keyword" > keywords.dict -``` - -Then merge all dictionaries into one (all.dict) - -``` - cat ./dictionaries/* | LC_ALL=C sort | uniq > all.dict -``` \ No newline at end of file diff --git a/tests/fuzz/all.dict b/tests/fuzz/all.dict index 1c3c657d6b0..30af3746fca 100644 --- a/tests/fuzz/all.dict +++ b/tests/fuzz/all.dict @@ -1,15 +1,971 @@ -"abs" -"accurateCast" -"accurateCastOrDefault" -"accurateCastOrNull" -"acos" -"acosh" "ADD COLUMN" "ADD CONSTRAINT" "ADD INDEX" "ADD PROJECTION" "ADD STATISTICS" "ADD" +"ADMIN OPTION FOR" +"AFTER" +"ALGORITHM" +"ALIAS" +"ALL" +"ALLOWED_LATENESS" +"ALTER COLUMN" +"ALTER DATABASE" +"ALTER LIVE VIEW" +"ALTER POLICY" +"ALTER PROFILE" +"ALTER QUOTA" +"ALTER ROLE" +"ALTER ROW POLICY" +"ALTER SETTINGS PROFILE" +"ALTER TABLE" +"ALTER TEMPORARY TABLE" +"ALTER USER" +"ALTER" +"AND STDOUT" +"AND" +"ANTI" +"ANY" +"APPEND" +"APPLY DELETED MASK" +"APPLY" +"ARRAY JOIN" +"AS" +"ASC" +"ASCENDING" +"ASOF" +"ASSUME" +"AST" +"ASYNC" +"ATTACH PART" +"ATTACH PARTITION" +"ATTACH POLICY" +"ATTACH PROFILE" +"ATTACH QUOTA" +"ATTACH ROLE" +"ATTACH ROW POLICY" +"ATTACH SETTINGS PROFILE" +"ATTACH USER" +"ATTACH" +"AUTO_INCREMENT" +"AZURE" +"AggregateFunction" +"Array" +"BACKUP" +"BCRYPT_HASH" +"BCRYPT_PASSWORD" +"BEGIN TRANSACTION" +"BETWEEN" +"BIDIRECTIONAL" +"BIGINT SIGNED" +"BIGINT UNSIGNED" +"BIGINT" +"BINARY LARGE OBJECT" +"BINARY VARYING" +"BINARY" +"BIT" +"BIT_AND" +"BIT_ANDArgMax" +"BIT_ANDArgMin" +"BIT_ANDArray" +"BIT_ANDDistinct" +"BIT_ANDForEach" +"BIT_ANDIf" +"BIT_ANDMap" +"BIT_ANDMerge" +"BIT_ANDNull" +"BIT_ANDOrDefault" +"BIT_ANDOrNull" +"BIT_ANDResample" +"BIT_ANDSimpleState" +"BIT_ANDState" +"BIT_OR" +"BIT_ORArgMax" +"BIT_ORArgMin" +"BIT_ORArray" +"BIT_ORDistinct" +"BIT_ORForEach" +"BIT_ORIf" +"BIT_ORMap" +"BIT_ORMerge" +"BIT_ORNull" +"BIT_OROrDefault" +"BIT_OROrNull" +"BIT_ORResample" +"BIT_ORSimpleState" +"BIT_ORState" +"BIT_XOR" +"BIT_XORArgMax" +"BIT_XORArgMin" +"BIT_XORArray" +"BIT_XORDistinct" +"BIT_XORForEach" +"BIT_XORIf" +"BIT_XORMap" +"BIT_XORMerge" +"BIT_XORNull" +"BIT_XOROrDefault" +"BIT_XOROrNull" +"BIT_XORResample" +"BIT_XORSimpleState" +"BIT_XORState" +"BLAKE3" +"BLOB" +"BOTH" +"BY" +"BYTE" +"BYTEA" +"Bool" +"CASCADE" +"CASE" +"CAST" +"CHANGE" +"CHANGEABLE_IN_READONLY" +"CHANGED" +"CHAR LARGE OBJECT" +"CHAR VARYING" +"CHAR" +"CHARACTER LARGE OBJECT" +"CHARACTER VARYING" +"CHARACTER" +"CHARACTER_LENGTH" +"CHAR_LENGTH" +"CHECK ALL TABLES" +"CHECK TABLE" +"CHECK" +"CLEANUP" +"CLEAR COLUMN" +"CLEAR INDEX" +"CLEAR PROJECTION" +"CLEAR STATISTICS" +"CLOB" +"CLUSTER" +"CLUSTERS" +"CN" +"CODEC" +"COLLATE" +"COLUMN" +"COLUMNS" +"COMMENT COLUMN" +"COMMENT" +"COMMIT" +"COMPRESSION" +"CONST" +"CONSTRAINT" +"COVAR_POP" +"COVAR_POPArgMax" +"COVAR_POPArgMin" +"COVAR_POPArray" +"COVAR_POPDistinct" +"COVAR_POPForEach" +"COVAR_POPIf" +"COVAR_POPMap" +"COVAR_POPMerge" +"COVAR_POPNull" +"COVAR_POPOrDefault" +"COVAR_POPOrNull" +"COVAR_POPResample" +"COVAR_POPSimpleState" +"COVAR_POPState" +"COVAR_SAMP" +"COVAR_SAMPArgMax" +"COVAR_SAMPArgMin" +"COVAR_SAMPArray" +"COVAR_SAMPDistinct" +"COVAR_SAMPForEach" +"COVAR_SAMPIf" +"COVAR_SAMPMap" +"COVAR_SAMPMerge" +"COVAR_SAMPNull" +"COVAR_SAMPOrDefault" +"COVAR_SAMPOrNull" +"COVAR_SAMPResample" +"COVAR_SAMPSimpleState" +"COVAR_SAMPState" +"CRC32" +"CRC32IEEE" +"CRC64" +"CREATE POLICY" +"CREATE PROFILE" +"CREATE QUOTA" +"CREATE ROLE" +"CREATE ROW POLICY" +"CREATE SETTINGS PROFILE" +"CREATE TABLE" +"CREATE TEMPORARY TABLE" +"CREATE USER" +"CREATE" +"CROSS" +"CUBE" +"CURRENT GRANTS" +"CURRENT QUOTA" +"CURRENT ROLES" +"CURRENT ROW" +"CURRENT TRANSACTION" +"CURRENTUSER" +"CURRENT_USER" +"D" +"DATA INNER UUID" +"DATA" +"DATABASE" +"DATABASES" +"DATE" +"DATEADD" +"DATEDIFF" +"DATESUB" +"DATE_ADD" +"DATE_DIFF" +"DATE_FORMAT" +"DATE_SUB" +"DATE_TRUNC" +"DAY" +"DAYOFMONTH" +"DAYOFWEEK" +"DAYOFYEAR" +"DAYS" +"DD" +"DEC" +"DEDUPLICATE" +"DEFAULT DATABASE" +"DEFAULT ROLE" +"DEFAULT" +"DEFINER" +"DELETE WHERE" +"DELETE" +"DEPENDS ON" +"DESC" +"DESCENDING" +"DESCRIBE" +"DETACH PART" +"DETACH PARTITION" +"DETACH" +"DICTIONARIES" +"DICTIONARY" +"DISK" +"DISTINCT ON" +"DISTINCT" +"DIV" +"DOUBLE PRECISION" +"DOUBLE" +"DOUBLE_SHA1_HASH" +"DOUBLE_SHA1_PASSWORD" +"DROP COLUMN" +"DROP CONSTRAINT" +"DROP DEFAULT" +"DROP DETACHED PART" +"DROP DETACHED PARTITION" +"DROP INDEX" +"DROP PART" +"DROP PARTITION" +"DROP PROJECTION" +"DROP STATISTICS" +"DROP TABLE" +"DROP TEMPORARY TABLE" +"DROP" +"Date" +"Date32" +"DateTime" +"DateTime32" +"DateTime64" +"Decimal" +"Decimal128" +"Decimal256" +"Decimal32" +"Decimal64" +"Dynamic" +"ELSE" +"EMPTY AS" +"EMPTY" +"ENABLED ROLES" +"END" +"ENFORCED" +"ENGINE" +"ENUM" +"EPHEMERAL SEQUENTIAL" +"EPHEMERAL" +"ESTIMATE" +"EVENT" +"EVENTS" +"EVERY" +"EXCEPT DATABASE" +"EXCEPT DATABASES" +"EXCEPT TABLE" +"EXCEPT TABLES" +"EXCEPT" +"EXCHANGE DICTIONARIES" +"EXCHANGE TABLES" +"EXISTS" +"EXPLAIN" +"EXPRESSION" +"EXTENDED" +"EXTERNAL DDL FROM" +"EXTRACT" +"Enum" +"Enum16" +"Enum8" +"FALSE" +"FETCH PART" +"FETCH PARTITION" +"FETCH" +"FIELDS" +"FILE" +"FILESYSTEM CACHE" +"FILESYSTEM CACHES" +"FILTER" +"FINAL" +"FIRST" +"FIXED" +"FLOAT" +"FOLLOWING" +"FOR" +"FOREIGN KEY" +"FOREIGN" +"FORGET PARTITION" +"FORMAT" +"FORMAT_BYTES" +"FQDN" +"FREEZE" +"FROM INFILE" +"FROM SHARD" +"FROM" +"FROM_BASE64" +"FROM_DAYS" +"FROM_UNIXTIME" +"FULL" +"FULLTEXT" +"FUNCTION" +"FixedString" +"Float32" +"Float64" +"ForEach" +"GEOMETRY" +"GLOBAL IN" +"GLOBAL NOT IN" +"GLOBAL" +"GRANT OPTION FOR" +"GRANT" +"GRANTEES" +"GRANULARITY" +"GROUP BY" +"GROUPING SETS" +"GROUPS" +"H" +"HASH" +"HAVING" +"HDFS" +"HH" +"HIERARCHICAL" +"HOST" +"HOUR" +"HOURS" +"HTTP" +"ID" +"IDENTIFIED" +"IF EMPTY" +"IF EXISTS" +"IF NOT EXISTS" +"IGNORE NULLS" +"ILIKE" +"IN PARTITION" +"IN" +"INDEX" +"INDEXES" +"INDICES" +"INET4" +"INET6" +"INET6_ATON" +"INET6_NTOA" +"INET_ATON" +"INET_NTOA" +"INHERIT" +"INJECTIVE" +"INNER" +"INSERT INTO" +"INT SIGNED" +"INT UNSIGNED" +"INT" +"INT1 SIGNED" +"INT1 UNSIGNED" +"INT1" +"INTEGER SIGNED" +"INTEGER UNSIGNED" +"INTEGER" +"INTERPOLATE" +"INTERSECT" +"INTERVAL" +"INTO OUTFILE" +"INVISIBLE" +"INVOKER" +"IP" +"IPv4" +"IPv4CIDRToRange" +"IPv4NumToString" +"IPv4NumToStringClassC" +"IPv4StringToNum" +"IPv4StringToNumOrDefault" +"IPv4StringToNumOrNull" +"IPv4ToIPv6" +"IPv6" +"IPv6CIDRToRange" +"IPv6NumToString" +"IPv6StringToNum" +"IPv6StringToNumOrDefault" +"IPv6StringToNumOrNull" +"IS NOT DISTINCT FROM" +"IS NOT NULL" +"IS NULL" +"IS_OBJECT_ID" +"Int128" +"Int16" +"Int256" +"Int32" +"Int64" +"Int8" +"IntervalDay" +"IntervalHour" +"IntervalMicrosecond" +"IntervalMillisecond" +"IntervalMinute" +"IntervalMonth" +"IntervalNanosecond" +"IntervalQuarter" +"IntervalSecond" +"IntervalWeek" +"IntervalYear" +"JOIN" +"JSON" +"JSONArrayLength" +"JSONExtract" +"JSONExtractArrayRaw" +"JSONExtractBool" +"JSONExtractFloat" +"JSONExtractInt" +"JSONExtractKeys" +"JSONExtractKeysAndValues" +"JSONExtractKeysAndValuesRaw" +"JSONExtractRaw" +"JSONExtractString" +"JSONExtractUInt" +"JSONHas" +"JSONKey" +"JSONLength" +"JSONMergePatch" +"JSONType" +"JSON_ARRAY_LENGTH" +"JSON_EXISTS" +"JSON_QUERY" +"JSON_VALUE" +"JWT" +"KERBEROS" +"KEY BY" +"KEY" +"KEYED BY" +"KEYS" +"KILL" +"KIND" +"L1Distance" +"L1Norm" +"L1Normalize" +"L2Distance" +"L2Norm" +"L2Normalize" +"L2SquaredDistance" +"L2SquaredNorm" +"LARGE OBJECT" +"LAST" +"LAST_DAY" +"LAYOUT" +"LDAP" +"LEADING" +"LEFT ARRAY JOIN" +"LEFT" +"LESS THAN" +"LEVEL" +"LIFETIME" +"LIGHTWEIGHT" +"LIKE" +"LIMIT" +"LINEAR" +"LIST" +"LIVE" +"LOCAL" +"LONGBLOB" +"LONGTEXT" +"LTRIM" +"LineString" +"LinfDistance" +"LinfNorm" +"LinfNormalize" +"LowCardinality" +"LpDistance" +"LpNorm" +"LpNormalize" +"M" +"MACNumToString" +"MACStringToNum" +"MACStringToOUI" +"MAP_FROM_ARRAYS" +"MATCH" +"MATERIALIZE COLUMN" +"MATERIALIZE INDEX" +"MATERIALIZE PROJECTION" +"MATERIALIZE STATISTICS" +"MATERIALIZE TTL" +"MATERIALIZE" +"MATERIALIZED" +"MAX" +"MCS" +"MD4" +"MD5" +"MEDIUMBLOB" +"MEDIUMINT SIGNED" +"MEDIUMINT UNSIGNED" +"MEDIUMINT" +"MEDIUMTEXT" +"MEMORY" +"MERGES" +"METRICS INNER UUID" +"METRICS" +"MI" +"MICROSECOND" +"MICROSECONDS" +"MILLISECOND" +"MILLISECONDS" +"MIN" +"MINUTE" +"MINUTES" +"MM" +"MOD" +"MODIFY COLUMN" +"MODIFY COMMENT" +"MODIFY DEFINER" +"MODIFY ORDER BY" +"MODIFY QUERY" +"MODIFY REFRESH" +"MODIFY SAMPLE BY" +"MODIFY SETTING" +"MODIFY SQL SECURITY" +"MODIFY STATISTICS" +"MODIFY TTL" +"MODIFY" +"MONTH" +"MONTHS" +"MOVE PART" +"MOVE PARTITION" +"MOVE" +"MS" +"MUTATION" +"Map" +"Merge" +"MultiLineString" +"MultiPolygon" +"N" +"NAME" +"NAMED COLLECTION" +"NANOSECOND" +"NANOSECONDS" +"NATIONAL CHAR VARYING" +"NATIONAL CHAR" +"NATIONAL CHARACTER LARGE OBJECT" +"NATIONAL CHARACTER VARYING" +"NATIONAL CHARACTER" +"NCHAR LARGE OBJECT" +"NCHAR VARYING" +"NCHAR" +"NEXT" +"NO ACTION" +"NO DELAY" +"NO LIMITS" +"NONE" +"NOT BETWEEN" +"NOT IDENTIFIED" +"NOT ILIKE" +"NOT IN" +"NOT KEYED" +"NOT LIKE" +"NOT OVERRIDABLE" +"NOT" +"NO_PASSWORD" +"NS" +"NULL" +"NULLS" +"NUMERIC" +"NVARCHAR" +"Nested" +"Nothing" +"Null" +"Nullable" +"OCTET_LENGTH" +"OFFSET" +"ON DELETE" +"ON UPDATE" +"ON VOLUME" +"ON" +"ONLY" +"OPTIMIZE TABLE" +"OR REPLACE" +"OR" +"ORDER BY" +"OUTER" +"OVER" +"OVERRIDABLE" +"Object" +"PART" +"PARTIAL" +"PARTITION BY" +"PARTITION" +"PARTITIONS" +"PART_MOVE_TO_SHARD" +"PASTE" +"PERIODIC REFRESH" +"PERMANENTLY" +"PERMISSIVE" +"PERSISTENT SEQUENTIAL" +"PERSISTENT" +"PIPELINE" +"PLAINTEXT_PASSWORD" +"PLAN" +"POPULATE" +"PRECEDING" +"PRECISION" +"PREWHERE" +"PRIMARY KEY" +"PRIMARY" +"PROFILE" +"PROJECTION" +"PULL" +"Point" +"Polygon" +"Protobuf" +"Q" +"QQ" +"QUALIFY" +"QUARTER" +"QUARTERS" +"QUERY TREE" +"QUERY" +"QUOTA" +"RANDOMIZE FOR" +"RANDOMIZED" +"RANGE" +"READONLY" +"REAL" +"REALM" +"RECOMPRESS" +"RECURSIVE" +"REFERENCES" +"REFRESH" +"REGEXP" +"REGEXP_EXTRACT" +"REGEXP_MATCHES" +"REGEXP_REPLACE" +"REMOVE SAMPLE BY" +"REMOVE TTL" +"REMOVE" +"RENAME COLUMN" +"RENAME DATABASE" +"RENAME DICTIONARY" +"RENAME TABLE" +"RENAME TO" +"RENAME" +"REPLACE PARTITION" +"REPLACE" +"RESET SETTING" +"RESPECT NULLS" +"RESTORE" +"RESTRICT" +"RESTRICTIVE" +"RESUME" +"REVOKE" +"RIGHT" +"ROLLBACK" +"ROLLUP" +"ROW" +"ROWS" +"RTRIM" +"Resample" +"Ring" +"S" +"S3" +"SALT" +"SAMPLE BY" +"SAMPLE" +"SAN" +"SCHEMA" +"SCHEME" +"SECOND" +"SECONDS" +"SELECT" +"SEMI" +"SERVER" +"SET DEFAULT ROLE" +"SET DEFAULT" +"SET FAKE TIME" +"SET NULL" +"SET ROLE DEFAULT" +"SET ROLE" +"SET TRANSACTION SNAPSHOT" +"SET" +"SETTINGS" +"SHA1" +"SHA224" +"SHA256" +"SHA256_HASH" +"SHA256_PASSWORD" +"SHA384" +"SHA512" +"SHA512_256" +"SHOW ACCESS" +"SHOW CREATE" +"SHOW ENGINES" +"SHOW FUNCTIONS" +"SHOW GRANTS" +"SHOW PRIVILEGES" +"SHOW PROCESSLIST" +"SHOW SETTING" +"SHOW" +"SIGNED" +"SIMPLE" +"SINGLE" +"SMALLINT SIGNED" +"SMALLINT UNSIGNED" +"SMALLINT" +"SOURCE" +"SPATIAL" +"SQL SECURITY" +"SQL_TSI_DAY" +"SQL_TSI_HOUR" +"SQL_TSI_MICROSECOND" +"SQL_TSI_MILLISECOND" +"SQL_TSI_MINUTE" +"SQL_TSI_MONTH" +"SQL_TSI_NANOSECOND" +"SQL_TSI_QUARTER" +"SQL_TSI_SECOND" +"SQL_TSI_WEEK" +"SQL_TSI_YEAR" +"SS" +"SSH_KEY" +"SSL_CERTIFICATE" +"START TRANSACTION" +"STATISTICS" +"STD" +"STDArgMax" +"STDArgMin" +"STDArray" +"STDDEV_POP" +"STDDEV_POPArgMax" +"STDDEV_POPArgMin" +"STDDEV_POPArray" +"STDDEV_POPDistinct" +"STDDEV_POPForEach" +"STDDEV_POPIf" +"STDDEV_POPMap" +"STDDEV_POPMerge" +"STDDEV_POPNull" +"STDDEV_POPOrDefault" +"STDDEV_POPOrNull" +"STDDEV_POPResample" +"STDDEV_POPSimpleState" +"STDDEV_POPState" +"STDDEV_SAMP" +"STDDEV_SAMPArgMax" +"STDDEV_SAMPArgMin" +"STDDEV_SAMPArray" +"STDDEV_SAMPDistinct" +"STDDEV_SAMPForEach" +"STDDEV_SAMPIf" +"STDDEV_SAMPMap" +"STDDEV_SAMPMerge" +"STDDEV_SAMPNull" +"STDDEV_SAMPOrDefault" +"STDDEV_SAMPOrNull" +"STDDEV_SAMPResample" +"STDDEV_SAMPSimpleState" +"STDDEV_SAMPState" +"STDDistinct" +"STDForEach" +"STDIf" +"STDMap" +"STDMerge" +"STDNull" +"STDOrDefault" +"STDOrNull" +"STDResample" +"STDSimpleState" +"STDState" +"STEP" +"STORAGE" +"STRICT" +"STRICTLY_ASCENDING" +"SUBPARTITION BY" +"SUBPARTITION" +"SUBPARTITIONS" +"SUBSTRING" +"SUBSTRING_INDEX" +"SUSPEND" +"SVG" +"SYNC" +"SYNTAX" +"SYSTEM" +"SimpleAggregateFunction" +"State" +"String" +"TABLE OVERRIDE" +"TABLE" +"TABLES" +"TAGS INNER UUID" +"TAGS" +"TEMPORARY TABLE" +"TEMPORARY" +"TEST" +"TEXT" +"THEN" +"TIME" +"TIMESTAMP" +"TIMESTAMPADD" +"TIMESTAMPDIFF" +"TIMESTAMPSUB" +"TIMESTAMP_ADD" +"TIMESTAMP_DIFF" +"TIMESTAMP_SUB" +"TINYBLOB" +"TINYINT SIGNED" +"TINYINT UNSIGNED" +"TINYINT" +"TINYTEXT" +"TO DISK" +"TO INNER UUID" +"TO SHARD" +"TO TABLE" +"TO VOLUME" +"TO" +"TOP" +"TOTALS" +"TO_BASE64" +"TO_DAYS" +"TO_UNIXTIME" +"TRACKING ONLY" +"TRAILING" +"TRANSACTION" +"TRIGGER" +"TRIM" +"TRUE" +"TRUNCATE" +"TTL" +"TYPE" +"TYPEOF" +"Tuple" +"UInt128" +"UInt16" +"UInt256" +"UInt32" +"UInt64" +"UInt8" +"ULIDStringToDateTime" +"UNBOUNDED" +"UNDROP" +"UNFREEZE" +"UNION" +"UNIQUE" +"UNSET FAKE TIME" +"UNSIGNED" +"UPDATE" +"URL" +"URLHash" +"URLHierarchy" +"URLPathHierarchy" +"USE" +"USING" +"UTCTimestamp" +"UTC_timestamp" +"UUID" +"UUIDNumToString" +"UUIDStringToNum" +"UUIDToNum" +"UUIDv7ToDateTime" +"VALID UNTIL" +"VALUES" +"VARBINARY" +"VARCHAR" +"VARCHAR2" +"VARYING" +"VAR_POP" +"VAR_POPArgMax" +"VAR_POPArgMin" +"VAR_POPArray" +"VAR_POPDistinct" +"VAR_POPForEach" +"VAR_POPIf" +"VAR_POPMap" +"VAR_POPMerge" +"VAR_POPNull" +"VAR_POPOrDefault" +"VAR_POPOrNull" +"VAR_POPResample" +"VAR_POPSimpleState" +"VAR_POPState" +"VAR_SAMP" +"VAR_SAMPArgMax" +"VAR_SAMPArgMin" +"VAR_SAMPArray" +"VAR_SAMPDistinct" +"VAR_SAMPForEach" +"VAR_SAMPIf" +"VAR_SAMPMap" +"VAR_SAMPMerge" +"VAR_SAMPNull" +"VAR_SAMPOrDefault" +"VAR_SAMPOrNull" +"VAR_SAMPResample" +"VAR_SAMPSimpleState" +"VAR_SAMPState" +"VIEW" +"VISIBLE" +"Variant" +"WATCH" +"WATERMARK" +"WEEK" +"WEEKS" +"WHEN" +"WHERE" +"WINDOW" +"WITH ADMIN OPTION" +"WITH CHECK" +"WITH FILL" +"WITH GRANT OPTION" +"WITH NAME" +"WITH REPLACE OPTION" +"WITH TIES" +"WITH" +"WITH_ITEMINDEX" +"WK" +"WRITABLE" +"WW" +"YEAR" +"YEARS" +"YY" +"YYYY" +"YYYYMMDDToDate" +"YYYYMMDDToDate32" +"YYYYMMDDhhmmssToDateTime" +"YYYYMMDDhhmmssToDateTime64" +"ZKPATH" +"_CAST" +"__actionName" +"__bitBoolMaskAnd" +"__bitBoolMaskOr" +"__bitSwapLastTwo" +"__bitWrapperFunc" +"__getScalar" +"__scalarSubqueryResult" +"abs" +"accurateCast" +"accurateCastOrDefault" +"accurateCastOrNull" +"acos" +"acosh" "addDate" "addDays" "addHours" @@ -20,19 +976,16 @@ "addMonths" "addNanoseconds" "addQuarters" -"addressToLine" -"addressToLineWithInlines" -"addressToSymbol" "addSeconds" "addTupleOfIntervals" "addWeeks" "addYears" -"ADMIN OPTION FOR" +"addressToLine" +"addressToLineWithInlines" +"addressToSymbol" "aes_decrypt_mysql" "aes_encrypt_mysql" -"AFTER" "age" -"AggregateFunction" "aggThrow" "aggThrowArgMax" "aggThrowArgMin" @@ -48,24 +1001,7 @@ "aggThrowResample" "aggThrowSimpleState" "aggThrowState" -"ALGORITHM" -"ALIAS" -"ALL" -"ALLOWED_LATENESS" "alphaTokens" -"ALTER COLUMN" -"ALTER DATABASE" -"ALTER LIVE VIEW" -"ALTER POLICY" -"ALTER PROFILE" -"ALTER QUOTA" -"ALTER ROLE" -"ALTER ROW POLICY" -"ALTER SETTINGS PROFILE" -"ALTER TABLE" -"ALTER TEMPORARY TABLE" -"ALTER USER" -"ALTER" "analysisOfVariance" "analysisOfVarianceArgMax" "analysisOfVarianceArgMin" @@ -81,8 +1017,6 @@ "analysisOfVarianceResample" "analysisOfVarianceSimpleState" "analysisOfVarianceState" -"AND STDOUT" -"AND" "and" "anova" "anovaArgMax" @@ -99,8 +1033,6 @@ "anovaResample" "anovaSimpleState" "anovaState" -"ANTI" -"ANY" "any" "anyArgMax" "anyArgMin" @@ -206,10 +1138,7 @@ "any_value_respect_nullsResample" "any_value_respect_nullsSimpleState" "any_value_respect_nullsState" -"APPEND" "appendTrailingCharIfAbsent" -"APPLY DELETED MASK" -"APPLY" "approx_top_count" "approx_top_countArgMax" "approx_top_countArgMin" @@ -285,11 +1214,9 @@ "argMinResample" "argMinSimpleState" "argMinState" -"ARRAY JOIN" -"Array" "array" -"arrayAll" "arrayAUC" +"arrayAll" "arrayAvg" "arrayCompact" "arrayConcat" @@ -382,31 +1309,13 @@ "array_concat_aggResample" "array_concat_aggSimpleState" "array_concat_aggState" -"AS" -"ASC" -"ASCENDING" "ascii" "asin" "asinh" -"ASOF" -"ASSUME" "assumeNotNull" -"AST" -"ASYNC" "atan" "atan2" "atanh" -"ATTACH PART" -"ATTACH PARTITION" -"ATTACH POLICY" -"ATTACH PROFILE" -"ATTACH QUOTA" -"ATTACH ROLE" -"ATTACH ROW POLICY" -"ATTACH SETTINGS PROFILE" -"ATTACH USER" -"ATTACH" -"AUTO_INCREMENT" "avg" "avgArgMax" "avgArgMin" @@ -437,8 +1346,6 @@ "avgWeightedResample" "avgWeightedSimpleState" "avgWeightedState" -"AZURE" -"BACKUP" "bagexpansion" "bar" "base58Decode" @@ -447,17 +1354,24 @@ "base64Encode" "base64URLDecode" "base64URLEncode" -"basename" "base_backup" -"BCRYPT_HASH" -"BCRYPT_PASSWORD" -"BEGIN TRANSACTION" -"BETWEEN" -"BIDIRECTIONAL" +"basename" "bin" "bitAnd" "bitCount" "bitHammingDistance" +"bitNot" +"bitOr" +"bitPositionsToArray" +"bitRotateLeft" +"bitRotateRight" +"bitShiftLeft" +"bitShiftRight" +"bitSlice" +"bitTest" +"bitTestAll" +"bitTestAny" +"bitXor" "bitmapAnd" "bitmapAndCardinality" "bitmapAndnot" @@ -479,68 +1393,11 @@ "bitmapXorCardinality" "bitmaskToArray" "bitmaskToList" -"bitNot" -"bitOr" -"bitPositionsToArray" -"bitRotateLeft" -"bitRotateRight" -"bitShiftLeft" -"bitShiftRight" -"bitSlice" -"bitTest" -"bitTestAll" -"bitTestAny" -"bitXor" -"BIT_AND" -"BIT_ANDArgMax" -"BIT_ANDArgMin" -"BIT_ANDArray" -"BIT_ANDDistinct" -"BIT_ANDForEach" -"BIT_ANDIf" -"BIT_ANDMap" -"BIT_ANDMerge" -"BIT_ANDNull" -"BIT_ANDOrDefault" -"BIT_ANDOrNull" -"BIT_ANDResample" -"BIT_ANDSimpleState" -"BIT_ANDState" -"BIT_OR" -"BIT_ORArgMax" -"BIT_ORArgMin" -"BIT_ORArray" -"BIT_ORDistinct" -"BIT_ORForEach" -"BIT_ORIf" -"BIT_ORMap" -"BIT_ORMerge" -"BIT_ORNull" -"BIT_OROrDefault" -"BIT_OROrNull" -"BIT_ORResample" -"BIT_ORSimpleState" -"BIT_ORState" -"BIT_XOR" -"BIT_XORArgMax" -"BIT_XORArgMin" -"BIT_XORArray" -"BIT_XORDistinct" -"BIT_XORForEach" -"BIT_XORIf" -"BIT_XORMap" -"BIT_XORMerge" -"BIT_XORNull" -"BIT_XOROrDefault" -"BIT_XOROrNull" -"BIT_XORResample" -"BIT_XORSimpleState" -"BIT_XORState" -"BLAKE3" "blockNumber" "blockSerializedSize" "blockSize" -"BOTH" +"bool" +"boolean" "boundingRatio" "boundingRatioArgMax" "boundingRatioArgMin" @@ -557,18 +1414,14 @@ "boundingRatioSimpleState" "boundingRatioState" "buildId" -"BY" "byteHammingDistance" "byteSize" "byteSlice" "byteSwap" -"CASCADE" -"CASE" "caseWithExpr" "caseWithExpression" "caseWithoutExpr" "caseWithoutExpression" -"CAST" "catboostEvaluate" "categoricalInformationValue" "categoricalInformationValueArgMax" @@ -588,46 +1441,17 @@ "cbrt" "ceil" "ceiling" -"CHANGE" -"CHANGEABLE_IN_READONLY" -"CHANGED" "changeDay" "changeHour" "changeMinute" "changeMonth" "changeSecond" "changeYear" -"CHAR VARYING" -"CHAR" "char" -"CHARACTER LARGE OBJECT" -"CHARACTER VARYING" -"CHARACTER" -"CHARACTER_LENGTH" -"CHAR_LENGTH" -"CHECK ALL TABLES" -"CHECK TABLE" -"CHECK" "cityHash64" "clamp" -"CLEANUP" -"CLEAR COLUMN" -"CLEAR INDEX" -"CLEAR PROJECTION" -"CLEAR STATISTICS" -"CLUSTER" -"CLUSTERS" "cluster_host_ids" -"CN" "coalesce" -"CODEC" -"COLLATE" -"COLUMN" -"COLUMNS" -"COMMENT COLUMN" -"COMMENT" -"COMMIT" -"COMPRESSION" "concat" "concatAssumeInjective" "concatWithSeparator" @@ -635,8 +1459,6 @@ "concat_ws" "connectionId" "connection_id" -"CONST" -"CONSTRAINT" "contingency" "contingencyArgMax" "contingencyArgMin" @@ -813,36 +1635,6 @@ "covarSampStableSimpleState" "covarSampStableState" "covarSampState" -"COVAR_POP" -"COVAR_POPArgMax" -"COVAR_POPArgMin" -"COVAR_POPArray" -"COVAR_POPDistinct" -"COVAR_POPForEach" -"COVAR_POPIf" -"COVAR_POPMap" -"COVAR_POPMerge" -"COVAR_POPNull" -"COVAR_POPOrDefault" -"COVAR_POPOrNull" -"COVAR_POPResample" -"COVAR_POPSimpleState" -"COVAR_POPState" -"COVAR_SAMP" -"COVAR_SAMPArgMax" -"COVAR_SAMPArgMin" -"COVAR_SAMPArray" -"COVAR_SAMPDistinct" -"COVAR_SAMPForEach" -"COVAR_SAMPIf" -"COVAR_SAMPMap" -"COVAR_SAMPMerge" -"COVAR_SAMPNull" -"COVAR_SAMPOrDefault" -"COVAR_SAMPOrNull" -"COVAR_SAMPResample" -"COVAR_SAMPSimpleState" -"COVAR_SAMPState" "cramersV" "cramersVArgMax" "cramersVArgMin" @@ -873,38 +1665,16 @@ "cramersVResample" "cramersVSimpleState" "cramersVState" -"CRC32" -"CRC32IEEE" -"CRC64" -"CREATE POLICY" -"CREATE PROFILE" -"CREATE QUOTA" -"CREATE ROLE" -"CREATE ROW POLICY" -"CREATE SETTINGS PROFILE" -"CREATE TABLE" -"CREATE TEMPORARY TABLE" -"CREATE USER" -"CREATE" -"CROSS" -"CUBE" "curdate" -"CURRENT GRANTS" -"CURRENT QUOTA" -"CURRENT ROLES" -"CURRENT ROW" -"CURRENT TRANSACTION" "currentDatabase" "currentProfiles" "currentRoles" "currentSchemas" -"CURRENTUSER" "currentUser" "current_database" "current_date" "current_schemas" "current_timestamp" -"CURRENT_USER" "current_user" "cutFragment" "cutIPv6" @@ -920,59 +1690,25 @@ "cutToFirstSignificantSubdomainWithWWWRFC" "cutURLParameter" "cutWWW" -"D" "damerauLevenshteinDistance" -"DATA INNER UUID" -"DATA" -"DATABASE" -"DATABASES" -"DATE" -"Date" -"DATEADD" -"DATEDIFF" "dateDiff" "dateName" -"DATESUB" -"DateTime" -"DateTime64" "dateTime64ToSnowflake" "dateTime64ToSnowflakeID" "dateTimeToSnowflake" "dateTimeToSnowflakeID" "dateTrunc" -"DATE_ADD" -"DATE_DIFF" "date_diff" -"DATE_FORMAT" -"DATE_SUB" -"DATE_TRUNC" -"DAY" -"DAYOFMONTH" -"DAYOFWEEK" -"DAYOFYEAR" -"DAYS" -"DD" -"Decimal" -"Decimal128" -"Decimal32" -"Decimal64" "decodeHTMLComponent" "decodeURLComponent" "decodeURLFormComponent" "decodeXMLComponent" "decrypt" -"DEDUPLICATE" -"DEFAULT DATABASE" -"DEFAULT ROLE" -"DEFAULT" "defaultProfiles" "defaultRoles" "defaultValueOfArgumentType" "defaultValueOfTypeName" -"DEFINER" "degrees" -"DELETE WHERE" -"DELETE" "deltaSum" "deltaSumArgMax" "deltaSumArgMin" @@ -1034,13 +1770,6 @@ "dense_rankResample" "dense_rankSimpleState" "dense_rankState" -"DEPENDS ON" -"DESC" -"DESCENDING" -"DESCRIBE" -"DETACH PART" -"DETACH PARTITION" -"DETACH" "detectCharset" "detectLanguage" "detectLanguageMixed" @@ -1060,6 +1789,10 @@ "dictGetFloat64" "dictGetFloat64OrDefault" "dictGetHierarchy" +"dictGetIPv4" +"dictGetIPv4OrDefault" +"dictGetIPv6" +"dictGetIPv6OrDefault" "dictGetInt16" "dictGetInt16OrDefault" "dictGetInt32" @@ -1068,10 +1801,6 @@ "dictGetInt64OrDefault" "dictGetInt8" "dictGetInt8OrDefault" -"dictGetIPv4" -"dictGetIPv4OrDefault" -"dictGetIPv6" -"dictGetIPv6OrDefault" "dictGetOrDefault" "dictGetOrNull" "dictGetString" @@ -1087,19 +1816,13 @@ "dictGetUUID" "dictGetUUIDOrDefault" "dictHas" -"DICTIONARIES" -"DICTIONARY" "dictIsIn" -"DISK" "displayName" "distanceL1" "distanceL2" "distanceL2Squared" "distanceLinf" "distanceLp" -"DISTINCT ON" -"DISTINCT" -"DIV" "divide" "divideDecimal" "domain" @@ -1107,30 +1830,12 @@ "domainWithoutWWW" "domainWithoutWWWRFC" "dotProduct" -"DOUBLE_SHA1_HASH" -"DOUBLE_SHA1_PASSWORD" -"DROP COLUMN" -"DROP CONSTRAINT" -"DROP DEFAULT" -"DROP DETACHED PART" -"DROP DETACHED PARTITION" -"DROP INDEX" -"DROP PART" -"DROP PARTITION" -"DROP PROJECTION" -"DROP STATISTICS" -"DROP TABLE" -"DROP TEMPORARY TABLE" -"DROP" "dumpColumnStructure" "dynamicElement" "dynamicType" "e" "editDistance" "editDistanceUTF8" -"ELSE" -"EMPTY AS" -"EMPTY" "empty" "emptyArrayDate" "emptyArrayDateTime" @@ -1146,18 +1851,14 @@ "emptyArrayUInt32" "emptyArrayUInt64" "emptyArrayUInt8" -"ENABLED ROLES" "enabledProfiles" "enabledRoles" "encodeURLComponent" "encodeURLFormComponent" "encodeXMLComponent" "encrypt" -"END" "endsWith" "endsWithUTF8" -"ENFORCED" -"ENGINE" "entropy" "entropyArgMax" "entropyArgMin" @@ -1173,32 +1874,14 @@ "entropyResample" "entropySimpleState" "entropyState" -"Enum" -"Enum16" -"Enum8" -"EPHEMERAL SEQUENTIAL" -"EPHEMERAL" "equals" "erf" "erfc" "errorCodeToName" -"ESTIMATE" "evalMLMethod" -"EVENT" -"EVENTS" -"EVERY" -"EXCEPT DATABASE" -"EXCEPT DATABASES" -"EXCEPT TABLE" -"EXCEPT TABLES" -"EXCEPT" -"EXCHANGE DICTIONARIES" -"EXCHANGE TABLES" -"EXISTS" "exp" "exp10" "exp2" -"EXPLAIN" "exponentialMovingAverage" "exponentialMovingAverageArgMax" "exponentialMovingAverageArgMin" @@ -1274,10 +1957,6 @@ "exponentialTimeDecayedSumResample" "exponentialTimeDecayedSumSimpleState" "exponentialTimeDecayedSumState" -"EXPRESSION" -"EXTENDED" -"EXTERNAL DDL FROM" -"EXTRACT" "extract" "extractAll" "extractAllGroups" @@ -1291,24 +1970,13 @@ "extractURLParameterNames" "extractURLParameters" "factorial" -"FALSE" "farmFingerprint64" "farmHash64" -"FETCH PART" -"FETCH PARTITION" -"FETCH" -"FIELDS" -"FILE" "file" -"FILESYSTEM CACHE" -"FILESYSTEM CACHES" "filesystemAvailable" "filesystemCapacity" "filesystemUnreserved" -"FILTER" -"FINAL" "finalizeAggregation" -"FIRST" "firstLine" "firstSignificantSubdomain" "firstSignificantSubdomainCustom" @@ -1344,7 +2012,6 @@ "first_value_respect_nullsResample" "first_value_respect_nullsSimpleState" "first_value_respect_nullsState" -"FixedString" "flameGraph" "flameGraphArgMax" "flameGraphArgMin" @@ -1362,16 +2029,7 @@ "flameGraphState" "flatten" "flattenTuple" -"Float32" -"Float64" "floor" -"FOLLOWING" -"FOR" -"ForEach" -"FOREIGN KEY" -"FOREIGN" -"FORGET PARTITION" -"FORMAT" "format" "formatDateTime" "formatDateTimeInJodaSyntax" @@ -1385,31 +2043,19 @@ "formatReadableTimeDelta" "formatRow" "formatRowNoNewline" -"FORMAT_BYTES" -"FQDN" "fragment" -"FREEZE" -"FROM INFILE" -"FROM SHARD" -"FROM" "fromDaysSinceYearZero" "fromDaysSinceYearZero32" "fromModifiedJulianDay" "fromModifiedJulianDayOrNull" +"fromUTCTimestamp" "fromUnixTimestamp" "fromUnixTimestamp64Micro" "fromUnixTimestamp64Milli" "fromUnixTimestamp64Nano" "fromUnixTimestampInJodaSyntax" -"fromUTCTimestamp" -"FROM_BASE64" -"FROM_DAYS" -"FROM_UNIXTIME" "from_utc_timestamp" -"FULL" "fullHostName" -"FULLTEXT" -"FUNCTION" "fuzzBits" "gccMurmurHash" "gcd" @@ -1419,11 +2065,11 @@ "generateUUIDv4" "generateUUIDv7" "geoDistance" +"geoToH3" +"geoToS2" "geohashDecode" "geohashEncode" "geohashesInBox" -"geoToH3" -"geoToS2" "getClientHTTPHeader" "getMacro" "getOSKernelVersion" @@ -1432,9 +2078,6 @@ "getSizeOfEnumType" "getSubcolumn" "getTypeSerializationStreams" -"GLOBAL IN" -"GLOBAL NOT IN" -"GLOBAL" "globalIn" "globalInIgnoreSet" "globalNotIn" @@ -1444,16 +2087,11 @@ "globalNullIn" "globalNullInIgnoreSet" "globalVariable" -"GRANT OPTION FOR" -"GRANT" -"GRANTEES" -"GRANULARITY" "greatCircleAngle" "greatCircleDistance" "greater" "greaterOrEquals" "greatest" -"GROUP BY" "groupArray" "groupArrayArgMax" "groupArrayArgMin" @@ -1589,6 +2227,36 @@ "groupBitAndResample" "groupBitAndSimpleState" "groupBitAndState" +"groupBitOr" +"groupBitOrArgMax" +"groupBitOrArgMin" +"groupBitOrArray" +"groupBitOrDistinct" +"groupBitOrForEach" +"groupBitOrIf" +"groupBitOrMap" +"groupBitOrMerge" +"groupBitOrNull" +"groupBitOrOrDefault" +"groupBitOrOrNull" +"groupBitOrResample" +"groupBitOrSimpleState" +"groupBitOrState" +"groupBitXor" +"groupBitXorArgMax" +"groupBitXorArgMin" +"groupBitXorArray" +"groupBitXorDistinct" +"groupBitXorForEach" +"groupBitXorIf" +"groupBitXorMap" +"groupBitXorMerge" +"groupBitXorNull" +"groupBitXorOrDefault" +"groupBitXorOrNull" +"groupBitXorResample" +"groupBitXorSimpleState" +"groupBitXorState" "groupBitmap" "groupBitmapAnd" "groupBitmapAndArgMax" @@ -1648,36 +2316,6 @@ "groupBitmapXorResample" "groupBitmapXorSimpleState" "groupBitmapXorState" -"groupBitOr" -"groupBitOrArgMax" -"groupBitOrArgMin" -"groupBitOrArray" -"groupBitOrDistinct" -"groupBitOrForEach" -"groupBitOrIf" -"groupBitOrMap" -"groupBitOrMerge" -"groupBitOrNull" -"groupBitOrOrDefault" -"groupBitOrOrNull" -"groupBitOrResample" -"groupBitOrSimpleState" -"groupBitOrState" -"groupBitXor" -"groupBitXorArgMax" -"groupBitXorArgMin" -"groupBitXorArray" -"groupBitXorDistinct" -"groupBitXorForEach" -"groupBitXorIf" -"groupBitXorMap" -"groupBitXorMerge" -"groupBitXorNull" -"groupBitXorOrDefault" -"groupBitXorOrNull" -"groupBitXorResample" -"groupBitXorSimpleState" -"groupBitXorState" "groupConcat" "groupConcatArgMax" "groupConcatArgMin" @@ -1693,8 +2331,6 @@ "groupConcatResample" "groupConcatSimpleState" "groupConcatState" -"GROUPING SETS" -"GROUPS" "groupUniqArray" "groupUniqArrayArgMax" "groupUniqArrayArgMin" @@ -1725,7 +2361,6 @@ "group_concatResample" "group_concatSimpleState" "group_concatState" -"H" "h3CellAreaM2" "h3CellAreaRads2" "h3Distance" @@ -1753,7 +2388,6 @@ "h3IsPentagon" "h3IsResClassIII" "h3IsValid" -"h3kRing" "h3Line" "h3NumHexagons" "h3PointDistKm" @@ -1766,12 +2400,12 @@ "h3ToParent" "h3ToString" "h3UnidirectionalEdgeIsValid" +"h3kRing" "halfMD5" "has" "hasAll" "hasAny" "hasColumnInTable" -"HASH" "hasSubsequence" "hasSubsequenceCaseInsensitive" "hasSubsequenceCaseInsensitiveUTF8" @@ -1782,11 +2416,7 @@ "hasTokenCaseInsensitive" "hasTokenCaseInsensitiveOrNull" "hasTokenOrNull" -"HAVING" -"HDFS" "hex" -"HH" -"HIERARCHICAL" "hilbertDecode" "hilbertEncode" "histogram" @@ -1808,62 +2438,33 @@ "hop" "hopEnd" "hopStart" -"HOST" "hostName" "hostname" -"HOUR" -"HOURS" -"HTTP" "hypot" -"ID" -"IDENTIFIED" "identity" "idnaDecode" "idnaEncode" -"IF EMPTY" -"IF EXISTS" -"IF NOT EXISTS" "if" "ifNotFinite" "ifNull" -"IGNORE NULLS" "ignore" -"ILIKE" "ilike" -"IN PARTITION" -"IN" "in" -"INDEX" -"INDEXES" +"inIgnoreSet" "indexHint" "indexOf" -"INDICES" -"INET6_ATON" -"INET6_NTOA" -"INET_ATON" -"INET_NTOA" -"INHERIT" -"inIgnoreSet" "initcap" "initcapUTF8" -"initializeAggregation" "initialQueryID" "initial_query_id" -"INJECTIVE" -"INNER" -"INSERT INTO" +"initializeAggregation" "instr" -"Int16" -"Int32" -"Int64" -"Int8" "intDiv" "intDivOrZero" -"INTERPOLATE" -"INTERSECT" -"INTERVAL" -"IntervalDay" -"IntervalHour" +"intExp10" +"intExp2" +"intHash32" +"intHash64" "intervalLengthSum" "intervalLengthSumArgMax" "intervalLengthSumArgMin" @@ -1879,42 +2480,13 @@ "intervalLengthSumResample" "intervalLengthSumSimpleState" "intervalLengthSumState" -"IntervalMinute" -"IntervalMonth" -"IntervalQuarter" -"IntervalSecond" -"IntervalWeek" -"IntervalYear" -"intExp10" -"intExp2" -"intHash32" -"intHash64" -"INTO OUTFILE" -"INVISIBLE" -"INVOKER" -"IP" -"IPv4CIDRToRange" -"IPv4NumToString" -"IPv4NumToStringClassC" -"IPv4StringToNum" -"IPv4StringToNumOrDefault" -"IPv4StringToNumOrNull" -"IPv4ToIPv6" -"IPv6CIDRToRange" -"IPv6NumToString" -"IPv6StringToNum" -"IPv6StringToNumOrDefault" -"IPv6StringToNumOrNull" -"IS NOT DISTINCT FROM" -"IS NOT NULL" -"IS NULL" "isConstant" "isDecimalOverflow" "isFinite" -"isInfinite" "isIPAddressInRange" "isIPv4String" "isIPv6String" +"isInfinite" "isNaN" "isNotDistinctFrom" "isNotNull" @@ -1923,46 +2495,15 @@ "isValidJSON" "isValidUTF8" "isZeroOrNull" -"IS_OBJECT_ID" "jaroSimilarity" "jaroWinklerSimilarity" "javaHash" "javaHashUTF16LE" -"JOIN" "joinGet" "joinGetOrNull" -"JSONArrayLength" -"JSONExtract" -"JSONExtractArrayRaw" -"JSONExtractBool" -"JSONExtractFloat" -"JSONExtractInt" -"JSONExtractKeys" -"JSONExtractKeysAndValues" -"JSONExtractKeysAndValuesRaw" -"JSONExtractRaw" -"JSONExtractString" -"JSONExtractUInt" -"JSONHas" -"JSONKey" -"JSONLength" -"JSONMergePatch" "jsonMergePatch" -"JSONType" -"JSON_ARRAY_LENGTH" -"JSON_EXISTS" -"JSON_QUERY" -"JSON_VALUE" "jumpConsistentHash" -"JWT" "kafkaMurmurHash" -"KERBEROS" -"KEY BY" -"KEY" -"KEYED BY" -"KEYS" -"KILL" -"KIND" "kolmogorovSmirnovTest" "kolmogorovSmirnovTestArgMax" "kolmogorovSmirnovTestArgMin" @@ -2011,14 +2552,6 @@ "kurtSampResample" "kurtSampSimpleState" "kurtSampState" -"L1Distance" -"L1Norm" -"L1Normalize" -"L2Distance" -"L2Norm" -"L2Normalize" -"L2SquaredDistance" -"L2SquaredNorm" "lagInFrame" "lagInFrameArgMax" "lagInFrameArgMin" @@ -2034,7 +2567,6 @@ "lagInFrameResample" "lagInFrameSimpleState" "lagInFrameState" -"LARGE OBJECT" "largestTriangleThreeBuckets" "largestTriangleThreeBucketsArgMax" "largestTriangleThreeBucketsArgMin" @@ -2050,8 +2582,6 @@ "largestTriangleThreeBucketsResample" "largestTriangleThreeBucketsSimpleState" "largestTriangleThreeBucketsState" -"LAST" -"LAST_DAY" "last_value" "last_valueArgMax" "last_valueArgMin" @@ -2082,10 +2612,8 @@ "last_value_respect_nullsResample" "last_value_respect_nullsSimpleState" "last_value_respect_nullsState" -"LAYOUT" "lcase" "lcm" -"LDAP" "leadInFrame" "leadInFrameArgMax" "leadInFrameArgMin" @@ -2101,10 +2629,7 @@ "leadInFrameResample" "leadInFrameSimpleState" "leadInFrameState" -"LEADING" "least" -"LEFT ARRAY JOIN" -"LEFT" "left" "leftPad" "leftPadUTF8" @@ -2112,42 +2637,24 @@ "lemmatize" "length" "lengthUTF8" -"LESS THAN" "less" "lessOrEquals" -"LEVEL" "levenshteinDistance" "levenshteinDistanceUTF8" "lgamma" -"LIFETIME" -"LIGHTWEIGHT" -"LIKE" "like" -"LIMIT" -"LINEAR" -"LinfDistance" -"LinfNorm" -"LinfNormalize" -"LIST" -"LIVE" "ln" -"LOCAL" "locate" "log" "log10" "log1p" "log2" "logTrace" -"LowCardinality" "lowCardinalityIndices" "lowCardinalityKeys" "lower" "lowerUTF8" "lpad" -"LpDistance" -"LpNorm" -"LpNormalize" -"LTRIM" "ltrim" "lttb" "lttbArgMax" @@ -2164,10 +2671,6 @@ "lttbResample" "lttbSimpleState" "lttbState" -"M" -"MACNumToString" -"MACStringToNum" -"MACStringToOUI" "makeDate" "makeDate32" "makeDateTime" @@ -2208,18 +2711,8 @@ "mapSubtract" "mapUpdate" "mapValues" -"MAP_FROM_ARRAYS" -"MATCH" "match" -"MATERIALIZE COLUMN" -"MATERIALIZE INDEX" -"MATERIALIZE PROJECTION" -"MATERIALIZE STATISTICS" -"MATERIALIZE TTL" -"MATERIALIZE" "materialize" -"MATERIALIZED" -"MAX" "max" "max2" "maxArgMax" @@ -2281,9 +2774,6 @@ "maxResample" "maxSimpleState" "maxState" -"MCS" -"MD4" -"MD5" "meanZTest" "meanZTestArgMax" "meanZTestArgMin" @@ -2524,19 +3014,8 @@ "medianTimingWeightedResample" "medianTimingWeightedSimpleState" "medianTimingWeightedState" -"MEMORY" -"Merge" -"MERGES" -"METRICS INNER UUID" -"METRICS" "metroHash64" -"MI" -"MICROSECOND" -"MICROSECONDS" "mid" -"MILLISECOND" -"MILLISECONDS" -"MIN" "min" "min2" "minArgMax" @@ -2572,37 +3051,15 @@ "minSimpleState" "minState" "minus" -"MINUTE" -"MINUTES" "mismatches" -"MM" -"MOD" "mod" -"MODIFY COLUMN" -"MODIFY COMMENT" -"MODIFY DEFINER" -"MODIFY ORDER BY" -"MODIFY QUERY" -"MODIFY REFRESH" -"MODIFY SAMPLE BY" -"MODIFY SETTING" -"MODIFY SQL SECURITY" -"MODIFY STATISTICS" -"MODIFY TTL" -"MODIFY" "modulo" "moduloLegacy" "moduloOrZero" -"MONTH" "monthName" -"MONTHS" "mortonDecode" "mortonEncode" -"MOVE PART" -"MOVE PARTITION" -"MOVE" "movingXXX" -"MS" "multiFuzzyMatchAllIndices" "multiFuzzyMatchAny" "multiFuzzyMatchAnyIndex" @@ -2610,8 +3067,6 @@ "multiMatchAllIndices" "multiMatchAny" "multiMatchAnyIndex" -"multiply" -"multiplyDecimal" "multiSearchAllPositions" "multiSearchAllPositionsCaseInsensitive" "multiSearchAllPositionsCaseInsensitiveUTF8" @@ -2628,23 +3083,17 @@ "multiSearchFirstPositionCaseInsensitive" "multiSearchFirstPositionCaseInsensitiveUTF8" "multiSearchFirstPositionUTF8" +"multiply" +"multiplyDecimal" "murmurHash2_32" "murmurHash2_64" "murmurHash3_128" "murmurHash3_32" "murmurHash3_64" -"MUTATION" -"N" -"NAME" -"NAMED COLLECTION" -"NANOSECOND" -"NANOSECONDS" "negate" "neighbor" -"Nested" "nested" "netloc" -"NEXT" "ngramDistance" "ngramDistanceCaseInsensitive" "ngramDistanceCaseInsensitiveUTF8" @@ -2657,7 +3106,6 @@ "ngramMinHashCaseInsensitive" "ngramMinHashCaseInsensitiveUTF8" "ngramMinHashUTF8" -"ngrams" "ngramSearch" "ngramSearchCaseInsensitive" "ngramSearchCaseInsensitiveUTF8" @@ -2666,10 +3114,7 @@ "ngramSimHashCaseInsensitive" "ngramSimHashCaseInsensitiveUTF8" "ngramSimHashUTF8" -"NO ACTION" -"NO DELAY" -"NO LIMITS" -"NONE" +"ngrams" "nonNegativeDerivative" "nonNegativeDerivativeArgMax" "nonNegativeDerivativeArgMin" @@ -2685,8 +3130,11 @@ "nonNegativeDerivativeResample" "nonNegativeDerivativeSimpleState" "nonNegativeDerivativeState" -"normalizedQueryHash" -"normalizedQueryHashKeepNames" +"normL1" +"normL2" +"normL2Squared" +"normLinf" +"normLp" "normalizeL1" "normalizeL2" "normalizeLinf" @@ -2697,23 +3145,17 @@ "normalizeUTF8NFD" "normalizeUTF8NFKC" "normalizeUTF8NFKD" -"normL1" -"normL2" -"normL2Squared" -"normLinf" -"normLp" -"NOT BETWEEN" -"NOT IDENTIFIED" -"NOT ILIKE" -"NOT IN" -"NOT KEYED" -"NOT LIKE" -"NOT OVERRIDABLE" -"NOT" +"normalizedQueryHash" +"normalizedQueryHashKeepNames" "not" "notEmpty" "notEquals" -"Nothing" +"notILike" +"notIn" +"notInIgnoreSet" +"notLike" +"notNullIn" +"notNullInIgnoreSet" "nothing" "nothingArgMax" "nothingArgMin" @@ -2758,17 +3200,9 @@ "nothingUInt64Resample" "nothingUInt64SimpleState" "nothingUInt64State" -"notILike" -"notIn" -"notInIgnoreSet" -"notLike" -"notNullIn" -"notNullInIgnoreSet" "now" "now64" "nowInBlock" -"NO_PASSWORD" -"NS" "nth_value" "nth_valueArgMax" "nth_valueArgMin" @@ -2799,28 +3233,10 @@ "ntileResample" "ntileSimpleState" "ntileState" -"NULL" -"Null" -"Nullable" "nullIf" "nullIn" "nullInIgnoreSet" -"NULLS" -"OCTET_LENGTH" -"OFFSET" -"ON DELETE" -"ON UPDATE" -"ON VOLUME" -"ON" -"ONLY" -"OPTIMIZE TABLE" -"OR REPLACE" -"OR" "or" -"ORDER BY" -"OUTER" -"OVER" -"OVERRIDABLE" "parseDateTime" "parseDateTime32BestEffort" "parseDateTime32BestEffortOrNull" @@ -2846,15 +3262,8 @@ "parseReadableSizeOrNull" "parseReadableSizeOrZero" "parseTimeDelta" -"PART" -"PARTIAL" -"PARTITION BY" -"PARTITION" "partitionID" "partitionId" -"PARTITIONS" -"PART_MOVE_TO_SHARD" -"PASTE" "path" "pathFull" "percentRank" @@ -2887,15 +3296,7 @@ "percent_rankResample" "percent_rankSimpleState" "percent_rankState" -"PERIODIC REFRESH" -"PERMANENTLY" -"PERMISSIVE" -"PERSISTENT SEQUENTIAL" -"PERSISTENT" "pi" -"PIPELINE" -"PLAINTEXT_PASSWORD" -"PLAN" "plus" "pmod" "pointInEllipses" @@ -2916,7 +3317,6 @@ "polygonsUnionSpherical" "polygonsWithinCartesian" "polygonsWithinSpherical" -"POPULATE" "port" "portRFC" "position" @@ -2927,23 +3327,11 @@ "positive_modulo" "pow" "power" -"PRECEDING" -"PRECISION" -"PREWHERE" -"PRIMARY KEY" -"PRIMARY" "printf" -"PROFILE" -"PROJECTION" "proportionsZTest" -"Protobuf" "protocol" -"PULL" "punycodeDecode" "punycodeEncode" -"Q" -"QQ" -"QUALIFY" "quantile" "quantileArgMax" "quantileArgMin" @@ -3137,6 +3525,68 @@ "quantileOrDefault" "quantileOrNull" "quantileResample" +"quantileSimpleState" +"quantileState" +"quantileTDigest" +"quantileTDigestArgMax" +"quantileTDigestArgMin" +"quantileTDigestArray" +"quantileTDigestDistinct" +"quantileTDigestForEach" +"quantileTDigestIf" +"quantileTDigestMap" +"quantileTDigestMerge" +"quantileTDigestNull" +"quantileTDigestOrDefault" +"quantileTDigestOrNull" +"quantileTDigestResample" +"quantileTDigestSimpleState" +"quantileTDigestState" +"quantileTDigestWeighted" +"quantileTDigestWeightedArgMax" +"quantileTDigestWeightedArgMin" +"quantileTDigestWeightedArray" +"quantileTDigestWeightedDistinct" +"quantileTDigestWeightedForEach" +"quantileTDigestWeightedIf" +"quantileTDigestWeightedMap" +"quantileTDigestWeightedMerge" +"quantileTDigestWeightedNull" +"quantileTDigestWeightedOrDefault" +"quantileTDigestWeightedOrNull" +"quantileTDigestWeightedResample" +"quantileTDigestWeightedSimpleState" +"quantileTDigestWeightedState" +"quantileTiming" +"quantileTimingArgMax" +"quantileTimingArgMin" +"quantileTimingArray" +"quantileTimingDistinct" +"quantileTimingForEach" +"quantileTimingIf" +"quantileTimingMap" +"quantileTimingMerge" +"quantileTimingNull" +"quantileTimingOrDefault" +"quantileTimingOrNull" +"quantileTimingResample" +"quantileTimingSimpleState" +"quantileTimingState" +"quantileTimingWeighted" +"quantileTimingWeightedArgMax" +"quantileTimingWeightedArgMin" +"quantileTimingWeightedArray" +"quantileTimingWeightedDistinct" +"quantileTimingWeightedForEach" +"quantileTimingWeightedIf" +"quantileTimingWeightedMap" +"quantileTimingWeightedMerge" +"quantileTimingWeightedNull" +"quantileTimingWeightedOrDefault" +"quantileTimingWeightedOrNull" +"quantileTimingWeightedResample" +"quantileTimingWeightedSimpleState" +"quantileTimingWeightedState" "quantiles" "quantilesArgMax" "quantilesArgMin" @@ -3309,7 +3759,6 @@ "quantilesGKSimpleState" "quantilesGKState" "quantilesIf" -"quantileSimpleState" "quantilesInterpolatedWeighted" "quantilesInterpolatedWeightedArgMax" "quantilesInterpolatedWeightedArgMin" @@ -3333,7 +3782,6 @@ "quantilesResample" "quantilesSimpleState" "quantilesState" -"quantileState" "quantilesTDigest" "quantilesTDigestArgMax" "quantilesTDigestArgMin" @@ -3394,75 +3842,10 @@ "quantilesTimingWeightedResample" "quantilesTimingWeightedSimpleState" "quantilesTimingWeightedState" -"quantileTDigest" -"quantileTDigestArgMax" -"quantileTDigestArgMin" -"quantileTDigestArray" -"quantileTDigestDistinct" -"quantileTDigestForEach" -"quantileTDigestIf" -"quantileTDigestMap" -"quantileTDigestMerge" -"quantileTDigestNull" -"quantileTDigestOrDefault" -"quantileTDigestOrNull" -"quantileTDigestResample" -"quantileTDigestSimpleState" -"quantileTDigestState" -"quantileTDigestWeighted" -"quantileTDigestWeightedArgMax" -"quantileTDigestWeightedArgMin" -"quantileTDigestWeightedArray" -"quantileTDigestWeightedDistinct" -"quantileTDigestWeightedForEach" -"quantileTDigestWeightedIf" -"quantileTDigestWeightedMap" -"quantileTDigestWeightedMerge" -"quantileTDigestWeightedNull" -"quantileTDigestWeightedOrDefault" -"quantileTDigestWeightedOrNull" -"quantileTDigestWeightedResample" -"quantileTDigestWeightedSimpleState" -"quantileTDigestWeightedState" -"quantileTiming" -"quantileTimingArgMax" -"quantileTimingArgMin" -"quantileTimingArray" -"quantileTimingDistinct" -"quantileTimingForEach" -"quantileTimingIf" -"quantileTimingMap" -"quantileTimingMerge" -"quantileTimingNull" -"quantileTimingOrDefault" -"quantileTimingOrNull" -"quantileTimingResample" -"quantileTimingSimpleState" -"quantileTimingState" -"quantileTimingWeighted" -"quantileTimingWeightedArgMax" -"quantileTimingWeightedArgMin" -"quantileTimingWeightedArray" -"quantileTimingWeightedDistinct" -"quantileTimingWeightedForEach" -"quantileTimingWeightedIf" -"quantileTimingWeightedMap" -"quantileTimingWeightedMerge" -"quantileTimingWeightedNull" -"quantileTimingWeightedOrDefault" -"quantileTimingWeightedOrNull" -"quantileTimingWeightedResample" -"quantileTimingWeightedSimpleState" -"quantileTimingWeightedState" -"QUARTER" -"QUARTERS" -"QUERY TREE" -"QUERY" "queryID" "queryString" "queryStringAndFragment" "query_id" -"QUOTA" "radians" "rand" "rand32" @@ -3477,16 +3860,13 @@ "randLogNormal" "randNegativeBinomial" "randNormal" -"randomFixedString" -"RANDOMIZE FOR" -"RANDOMIZED" -"randomPrintableASCII" -"randomString" -"randomStringUTF8" "randPoisson" "randStudentT" "randUniform" -"RANGE" +"randomFixedString" +"randomPrintableASCII" +"randomString" +"randomStringUTF8" "range" "rank" "rankArgMax" @@ -3518,24 +3898,14 @@ "rankResample" "rankSimpleState" "rankState" -"READONLY" "readWKTLineString" "readWKTMultiLineString" "readWKTMultiPolygon" "readWKTPoint" "readWKTPolygon" "readWKTRing" -"REALM" -"RECOMPRESS" -"RECURSIVE" -"REFERENCES" -"REFRESH" -"REGEXP" "regexpExtract" "regexpQuoteMeta" -"REGEXP_EXTRACT" -"REGEXP_MATCHES" -"REGEXP_REPLACE" "regionHierarchy" "regionIn" "regionToArea" @@ -3566,31 +3936,13 @@ "reinterpretAsUInt64" "reinterpretAsUInt8" "reinterpretAsUUID" -"REMOVE SAMPLE BY" -"REMOVE TTL" -"REMOVE" -"RENAME COLUMN" -"RENAME DATABASE" -"RENAME DICTIONARY" -"RENAME TABLE" -"RENAME TO" -"RENAME" "repeat" -"REPLACE PARTITION" -"REPLACE" "replace" "replaceAll" "replaceOne" "replaceRegexpAll" "replaceRegexpOne" "replicate" -"Resample" -"RESET SETTING" -"RESPECT NULLS" -"RESTORE" -"RESTRICT" -"RESTRICTIVE" -"RESUME" "retention" "retentionArgMax" "retentionArgMin" @@ -3609,24 +3961,18 @@ "reverse" "reverseUTF8" "revision" -"REVOKE" -"RIGHT" "right" "rightPad" "rightPadUTF8" "rightUTF8" -"ROLLBACK" -"ROLLUP" "round" "roundAge" "roundBankers" "roundDown" "roundDuration" "roundToExp2" -"ROW" "rowNumberInAllBlocks" "rowNumberInBlock" -"ROWS" "row_number" "row_numberArgMax" "row_numberArgMin" @@ -3643,13 +3989,11 @@ "row_numberSimpleState" "row_numberState" "rpad" -"RTRIM" "rtrim" "runningAccumulate" "runningConcurrency" "runningDifference" "runningDifferenceStartingWithFirstValue" -"S" "s2CapContains" "s2CapUnion" "s2CellsIntersect" @@ -3659,18 +4003,7 @@ "s2RectIntersection" "s2RectUnion" "s2ToGeo" -"S3" -"SALT" -"SAMPLE BY" -"SAMPLE" -"SAN" "scalarProduct" -"SCHEMA" -"SCHEME" -"SECOND" -"SECONDS" -"SELECT" -"SEMI" "sequenceCount" "sequenceCountArgMax" "sequenceCountArgMin" @@ -3719,43 +4052,14 @@ "seriesDecomposeSTL" "seriesOutliersDetectTukey" "seriesPeriodDetectFFT" -"SERVER" "serverTimeZone" "serverTimezone" "serverUUID" -"SET DEFAULT ROLE" -"SET DEFAULT" -"SET FAKE TIME" -"SET NULL" -"SET ROLE DEFAULT" -"SET ROLE" -"SET TRANSACTION SNAPSHOT" -"SET" -"SETTINGS" -"SHA1" -"SHA224" -"SHA256" -"SHA256_HASH" -"SHA256_PASSWORD" -"SHA384" -"SHA512" -"SHA512_256" "shardCount" "shardNum" -"SHOW ACCESS" -"SHOW CREATE" -"SHOW ENGINES" -"SHOW FUNCTIONS" -"SHOW GRANTS" -"SHOW PRIVILEGES" -"SHOW PROCESSLIST" -"SHOW SETTING" -"SHOW" "showCertificate" "sigmoid" "sign" -"SIGNED" -"SIMPLE" "simpleJSONExtractBool" "simpleJSONExtractFloat" "simpleJSONExtractInt" @@ -3838,39 +4142,37 @@ "snowflakeToDateTime" "snowflakeToDateTime64" "soundex" -"SOURCE" "space" "sparkBar" -"sparkbar" "sparkBarArgMax" -"sparkbarArgMax" "sparkBarArgMin" -"sparkbarArgMin" "sparkBarArray" -"sparkbarArray" "sparkBarDistinct" -"sparkbarDistinct" "sparkBarForEach" -"sparkbarForEach" "sparkBarIf" -"sparkbarIf" "sparkBarMap" -"sparkbarMap" "sparkBarMerge" -"sparkbarMerge" "sparkBarNull" -"sparkbarNull" "sparkBarOrDefault" -"sparkbarOrDefault" "sparkBarOrNull" -"sparkbarOrNull" "sparkBarResample" -"sparkbarResample" "sparkBarSimpleState" -"sparkbarSimpleState" "sparkBarState" +"sparkbar" +"sparkbarArgMax" +"sparkbarArgMin" +"sparkbarArray" +"sparkbarDistinct" +"sparkbarForEach" +"sparkbarIf" +"sparkbarMap" +"sparkbarMerge" +"sparkbarNull" +"sparkbarOrDefault" +"sparkbarOrNull" +"sparkbarResample" +"sparkbarSimpleState" "sparkbarState" -"SPATIAL" "splitByAlpha" "splitByChar" "splitByNonAlpha" @@ -3880,31 +4182,9 @@ "sqid" "sqidDecode" "sqidEncode" -"SQL SECURITY" -"SQL_TSI_DAY" -"SQL_TSI_HOUR" -"SQL_TSI_MICROSECOND" -"SQL_TSI_MILLISECOND" -"SQL_TSI_MINUTE" -"SQL_TSI_MONTH" -"SQL_TSI_NANOSECOND" -"SQL_TSI_QUARTER" -"SQL_TSI_SECOND" -"SQL_TSI_WEEK" -"SQL_TSI_YEAR" "sqrt" -"SS" -"SSH_KEY" -"SSL_CERTIFICATE" -"START TRANSACTION" "startsWith" "startsWithUTF8" -"State" -"STATISTICS" -"STD" -"STDArgMax" -"STDArgMin" -"STDArray" "stddevPop" "stddevPopArgMax" "stddevPopArgMin" @@ -3965,49 +4245,7 @@ "stddevSampStableSimpleState" "stddevSampStableState" "stddevSampState" -"STDDEV_POP" -"STDDEV_POPArgMax" -"STDDEV_POPArgMin" -"STDDEV_POPArray" -"STDDEV_POPDistinct" -"STDDEV_POPForEach" -"STDDEV_POPIf" -"STDDEV_POPMap" -"STDDEV_POPMerge" -"STDDEV_POPNull" -"STDDEV_POPOrDefault" -"STDDEV_POPOrNull" -"STDDEV_POPResample" -"STDDEV_POPSimpleState" -"STDDEV_POPState" -"STDDEV_SAMP" -"STDDEV_SAMPArgMax" -"STDDEV_SAMPArgMin" -"STDDEV_SAMPArray" -"STDDEV_SAMPDistinct" -"STDDEV_SAMPForEach" -"STDDEV_SAMPIf" -"STDDEV_SAMPMap" -"STDDEV_SAMPMerge" -"STDDEV_SAMPNull" -"STDDEV_SAMPOrDefault" -"STDDEV_SAMPOrNull" -"STDDEV_SAMPResample" -"STDDEV_SAMPSimpleState" -"STDDEV_SAMPState" -"STDDistinct" -"STDForEach" -"STDIf" -"STDMap" -"STDMerge" -"STDNull" -"STDOrDefault" -"STDOrNull" -"STDResample" -"STDSimpleState" -"STDState" "stem" -"STEP" "stochasticLinearRegression" "stochasticLinearRegressionArgMax" "stochasticLinearRegressionArgMin" @@ -4038,17 +4276,13 @@ "stochasticLogisticRegressionResample" "stochasticLogisticRegressionSimpleState" "stochasticLogisticRegressionState" -"STORAGE" -"STRICT" -"STRICTLY_ASCENDING" -"String" +"str_to_date" +"str_to_map" "stringJaccardIndex" "stringJaccardIndexUTF8" "stringToH3" "structureToCapnProtoSchema" "structureToProtobufSchema" -"str_to_date" -"str_to_map" "studentTTest" "studentTTestArgMax" "studentTTestArgMin" @@ -4066,16 +4300,11 @@ "studentTTestState" "subBitmap" "subDate" -"SUBPARTITION BY" -"SUBPARTITION" -"SUBPARTITIONS" "substr" -"SUBSTRING" "substring" "substringIndex" "substringIndexUTF8" "substringUTF8" -"SUBSTRING_INDEX" "subtractDays" "subtractHours" "subtractInterval" @@ -4157,21 +4386,6 @@ "sumMapFilteredWithOverflowResample" "sumMapFilteredWithOverflowSimpleState" "sumMapFilteredWithOverflowState" -"sumMappedArrays" -"sumMappedArraysArgMax" -"sumMappedArraysArgMin" -"sumMappedArraysArray" -"sumMappedArraysDistinct" -"sumMappedArraysForEach" -"sumMappedArraysIf" -"sumMappedArraysMap" -"sumMappedArraysMerge" -"sumMappedArraysNull" -"sumMappedArraysOrDefault" -"sumMappedArraysOrNull" -"sumMappedArraysResample" -"sumMappedArraysSimpleState" -"sumMappedArraysState" "sumMapWithOverflow" "sumMapWithOverflowArgMax" "sumMapWithOverflowArgMin" @@ -4187,6 +4401,21 @@ "sumMapWithOverflowResample" "sumMapWithOverflowSimpleState" "sumMapWithOverflowState" +"sumMappedArrays" +"sumMappedArraysArgMax" +"sumMappedArraysArgMin" +"sumMappedArraysArray" +"sumMappedArraysDistinct" +"sumMappedArraysForEach" +"sumMappedArraysIf" +"sumMappedArraysMap" +"sumMappedArraysMerge" +"sumMappedArraysNull" +"sumMappedArraysOrDefault" +"sumMappedArraysOrNull" +"sumMappedArraysResample" +"sumMappedArraysSimpleState" +"sumMappedArraysState" "sumMerge" "sumNull" "sumOrDefault" @@ -4209,24 +4438,11 @@ "sumWithOverflowResample" "sumWithOverflowSimpleState" "sumWithOverflowState" -"SUSPEND" -"SVG" "svg" -"SYNC" "synonyms" -"SYNTAX" -"SYSTEM" -"TABLE OVERRIDE" -"TABLE" -"TABLES" -"TAGS INNER UUID" -"TAGS" "tan" "tanh" "tcpPort" -"TEMPORARY TABLE" -"TEMPORARY" -"TEST" "tgamma" "theilsU" "theilsUArgMax" @@ -4243,34 +4459,20 @@ "theilsUResample" "theilsUSimpleState" "theilsUState" -"THEN" "throwIf" "tid" "timeDiff" "timeSlot" "timeSlots" -"TIMESTAMP" -"timestamp" -"TIMESTAMPADD" -"TIMESTAMPDIFF" -"timestampDiff" -"TIMESTAMPSUB" -"TIMESTAMP_ADD" -"TIMESTAMP_DIFF" -"timestamp_diff" -"TIMESTAMP_SUB" "timeZone" -"timezone" "timeZoneOf" -"timezoneOf" "timeZoneOffset" +"timestamp" +"timestampDiff" +"timestamp_diff" +"timezone" +"timezoneOf" "timezoneOffset" -"TO DISK" -"TO INNER UUID" -"TO SHARD" -"TO TABLE" -"TO VOLUME" -"TO" "toBool" "toColumnTypeName" "toDate" @@ -4290,7 +4492,6 @@ "toDateTimeOrDefault" "toDateTimeOrNull" "toDateTimeOrZero" -"today" "toDayOfMonth" "toDayOfWeek" "toDayOfYear" @@ -4322,6 +4523,16 @@ "toFloat64OrNull" "toFloat64OrZero" "toHour" +"toIPv4" +"toIPv4OrDefault" +"toIPv4OrNull" +"toIPv4OrZero" +"toIPv6" +"toIPv6OrDefault" +"toIPv6OrNull" +"toIPv6OrZero" +"toISOWeek" +"toISOYear" "toInt128" "toInt128OrDefault" "toInt128OrNull" @@ -4357,18 +4568,7 @@ "toIntervalSecond" "toIntervalWeek" "toIntervalYear" -"toIPv4" -"toIPv4OrDefault" -"toIPv4OrNull" -"toIPv4OrZero" -"toIPv6" -"toIPv6OrDefault" -"toIPv6OrNull" -"toIPv6OrZero" -"toISOWeek" -"toISOYear" "toJSONString" -"tokens" "toLastDayOfMonth" "toLastDayOfWeek" "toLowCardinality" @@ -4379,7 +4579,82 @@ "toMonday" "toMonth" "toNullable" -"TOP" +"toQuarter" +"toRelativeDayNum" +"toRelativeHourNum" +"toRelativeMinuteNum" +"toRelativeMonthNum" +"toRelativeQuarterNum" +"toRelativeSecondNum" +"toRelativeWeekNum" +"toRelativeYearNum" +"toSecond" +"toStartOfDay" +"toStartOfFifteenMinutes" +"toStartOfFiveMinute" +"toStartOfFiveMinutes" +"toStartOfHour" +"toStartOfISOYear" +"toStartOfInterval" +"toStartOfMicrosecond" +"toStartOfMillisecond" +"toStartOfMinute" +"toStartOfMonth" +"toStartOfNanosecond" +"toStartOfQuarter" +"toStartOfSecond" +"toStartOfTenMinutes" +"toStartOfWeek" +"toStartOfYear" +"toString" +"toStringCutToZero" +"toTime" +"toTimeZone" +"toTimezone" +"toTypeName" +"toUInt128" +"toUInt128OrDefault" +"toUInt128OrNull" +"toUInt128OrZero" +"toUInt16" +"toUInt16OrDefault" +"toUInt16OrNull" +"toUInt16OrZero" +"toUInt256" +"toUInt256OrDefault" +"toUInt256OrNull" +"toUInt256OrZero" +"toUInt32" +"toUInt32OrDefault" +"toUInt32OrNull" +"toUInt32OrZero" +"toUInt64" +"toUInt64OrDefault" +"toUInt64OrNull" +"toUInt64OrZero" +"toUInt8" +"toUInt8OrDefault" +"toUInt8OrNull" +"toUInt8OrZero" +"toUTCTimestamp" +"toUUID" +"toUUIDOrDefault" +"toUUIDOrNull" +"toUUIDOrZero" +"toUnixTimestamp" +"toUnixTimestamp64Micro" +"toUnixTimestamp64Milli" +"toUnixTimestamp64Nano" +"toValidUTF8" +"toWeek" +"toYYYYMM" +"toYYYYMMDD" +"toYYYYMMDDhhmmss" +"toYear" +"toYearWeek" +"to_utc_timestamp" +"today" +"tokens" "topK" "topKArgMax" "topKArgMin" @@ -4412,102 +4687,17 @@ "topKWeightedState" "topLevelDomain" "topLevelDomainRFC" -"toQuarter" -"toRelativeDayNum" -"toRelativeHourNum" -"toRelativeMinuteNum" -"toRelativeMonthNum" -"toRelativeQuarterNum" -"toRelativeSecondNum" -"toRelativeWeekNum" -"toRelativeYearNum" -"toSecond" -"toStartOfDay" -"toStartOfFifteenMinutes" -"toStartOfFiveMinute" -"toStartOfFiveMinutes" -"toStartOfHour" -"toStartOfInterval" -"toStartOfISOYear" -"toStartOfMicrosecond" -"toStartOfMillisecond" -"toStartOfMinute" -"toStartOfMonth" -"toStartOfNanosecond" -"toStartOfQuarter" -"toStartOfSecond" -"toStartOfTenMinutes" -"toStartOfWeek" -"toStartOfYear" -"toString" -"toStringCutToZero" -"TOTALS" -"toTime" -"toTimeZone" -"toTimezone" -"toTypeName" -"toUInt128" -"toUInt128OrDefault" -"toUInt128OrNull" -"toUInt128OrZero" -"toUInt16" -"toUInt16OrDefault" -"toUInt16OrNull" -"toUInt16OrZero" -"toUInt256" -"toUInt256OrDefault" -"toUInt256OrNull" -"toUInt256OrZero" -"toUInt32" -"toUInt32OrDefault" -"toUInt32OrNull" -"toUInt32OrZero" -"toUInt64" -"toUInt64OrDefault" -"toUInt64OrNull" -"toUInt64OrZero" -"toUInt8" -"toUInt8OrDefault" -"toUInt8OrNull" -"toUInt8OrZero" -"toUnixTimestamp" -"toUnixTimestamp64Micro" -"toUnixTimestamp64Milli" -"toUnixTimestamp64Nano" -"toUTCTimestamp" -"toUUID" -"toUUIDOrDefault" -"toUUIDOrNull" -"toUUIDOrZero" -"toValidUTF8" -"toWeek" -"toYear" -"toYearWeek" -"toYYYYMM" -"toYYYYMMDD" -"toYYYYMMDDhhmmss" -"TO_BASE64" -"TO_DAYS" -"TO_UNIXTIME" -"to_utc_timestamp" -"TRACKING ONLY" -"TRAILING" -"TRANSACTION" "transactionID" "transactionLatestSnapshot" "transactionOldestSnapshot" "transform" "translate" "translateUTF8" -"TRIGGER" -"TRIM" "trim" "trimBoth" "trimLeft" "trimRight" -"TRUE" "trunc" -"TRUNCATE" "truncate" "tryBase58Decode" "tryBase64Decode" @@ -4515,11 +4705,9 @@ "tryDecrypt" "tryIdnaEncode" "tryPunycodeDecode" -"TTL" "tumble" "tumbleEnd" "tumbleStart" -"Tuple" "tuple" "tupleConcat" "tupleDivide" @@ -4539,20 +4727,9 @@ "tupleNegate" "tuplePlus" "tupleToNameValuePairs" -"TYPE" -"TYPEOF" "ucase" -"UInt16" -"UInt32" -"UInt64" -"UInt8" -"ULIDStringToDateTime" "unbin" -"UNBOUNDED" -"UNDROP" -"UNFREEZE" "unhex" -"UNION" "uniq" "uniqArgMax" "uniqArgMin" @@ -4646,7 +4823,6 @@ "uniqThetaSimpleState" "uniqThetaState" "uniqThetaUnion" -"UNIQUE" "uniqUpTo" "uniqUpToArgMax" "uniqUpToArgMin" @@ -4662,31 +4838,11 @@ "uniqUpToResample" "uniqUpToSimpleState" "uniqUpToState" -"UNSET FAKE TIME" -"UNSIGNED" -"UPDATE" "upper" "upperUTF8" "uptime" -"URL" -"URLHash" -"URLHierarchy" -"URLPathHierarchy" -"USE" "user" -"USING" -"UTCTimestamp" -"UTC_timestamp" -"UUID" -"UUIDNumToString" -"UUIDStringToNum" -"UUIDToNum" -"UUIDv7ToDateTime" -"VALID UNTIL" "validateNestedArraySizes" -"VALUES" -"variantElement" -"variantType" "varPop" "varPopArgMax" "varPopArgMin" @@ -4747,42 +4903,11 @@ "varSampStableSimpleState" "varSampStableState" "varSampState" -"VARYING" -"VAR_POP" -"VAR_POPArgMax" -"VAR_POPArgMin" -"VAR_POPArray" -"VAR_POPDistinct" -"VAR_POPForEach" -"VAR_POPIf" -"VAR_POPMap" -"VAR_POPMerge" -"VAR_POPNull" -"VAR_POPOrDefault" -"VAR_POPOrNull" -"VAR_POPResample" -"VAR_POPSimpleState" -"VAR_POPState" -"VAR_SAMP" -"VAR_SAMPArgMax" -"VAR_SAMPArgMin" -"VAR_SAMPArray" -"VAR_SAMPDistinct" -"VAR_SAMPForEach" -"VAR_SAMPIf" -"VAR_SAMPMap" -"VAR_SAMPMerge" -"VAR_SAMPNull" -"VAR_SAMPOrDefault" -"VAR_SAMPOrNull" -"VAR_SAMPResample" -"VAR_SAMPSimpleState" -"VAR_SAMPState" +"variantElement" +"variantType" "vectorDifference" "vectorSum" "version" -"VIEW" -"VISIBLE" "visibleWidth" "visitParamExtractBool" "visitParamExtractFloat" @@ -4791,11 +4916,7 @@ "visitParamExtractString" "visitParamExtractUInt" "visitParamHas" -"WATCH" -"WATERMARK" -"WEEK" "week" -"WEEKS" "welchTTest" "welchTTestArgMax" "welchTTestArgMin" @@ -4811,11 +4932,8 @@ "welchTTestResample" "welchTTestSimpleState" "welchTTestState" -"WHEN" -"WHERE" "widthBucket" "width_bucket" -"WINDOW" "windowFunnel" "windowFunnelArgMax" "windowFunnelArgMin" @@ -4832,16 +4950,6 @@ "windowFunnelSimpleState" "windowFunnelState" "windowID" -"WITH ADMIN OPTION" -"WITH CHECK" -"WITH FILL" -"WITH GRANT OPTION" -"WITH NAME" -"WITH REPLACE OPTION" -"WITH TIES" -"WITH" -"WITH_ITEMINDEX" -"WK" "wkt" "wordShingleMinHash" "wordShingleMinHashArg" @@ -4855,31 +4963,12 @@ "wordShingleSimHashCaseInsensitive" "wordShingleSimHashCaseInsensitiveUTF8" "wordShingleSimHashUTF8" -"WRITABLE" -"WW" "wyHash64" "xor" -"xxh3" "xxHash32" "xxHash64" +"xxh3" "yandexConsistentHash" -"YEAR" -"YEARS" "yearweek" "yesterday" -"YY" -"YYYY" -"YYYYMMDDhhmmssToDateTime" -"YYYYMMDDhhmmssToDateTime64" -"YYYYMMDDToDate" -"YYYYMMDDToDate32" -"ZKPATH" "zookeeperSessionUptime" -"_CAST" -"__actionName" -"__bitBoolMaskAnd" -"__bitBoolMaskOr" -"__bitSwapLastTwo" -"__bitWrapperFunc" -"__getScalar" -"__scalarSubqueryResult" diff --git a/tests/fuzz/dictionaries/datatypes.dict b/tests/fuzz/dictionaries/datatypes.dict index e562595fb67..797905203b2 100644 --- a/tests/fuzz/dictionaries/datatypes.dict +++ b/tests/fuzz/dictionaries/datatypes.dict @@ -1,4283 +1,137 @@ -"BIT_AND" -"BIT_ANDArgMax" -"BIT_ANDArgMin" -"BIT_ANDArray" -"BIT_ANDDistinct" -"BIT_ANDForEach" -"BIT_ANDIf" -"BIT_ANDMap" -"BIT_ANDMerge" -"BIT_ANDNull" -"BIT_ANDOrDefault" -"BIT_ANDOrNull" -"BIT_ANDResample" -"BIT_ANDSimpleState" -"BIT_ANDState" -"BIT_OR" -"BIT_ORArgMax" -"BIT_ORArgMin" -"BIT_ORArray" -"BIT_ORDistinct" -"BIT_ORForEach" -"BIT_ORIf" -"BIT_ORMap" -"BIT_ORMerge" -"BIT_ORNull" -"BIT_OROrDefault" -"BIT_OROrNull" -"BIT_ORResample" -"BIT_ORSimpleState" -"BIT_ORState" -"BIT_XOR" -"BIT_XORArgMax" -"BIT_XORArgMin" -"BIT_XORArray" -"BIT_XORDistinct" -"BIT_XORForEach" -"BIT_XORIf" -"BIT_XORMap" -"BIT_XORMerge" -"BIT_XORNull" -"BIT_XOROrDefault" -"BIT_XOROrNull" -"BIT_XORResample" -"BIT_XORSimpleState" -"BIT_XORState" -"BLAKE3" -"CAST" -"CHARACTER_LENGTH" -"CHAR_LENGTH" -"COVAR_POP" -"COVAR_POPArgMax" -"COVAR_POPArgMin" -"COVAR_POPArray" -"COVAR_POPDistinct" -"COVAR_POPForEach" -"COVAR_POPIf" -"COVAR_POPMap" -"COVAR_POPMerge" -"COVAR_POPNull" -"COVAR_POPOrDefault" -"COVAR_POPOrNull" -"COVAR_POPResample" -"COVAR_POPSimpleState" -"COVAR_POPState" -"COVAR_SAMP" -"COVAR_SAMPArgMax" -"COVAR_SAMPArgMin" -"COVAR_SAMPArray" -"COVAR_SAMPDistinct" -"COVAR_SAMPForEach" -"COVAR_SAMPIf" -"COVAR_SAMPMap" -"COVAR_SAMPMerge" -"COVAR_SAMPNull" -"COVAR_SAMPOrDefault" -"COVAR_SAMPOrNull" -"COVAR_SAMPResample" -"COVAR_SAMPSimpleState" -"COVAR_SAMPState" -"CRC32" -"CRC32IEEE" -"CRC64" -"DATABASE" -"DATE" -"DATE_DIFF" -"DATE_FORMAT" -"DATE_TRUNC" -"DAY" -"DAYOFMONTH" -"DAYOFWEEK" -"DAYOFYEAR" -"FORMAT_BYTES" -"FQDN" -"FROM_BASE64" -"FROM_DAYS" -"FROM_UNIXTIME" -"HOUR" -"INET6_ATON" -"INET6_NTOA" -"INET_ATON" -"INET_NTOA" -"IPv4CIDRToRange" -"IPv4NumToString" -"IPv4NumToStringClassC" -"IPv4StringToNum" -"IPv4StringToNumOrDefault" -"IPv4StringToNumOrNull" -"IPv4ToIPv6" -"IPv6CIDRToRange" -"IPv6NumToString" -"IPv6StringToNum" -"IPv6StringToNumOrDefault" -"IPv6StringToNumOrNull" -"JSONArrayLength" -"JSONExtract" -"JSONExtractArrayRaw" -"JSONExtractBool" -"JSONExtractFloat" -"JSONExtractInt" -"JSONExtractKeys" -"JSONExtractKeysAndValues" -"JSONExtractKeysAndValuesRaw" -"JSONExtractRaw" -"JSONExtractString" -"JSONExtractUInt" -"JSONHas" -"JSONKey" -"JSONLength" -"JSONMergePatch" -"JSONType" -"JSON_ARRAY_LENGTH" -"JSON_EXISTS" -"JSON_QUERY" -"JSON_VALUE" -"L1Distance" -"L1Norm" -"L1Normalize" -"L2Distance" -"L2Norm" -"L2Normalize" -"L2SquaredDistance" -"L2SquaredNorm" -"LAST_DAY" -"LinfDistance" -"LinfNorm" -"LinfNormalize" -"LpDistance" -"LpNorm" -"LpNormalize" -"MACNumToString" -"MACStringToNum" -"MACStringToOUI" -"MAP_FROM_ARRAYS" -"MD4" -"MD5" -"MILLISECOND" -"MINUTE" -"MONTH" -"OCTET_LENGTH" -"QUARTER" -"REGEXP_EXTRACT" -"REGEXP_MATCHES" -"REGEXP_REPLACE" -"SCHEMA" -"SECOND" -"SHA1" -"SHA224" -"SHA256" -"SHA384" -"SHA512" -"SHA512_256" -"STD" -"STDArgMax" -"STDArgMin" -"STDArray" -"STDDEV_POP" -"STDDEV_POPArgMax" -"STDDEV_POPArgMin" -"STDDEV_POPArray" -"STDDEV_POPDistinct" -"STDDEV_POPForEach" -"STDDEV_POPIf" -"STDDEV_POPMap" -"STDDEV_POPMerge" -"STDDEV_POPNull" -"STDDEV_POPOrDefault" -"STDDEV_POPOrNull" -"STDDEV_POPResample" -"STDDEV_POPSimpleState" -"STDDEV_POPState" -"STDDEV_SAMP" -"STDDEV_SAMPArgMax" -"STDDEV_SAMPArgMin" -"STDDEV_SAMPArray" -"STDDEV_SAMPDistinct" -"STDDEV_SAMPForEach" -"STDDEV_SAMPIf" -"STDDEV_SAMPMap" -"STDDEV_SAMPMerge" -"STDDEV_SAMPNull" -"STDDEV_SAMPOrDefault" -"STDDEV_SAMPOrNull" -"STDDEV_SAMPResample" -"STDDEV_SAMPSimpleState" -"STDDEV_SAMPState" -"STDDistinct" -"STDForEach" -"STDIf" -"STDMap" -"STDMerge" -"STDNull" -"STDOrDefault" -"STDOrNull" -"STDResample" -"STDSimpleState" -"STDState" -"SUBSTRING_INDEX" -"SVG" -"TIMESTAMP_DIFF" -"TO_BASE64" -"TO_DAYS" -"TO_UNIXTIME" -"ULIDStringToDateTime" -"URLHash" -"URLHierarchy" -"URLPathHierarchy" -"UTCTimestamp" -"UTC_timestamp" -"UUIDNumToString" -"UUIDStringToNum" -"UUIDToNum" -"UUIDv7ToDateTime" -"VAR_POP" -"VAR_POPArgMax" -"VAR_POPArgMin" -"VAR_POPArray" -"VAR_POPDistinct" -"VAR_POPForEach" -"VAR_POPIf" -"VAR_POPMap" -"VAR_POPMerge" -"VAR_POPNull" -"VAR_POPOrDefault" -"VAR_POPOrNull" -"VAR_POPResample" -"VAR_POPSimpleState" -"VAR_POPState" -"VAR_SAMP" -"VAR_SAMPArgMax" -"VAR_SAMPArgMin" -"VAR_SAMPArray" -"VAR_SAMPDistinct" -"VAR_SAMPForEach" -"VAR_SAMPIf" -"VAR_SAMPMap" -"VAR_SAMPMerge" -"VAR_SAMPNull" -"VAR_SAMPOrDefault" -"VAR_SAMPOrNull" -"VAR_SAMPResample" -"VAR_SAMPSimpleState" -"VAR_SAMPState" +"AggregateFunction" +"Array" +"BIGINT" +"BIGINT SIGNED" +"BIGINT UNSIGNED" +"BINARY" +"BINARY LARGE OBJECT" +"BINARY VARYING" +"BIT" +"BLOB" +"BYTE" +"BYTEA" +"Bool" +"CHAR" +"CHAR LARGE OBJECT" +"CHAR VARYING" +"CHARACTER" +"CHARACTER LARGE OBJECT" +"CHARACTER VARYING" +"CLOB" +"DEC" +"DOUBLE" +"DOUBLE PRECISION" +"Date" +"Date32" +"DateTime" +"DateTime32" +"DateTime64" +"Decimal" +"Decimal128" +"Decimal256" +"Decimal32" +"Decimal64" +"Dynamic" +"ENUM" +"Enum" +"Enum16" +"Enum8" +"FIXED" +"FLOAT" +"FixedString" +"Float32" +"Float64" +"GEOMETRY" +"INET4" +"INET6" +"INT" +"INT SIGNED" +"INT UNSIGNED" +"INT1" +"INT1 SIGNED" +"INT1 UNSIGNED" +"INTEGER" +"INTEGER SIGNED" +"INTEGER UNSIGNED" +"IPv4" +"IPv6" +"Int128" +"Int16" +"Int256" +"Int32" +"Int64" +"Int8" +"IntervalDay" +"IntervalHour" +"IntervalMicrosecond" +"IntervalMillisecond" +"IntervalMinute" +"IntervalMonth" +"IntervalNanosecond" +"IntervalQuarter" +"IntervalSecond" +"IntervalWeek" +"IntervalYear" +"JSON" +"LONGBLOB" +"LONGTEXT" +"LineString" +"LowCardinality" +"MEDIUMBLOB" +"MEDIUMINT" +"MEDIUMINT SIGNED" +"MEDIUMINT UNSIGNED" +"MEDIUMTEXT" +"Map" +"MultiLineString" +"MultiPolygon" +"NATIONAL CHAR" +"NATIONAL CHAR VARYING" +"NATIONAL CHARACTER" +"NATIONAL CHARACTER LARGE OBJECT" +"NATIONAL CHARACTER VARYING" +"NCHAR" +"NCHAR LARGE OBJECT" +"NCHAR VARYING" +"NUMERIC" +"NVARCHAR" +"Nested" +"Nothing" +"Nullable" +"Object" +"Point" +"Polygon" +"REAL" +"Ring" +"SET" +"SIGNED" +"SINGLE" +"SMALLINT" +"SMALLINT SIGNED" +"SMALLINT UNSIGNED" +"SimpleAggregateFunction" +"String" +"TEXT" +"TIME" +"TIMESTAMP" +"TINYBLOB" +"TINYINT" +"TINYINT SIGNED" +"TINYINT UNSIGNED" +"TINYTEXT" +"Tuple" +"UInt128" +"UInt16" +"UInt256" +"UInt32" +"UInt64" +"UInt8" +"UNSIGNED" +"UUID" +"VARBINARY" +"VARCHAR" +"VARCHAR2" +"Variant" "YEAR" -"YYYYMMDDToDate" -"YYYYMMDDToDate32" -"YYYYMMDDhhmmssToDateTime" -"YYYYMMDDhhmmssToDateTime64" -"_CAST" -"__actionName" -"__bitBoolMaskAnd" -"__bitBoolMaskOr" -"__bitSwapLastTwo" -"__bitWrapperFunc" -"__getScalar" -"__scalarSubqueryResult" -"abs" -"accurateCast" -"accurateCastOrDefault" -"accurateCastOrNull" -"acos" -"acosh" -"addDate" -"addDays" -"addHours" -"addInterval" -"addMicroseconds" -"addMilliseconds" -"addMinutes" -"addMonths" -"addNanoseconds" -"addQuarters" -"addSeconds" -"addTupleOfIntervals" -"addWeeks" -"addYears" -"addressToLine" -"addressToLineWithInlines" -"addressToSymbol" -"aes_decrypt_mysql" -"aes_encrypt_mysql" -"age" -"aggThrow" -"aggThrowArgMax" -"aggThrowArgMin" -"aggThrowArray" -"aggThrowDistinct" -"aggThrowForEach" -"aggThrowIf" -"aggThrowMap" -"aggThrowMerge" -"aggThrowNull" -"aggThrowOrDefault" -"aggThrowOrNull" -"aggThrowResample" -"aggThrowSimpleState" -"aggThrowState" -"alphaTokens" -"analysisOfVariance" -"analysisOfVarianceArgMax" -"analysisOfVarianceArgMin" -"analysisOfVarianceArray" -"analysisOfVarianceDistinct" -"analysisOfVarianceForEach" -"analysisOfVarianceIf" -"analysisOfVarianceMap" -"analysisOfVarianceMerge" -"analysisOfVarianceNull" -"analysisOfVarianceOrDefault" -"analysisOfVarianceOrNull" -"analysisOfVarianceResample" -"analysisOfVarianceSimpleState" -"analysisOfVarianceState" -"and" -"anova" -"anovaArgMax" -"anovaArgMin" -"anovaArray" -"anovaDistinct" -"anovaForEach" -"anovaIf" -"anovaMap" -"anovaMerge" -"anovaNull" -"anovaOrDefault" -"anovaOrNull" -"anovaResample" -"anovaSimpleState" -"anovaState" -"any" -"anyArgMax" -"anyArgMin" -"anyArray" -"anyDistinct" -"anyForEach" -"anyHeavy" -"anyHeavyArgMax" -"anyHeavyArgMin" -"anyHeavyArray" -"anyHeavyDistinct" -"anyHeavyForEach" -"anyHeavyIf" -"anyHeavyMap" -"anyHeavyMerge" -"anyHeavyNull" -"anyHeavyOrDefault" -"anyHeavyOrNull" -"anyHeavyResample" -"anyHeavySimpleState" -"anyHeavyState" -"anyIf" -"anyLast" -"anyLastArgMax" -"anyLastArgMin" -"anyLastArray" -"anyLastDistinct" -"anyLastForEach" -"anyLastIf" -"anyLastMap" -"anyLastMerge" -"anyLastNull" -"anyLastOrDefault" -"anyLastOrNull" -"anyLastResample" -"anyLastSimpleState" -"anyLastState" -"anyLast_respect_nulls" -"anyLast_respect_nullsArgMax" -"anyLast_respect_nullsArgMin" -"anyLast_respect_nullsArray" -"anyLast_respect_nullsDistinct" -"anyLast_respect_nullsForEach" -"anyLast_respect_nullsIf" -"anyLast_respect_nullsMap" -"anyLast_respect_nullsMerge" -"anyLast_respect_nullsNull" -"anyLast_respect_nullsOrDefault" -"anyLast_respect_nullsOrNull" -"anyLast_respect_nullsResample" -"anyLast_respect_nullsSimpleState" -"anyLast_respect_nullsState" -"anyMap" -"anyMerge" -"anyNull" -"anyOrDefault" -"anyOrNull" -"anyResample" -"anySimpleState" -"anyState" -"any_respect_nulls" -"any_respect_nullsArgMax" -"any_respect_nullsArgMin" -"any_respect_nullsArray" -"any_respect_nullsDistinct" -"any_respect_nullsForEach" -"any_respect_nullsIf" -"any_respect_nullsMap" -"any_respect_nullsMerge" -"any_respect_nullsNull" -"any_respect_nullsOrDefault" -"any_respect_nullsOrNull" -"any_respect_nullsResample" -"any_respect_nullsSimpleState" -"any_respect_nullsState" -"any_value" -"any_valueArgMax" -"any_valueArgMin" -"any_valueArray" -"any_valueDistinct" -"any_valueForEach" -"any_valueIf" -"any_valueMap" -"any_valueMerge" -"any_valueNull" -"any_valueOrDefault" -"any_valueOrNull" -"any_valueResample" -"any_valueSimpleState" -"any_valueState" -"any_value_respect_nulls" -"any_value_respect_nullsArgMax" -"any_value_respect_nullsArgMin" -"any_value_respect_nullsArray" -"any_value_respect_nullsDistinct" -"any_value_respect_nullsForEach" -"any_value_respect_nullsIf" -"any_value_respect_nullsMap" -"any_value_respect_nullsMerge" -"any_value_respect_nullsNull" -"any_value_respect_nullsOrDefault" -"any_value_respect_nullsOrNull" -"any_value_respect_nullsResample" -"any_value_respect_nullsSimpleState" -"any_value_respect_nullsState" -"appendTrailingCharIfAbsent" -"approx_top_count" -"approx_top_countArgMax" -"approx_top_countArgMin" -"approx_top_countArray" -"approx_top_countDistinct" -"approx_top_countForEach" -"approx_top_countIf" -"approx_top_countMap" -"approx_top_countMerge" -"approx_top_countNull" -"approx_top_countOrDefault" -"approx_top_countOrNull" -"approx_top_countResample" -"approx_top_countSimpleState" -"approx_top_countState" -"approx_top_k" -"approx_top_kArgMax" -"approx_top_kArgMin" -"approx_top_kArray" -"approx_top_kDistinct" -"approx_top_kForEach" -"approx_top_kIf" -"approx_top_kMap" -"approx_top_kMerge" -"approx_top_kNull" -"approx_top_kOrDefault" -"approx_top_kOrNull" -"approx_top_kResample" -"approx_top_kSimpleState" -"approx_top_kState" -"approx_top_sum" -"approx_top_sumArgMax" -"approx_top_sumArgMin" -"approx_top_sumArray" -"approx_top_sumDistinct" -"approx_top_sumForEach" -"approx_top_sumIf" -"approx_top_sumMap" -"approx_top_sumMerge" -"approx_top_sumNull" -"approx_top_sumOrDefault" -"approx_top_sumOrNull" -"approx_top_sumResample" -"approx_top_sumSimpleState" -"approx_top_sumState" -"argMax" -"argMaxArgMax" -"argMaxArgMin" -"argMaxArray" -"argMaxDistinct" -"argMaxForEach" -"argMaxIf" -"argMaxMap" -"argMaxMerge" -"argMaxNull" -"argMaxOrDefault" -"argMaxOrNull" -"argMaxResample" -"argMaxSimpleState" -"argMaxState" -"argMin" -"argMinArgMax" -"argMinArgMin" -"argMinArray" -"argMinDistinct" -"argMinForEach" -"argMinIf" -"argMinMap" -"argMinMerge" -"argMinNull" -"argMinOrDefault" -"argMinOrNull" -"argMinResample" -"argMinSimpleState" -"argMinState" -"array" -"arrayAUC" -"arrayAll" -"arrayAvg" -"arrayCompact" -"arrayConcat" -"arrayCount" -"arrayCumSum" -"arrayCumSumNonNegative" -"arrayDifference" -"arrayDistinct" -"arrayDotProduct" -"arrayElement" -"arrayEnumerate" -"arrayEnumerateDense" -"arrayEnumerateDenseRanked" -"arrayEnumerateUniq" -"arrayEnumerateUniqRanked" -"arrayExists" -"arrayFill" -"arrayFilter" -"arrayFirst" -"arrayFirstIndex" -"arrayFirstOrNull" -"arrayFlatten" -"arrayFold" -"arrayIntersect" -"arrayJaccardIndex" -"arrayJoin" -"arrayLast" -"arrayLastIndex" -"arrayLastOrNull" -"arrayMap" -"arrayMax" -"arrayMin" -"arrayPartialReverseSort" -"arrayPartialShuffle" -"arrayPartialSort" -"arrayPopBack" -"arrayPopFront" -"arrayProduct" -"arrayPushBack" -"arrayPushFront" -"arrayRandomSample" -"arrayReduce" -"arrayReduceInRanges" -"arrayResize" -"arrayReverse" -"arrayReverseFill" -"arrayReverseSort" -"arrayReverseSplit" -"arrayRotateLeft" -"arrayRotateRight" -"arrayShiftLeft" -"arrayShiftRight" -"arrayShingles" -"arrayShuffle" -"arraySlice" -"arraySort" -"arraySplit" -"arrayStringConcat" -"arraySum" -"arrayUniq" -"arrayWithConstant" -"arrayZip" -"array_agg" -"array_aggArgMax" -"array_aggArgMin" -"array_aggArray" -"array_aggDistinct" -"array_aggForEach" -"array_aggIf" -"array_aggMap" -"array_aggMerge" -"array_aggNull" -"array_aggOrDefault" -"array_aggOrNull" -"array_aggResample" -"array_aggSimpleState" -"array_aggState" -"array_concat_agg" -"array_concat_aggArgMax" -"array_concat_aggArgMin" -"array_concat_aggArray" -"array_concat_aggDistinct" -"array_concat_aggForEach" -"array_concat_aggIf" -"array_concat_aggMap" -"array_concat_aggMerge" -"array_concat_aggNull" -"array_concat_aggOrDefault" -"array_concat_aggOrNull" -"array_concat_aggResample" -"array_concat_aggSimpleState" -"array_concat_aggState" -"ascii" -"asin" -"asinh" -"assumeNotNull" -"atan" -"atan2" -"atanh" -"avg" -"avgArgMax" -"avgArgMin" -"avgArray" -"avgDistinct" -"avgForEach" -"avgIf" -"avgMap" -"avgMerge" -"avgNull" -"avgOrDefault" -"avgOrNull" -"avgResample" -"avgSimpleState" -"avgState" -"avgWeighted" -"avgWeightedArgMax" -"avgWeightedArgMin" -"avgWeightedArray" -"avgWeightedDistinct" -"avgWeightedForEach" -"avgWeightedIf" -"avgWeightedMap" -"avgWeightedMerge" -"avgWeightedNull" -"avgWeightedOrDefault" -"avgWeightedOrNull" -"avgWeightedResample" -"avgWeightedSimpleState" -"avgWeightedState" -"bar" -"base58Decode" -"base58Encode" -"base64Decode" -"base64Encode" -"base64URLDecode" -"base64URLEncode" -"basename" -"bin" -"bitAnd" -"bitCount" -"bitHammingDistance" -"bitNot" -"bitOr" -"bitPositionsToArray" -"bitRotateLeft" -"bitRotateRight" -"bitShiftLeft" -"bitShiftRight" -"bitSlice" -"bitTest" -"bitTestAll" -"bitTestAny" -"bitXor" -"bitmapAnd" -"bitmapAndCardinality" -"bitmapAndnot" -"bitmapAndnotCardinality" -"bitmapBuild" -"bitmapCardinality" -"bitmapContains" -"bitmapHasAll" -"bitmapHasAny" -"bitmapMax" -"bitmapMin" -"bitmapOr" -"bitmapOrCardinality" -"bitmapSubsetInRange" -"bitmapSubsetLimit" -"bitmapToArray" -"bitmapTransform" -"bitmapXor" -"bitmapXorCardinality" -"bitmaskToArray" -"bitmaskToList" -"blockNumber" -"blockSerializedSize" -"blockSize" -"boundingRatio" -"boundingRatioArgMax" -"boundingRatioArgMin" -"boundingRatioArray" -"boundingRatioDistinct" -"boundingRatioForEach" -"boundingRatioIf" -"boundingRatioMap" -"boundingRatioMerge" -"boundingRatioNull" -"boundingRatioOrDefault" -"boundingRatioOrNull" -"boundingRatioResample" -"boundingRatioSimpleState" -"boundingRatioState" -"buildId" -"byteHammingDistance" -"byteSize" -"byteSlice" -"byteSwap" -"caseWithExpr" -"caseWithExpression" -"caseWithoutExpr" -"caseWithoutExpression" -"catboostEvaluate" -"categoricalInformationValue" -"categoricalInformationValueArgMax" -"categoricalInformationValueArgMin" -"categoricalInformationValueArray" -"categoricalInformationValueDistinct" -"categoricalInformationValueForEach" -"categoricalInformationValueIf" -"categoricalInformationValueMap" -"categoricalInformationValueMerge" -"categoricalInformationValueNull" -"categoricalInformationValueOrDefault" -"categoricalInformationValueOrNull" -"categoricalInformationValueResample" -"categoricalInformationValueSimpleState" -"categoricalInformationValueState" -"cbrt" -"ceil" -"ceiling" -"changeDay" -"changeHour" -"changeMinute" -"changeMonth" -"changeSecond" -"changeYear" -"char" -"cityHash64" -"clamp" -"coalesce" -"concat" -"concatAssumeInjective" -"concatWithSeparator" -"concatWithSeparatorAssumeInjective" -"concat_ws" -"connectionId" -"connection_id" -"contingency" -"contingencyArgMax" -"contingencyArgMin" -"contingencyArray" -"contingencyDistinct" -"contingencyForEach" -"contingencyIf" -"contingencyMap" -"contingencyMerge" -"contingencyNull" -"contingencyOrDefault" -"contingencyOrNull" -"contingencyResample" -"contingencySimpleState" -"contingencyState" -"convertCharset" -"corr" -"corrArgMax" -"corrArgMin" -"corrArray" -"corrDistinct" -"corrForEach" -"corrIf" -"corrMap" -"corrMatrix" -"corrMatrixArgMax" -"corrMatrixArgMin" -"corrMatrixArray" -"corrMatrixDistinct" -"corrMatrixForEach" -"corrMatrixIf" -"corrMatrixMap" -"corrMatrixMerge" -"corrMatrixNull" -"corrMatrixOrDefault" -"corrMatrixOrNull" -"corrMatrixResample" -"corrMatrixSimpleState" -"corrMatrixState" -"corrMerge" -"corrNull" -"corrOrDefault" -"corrOrNull" -"corrResample" -"corrSimpleState" -"corrStable" -"corrStableArgMax" -"corrStableArgMin" -"corrStableArray" -"corrStableDistinct" -"corrStableForEach" -"corrStableIf" -"corrStableMap" -"corrStableMerge" -"corrStableNull" -"corrStableOrDefault" -"corrStableOrNull" -"corrStableResample" -"corrStableSimpleState" -"corrStableState" -"corrState" -"cos" -"cosh" -"cosineDistance" -"count" -"countArgMax" -"countArgMin" -"countArray" -"countDigits" -"countDistinct" -"countEqual" -"countForEach" -"countIf" -"countMap" -"countMatches" -"countMatchesCaseInsensitive" -"countMerge" -"countNull" -"countOrDefault" -"countOrNull" -"countResample" -"countSimpleState" -"countState" -"countSubstrings" -"countSubstringsCaseInsensitive" -"countSubstringsCaseInsensitiveUTF8" -"covarPop" -"covarPopArgMax" -"covarPopArgMin" -"covarPopArray" -"covarPopDistinct" -"covarPopForEach" -"covarPopIf" -"covarPopMap" -"covarPopMatrix" -"covarPopMatrixArgMax" -"covarPopMatrixArgMin" -"covarPopMatrixArray" -"covarPopMatrixDistinct" -"covarPopMatrixForEach" -"covarPopMatrixIf" -"covarPopMatrixMap" -"covarPopMatrixMerge" -"covarPopMatrixNull" -"covarPopMatrixOrDefault" -"covarPopMatrixOrNull" -"covarPopMatrixResample" -"covarPopMatrixSimpleState" -"covarPopMatrixState" -"covarPopMerge" -"covarPopNull" -"covarPopOrDefault" -"covarPopOrNull" -"covarPopResample" -"covarPopSimpleState" -"covarPopStable" -"covarPopStableArgMax" -"covarPopStableArgMin" -"covarPopStableArray" -"covarPopStableDistinct" -"covarPopStableForEach" -"covarPopStableIf" -"covarPopStableMap" -"covarPopStableMerge" -"covarPopStableNull" -"covarPopStableOrDefault" -"covarPopStableOrNull" -"covarPopStableResample" -"covarPopStableSimpleState" -"covarPopStableState" -"covarPopState" -"covarSamp" -"covarSampArgMax" -"covarSampArgMin" -"covarSampArray" -"covarSampDistinct" -"covarSampForEach" -"covarSampIf" -"covarSampMap" -"covarSampMatrix" -"covarSampMatrixArgMax" -"covarSampMatrixArgMin" -"covarSampMatrixArray" -"covarSampMatrixDistinct" -"covarSampMatrixForEach" -"covarSampMatrixIf" -"covarSampMatrixMap" -"covarSampMatrixMerge" -"covarSampMatrixNull" -"covarSampMatrixOrDefault" -"covarSampMatrixOrNull" -"covarSampMatrixResample" -"covarSampMatrixSimpleState" -"covarSampMatrixState" -"covarSampMerge" -"covarSampNull" -"covarSampOrDefault" -"covarSampOrNull" -"covarSampResample" -"covarSampSimpleState" -"covarSampStable" -"covarSampStableArgMax" -"covarSampStableArgMin" -"covarSampStableArray" -"covarSampStableDistinct" -"covarSampStableForEach" -"covarSampStableIf" -"covarSampStableMap" -"covarSampStableMerge" -"covarSampStableNull" -"covarSampStableOrDefault" -"covarSampStableOrNull" -"covarSampStableResample" -"covarSampStableSimpleState" -"covarSampStableState" -"covarSampState" -"cramersV" -"cramersVArgMax" -"cramersVArgMin" -"cramersVArray" -"cramersVBiasCorrected" -"cramersVBiasCorrectedArgMax" -"cramersVBiasCorrectedArgMin" -"cramersVBiasCorrectedArray" -"cramersVBiasCorrectedDistinct" -"cramersVBiasCorrectedForEach" -"cramersVBiasCorrectedIf" -"cramersVBiasCorrectedMap" -"cramersVBiasCorrectedMerge" -"cramersVBiasCorrectedNull" -"cramersVBiasCorrectedOrDefault" -"cramersVBiasCorrectedOrNull" -"cramersVBiasCorrectedResample" -"cramersVBiasCorrectedSimpleState" -"cramersVBiasCorrectedState" -"cramersVDistinct" -"cramersVForEach" -"cramersVIf" -"cramersVMap" -"cramersVMerge" -"cramersVNull" -"cramersVOrDefault" -"cramersVOrNull" -"cramersVResample" -"cramersVSimpleState" -"cramersVState" -"curdate" -"currentDatabase" -"currentProfiles" -"currentRoles" -"currentSchemas" -"currentUser" -"current_database" -"current_date" -"current_schemas" -"current_timestamp" -"current_user" -"cutFragment" -"cutIPv6" -"cutQueryString" -"cutQueryStringAndFragment" -"cutToFirstSignificantSubdomain" -"cutToFirstSignificantSubdomainCustom" -"cutToFirstSignificantSubdomainCustomRFC" -"cutToFirstSignificantSubdomainCustomWithWWW" -"cutToFirstSignificantSubdomainCustomWithWWWRFC" -"cutToFirstSignificantSubdomainRFC" -"cutToFirstSignificantSubdomainWithWWW" -"cutToFirstSignificantSubdomainWithWWWRFC" -"cutURLParameter" -"cutWWW" -"damerauLevenshteinDistance" -"dateDiff" -"dateName" -"dateTime64ToSnowflake" -"dateTime64ToSnowflakeID" -"dateTimeToSnowflake" -"dateTimeToSnowflakeID" -"dateTrunc" -"date_diff" -"decodeHTMLComponent" -"decodeURLComponent" -"decodeURLFormComponent" -"decodeXMLComponent" -"decrypt" -"defaultProfiles" -"defaultRoles" -"defaultValueOfArgumentType" -"defaultValueOfTypeName" -"degrees" -"deltaSum" -"deltaSumArgMax" -"deltaSumArgMin" -"deltaSumArray" -"deltaSumDistinct" -"deltaSumForEach" -"deltaSumIf" -"deltaSumMap" -"deltaSumMerge" -"deltaSumNull" -"deltaSumOrDefault" -"deltaSumOrNull" -"deltaSumResample" -"deltaSumSimpleState" -"deltaSumState" -"deltaSumTimestamp" -"deltaSumTimestampArgMax" -"deltaSumTimestampArgMin" -"deltaSumTimestampArray" -"deltaSumTimestampDistinct" -"deltaSumTimestampForEach" -"deltaSumTimestampIf" -"deltaSumTimestampMap" -"deltaSumTimestampMerge" -"deltaSumTimestampNull" -"deltaSumTimestampOrDefault" -"deltaSumTimestampOrNull" -"deltaSumTimestampResample" -"deltaSumTimestampSimpleState" -"deltaSumTimestampState" -"demangle" -"denseRank" -"denseRankArgMax" -"denseRankArgMin" -"denseRankArray" -"denseRankDistinct" -"denseRankForEach" -"denseRankIf" -"denseRankMap" -"denseRankMerge" -"denseRankNull" -"denseRankOrDefault" -"denseRankOrNull" -"denseRankResample" -"denseRankSimpleState" -"denseRankState" -"dense_rank" -"dense_rankArgMax" -"dense_rankArgMin" -"dense_rankArray" -"dense_rankDistinct" -"dense_rankForEach" -"dense_rankIf" -"dense_rankMap" -"dense_rankMerge" -"dense_rankNull" -"dense_rankOrDefault" -"dense_rankOrNull" -"dense_rankResample" -"dense_rankSimpleState" -"dense_rankState" -"detectCharset" -"detectLanguage" -"detectLanguageMixed" -"detectLanguageUnknown" -"detectProgrammingLanguage" -"detectTonality" -"dictGet" -"dictGetAll" -"dictGetChildren" -"dictGetDate" -"dictGetDateOrDefault" -"dictGetDateTime" -"dictGetDateTimeOrDefault" -"dictGetDescendants" -"dictGetFloat32" -"dictGetFloat32OrDefault" -"dictGetFloat64" -"dictGetFloat64OrDefault" -"dictGetHierarchy" -"dictGetIPv4" -"dictGetIPv4OrDefault" -"dictGetIPv6" -"dictGetIPv6OrDefault" -"dictGetInt16" -"dictGetInt16OrDefault" -"dictGetInt32" -"dictGetInt32OrDefault" -"dictGetInt64" -"dictGetInt64OrDefault" -"dictGetInt8" -"dictGetInt8OrDefault" -"dictGetOrDefault" -"dictGetOrNull" -"dictGetString" -"dictGetStringOrDefault" -"dictGetUInt16" -"dictGetUInt16OrDefault" -"dictGetUInt32" -"dictGetUInt32OrDefault" -"dictGetUInt64" -"dictGetUInt64OrDefault" -"dictGetUInt8" -"dictGetUInt8OrDefault" -"dictGetUUID" -"dictGetUUIDOrDefault" -"dictHas" -"dictIsIn" -"displayName" -"distanceL1" -"distanceL2" -"distanceL2Squared" -"distanceLinf" -"distanceLp" -"divide" -"divideDecimal" -"domain" -"domainRFC" -"domainWithoutWWW" -"domainWithoutWWWRFC" -"dotProduct" -"dumpColumnStructure" -"dynamicElement" -"dynamicType" -"e" -"editDistance" -"editDistanceUTF8" -"empty" -"emptyArrayDate" -"emptyArrayDateTime" -"emptyArrayFloat32" -"emptyArrayFloat64" -"emptyArrayInt16" -"emptyArrayInt32" -"emptyArrayInt64" -"emptyArrayInt8" -"emptyArrayString" -"emptyArrayToSingle" -"emptyArrayUInt16" -"emptyArrayUInt32" -"emptyArrayUInt64" -"emptyArrayUInt8" -"enabledProfiles" -"enabledRoles" -"encodeURLComponent" -"encodeURLFormComponent" -"encodeXMLComponent" -"encrypt" -"endsWith" -"endsWithUTF8" -"entropy" -"entropyArgMax" -"entropyArgMin" -"entropyArray" -"entropyDistinct" -"entropyForEach" -"entropyIf" -"entropyMap" -"entropyMerge" -"entropyNull" -"entropyOrDefault" -"entropyOrNull" -"entropyResample" -"entropySimpleState" -"entropyState" -"equals" -"erf" -"erfc" -"errorCodeToName" -"evalMLMethod" -"exp" -"exp10" -"exp2" -"exponentialMovingAverage" -"exponentialMovingAverageArgMax" -"exponentialMovingAverageArgMin" -"exponentialMovingAverageArray" -"exponentialMovingAverageDistinct" -"exponentialMovingAverageForEach" -"exponentialMovingAverageIf" -"exponentialMovingAverageMap" -"exponentialMovingAverageMerge" -"exponentialMovingAverageNull" -"exponentialMovingAverageOrDefault" -"exponentialMovingAverageOrNull" -"exponentialMovingAverageResample" -"exponentialMovingAverageSimpleState" -"exponentialMovingAverageState" -"exponentialTimeDecayedAvg" -"exponentialTimeDecayedAvgArgMax" -"exponentialTimeDecayedAvgArgMin" -"exponentialTimeDecayedAvgArray" -"exponentialTimeDecayedAvgDistinct" -"exponentialTimeDecayedAvgForEach" -"exponentialTimeDecayedAvgIf" -"exponentialTimeDecayedAvgMap" -"exponentialTimeDecayedAvgMerge" -"exponentialTimeDecayedAvgNull" -"exponentialTimeDecayedAvgOrDefault" -"exponentialTimeDecayedAvgOrNull" -"exponentialTimeDecayedAvgResample" -"exponentialTimeDecayedAvgSimpleState" -"exponentialTimeDecayedAvgState" -"exponentialTimeDecayedCount" -"exponentialTimeDecayedCountArgMax" -"exponentialTimeDecayedCountArgMin" -"exponentialTimeDecayedCountArray" -"exponentialTimeDecayedCountDistinct" -"exponentialTimeDecayedCountForEach" -"exponentialTimeDecayedCountIf" -"exponentialTimeDecayedCountMap" -"exponentialTimeDecayedCountMerge" -"exponentialTimeDecayedCountNull" -"exponentialTimeDecayedCountOrDefault" -"exponentialTimeDecayedCountOrNull" -"exponentialTimeDecayedCountResample" -"exponentialTimeDecayedCountSimpleState" -"exponentialTimeDecayedCountState" -"exponentialTimeDecayedMax" -"exponentialTimeDecayedMaxArgMax" -"exponentialTimeDecayedMaxArgMin" -"exponentialTimeDecayedMaxArray" -"exponentialTimeDecayedMaxDistinct" -"exponentialTimeDecayedMaxForEach" -"exponentialTimeDecayedMaxIf" -"exponentialTimeDecayedMaxMap" -"exponentialTimeDecayedMaxMerge" -"exponentialTimeDecayedMaxNull" -"exponentialTimeDecayedMaxOrDefault" -"exponentialTimeDecayedMaxOrNull" -"exponentialTimeDecayedMaxResample" -"exponentialTimeDecayedMaxSimpleState" -"exponentialTimeDecayedMaxState" -"exponentialTimeDecayedSum" -"exponentialTimeDecayedSumArgMax" -"exponentialTimeDecayedSumArgMin" -"exponentialTimeDecayedSumArray" -"exponentialTimeDecayedSumDistinct" -"exponentialTimeDecayedSumForEach" -"exponentialTimeDecayedSumIf" -"exponentialTimeDecayedSumMap" -"exponentialTimeDecayedSumMerge" -"exponentialTimeDecayedSumNull" -"exponentialTimeDecayedSumOrDefault" -"exponentialTimeDecayedSumOrNull" -"exponentialTimeDecayedSumResample" -"exponentialTimeDecayedSumSimpleState" -"exponentialTimeDecayedSumState" -"extract" -"extractAll" -"extractAllGroups" -"extractAllGroupsHorizontal" -"extractAllGroupsVertical" -"extractGroups" -"extractKeyValuePairs" -"extractKeyValuePairsWithEscaping" -"extractTextFromHTML" -"extractURLParameter" -"extractURLParameterNames" -"extractURLParameters" -"factorial" -"farmFingerprint64" -"farmHash64" -"file" -"filesystemAvailable" -"filesystemCapacity" -"filesystemUnreserved" -"finalizeAggregation" -"firstLine" -"firstSignificantSubdomain" -"firstSignificantSubdomainCustom" -"firstSignificantSubdomainCustomRFC" -"firstSignificantSubdomainRFC" -"first_value" -"first_valueArgMax" -"first_valueArgMin" -"first_valueArray" -"first_valueDistinct" -"first_valueForEach" -"first_valueIf" -"first_valueMap" -"first_valueMerge" -"first_valueNull" -"first_valueOrDefault" -"first_valueOrNull" -"first_valueResample" -"first_valueSimpleState" -"first_valueState" -"first_value_respect_nulls" -"first_value_respect_nullsArgMax" -"first_value_respect_nullsArgMin" -"first_value_respect_nullsArray" -"first_value_respect_nullsDistinct" -"first_value_respect_nullsForEach" -"first_value_respect_nullsIf" -"first_value_respect_nullsMap" -"first_value_respect_nullsMerge" -"first_value_respect_nullsNull" -"first_value_respect_nullsOrDefault" -"first_value_respect_nullsOrNull" -"first_value_respect_nullsResample" -"first_value_respect_nullsSimpleState" -"first_value_respect_nullsState" -"flameGraph" -"flameGraphArgMax" -"flameGraphArgMin" -"flameGraphArray" -"flameGraphDistinct" -"flameGraphForEach" -"flameGraphIf" -"flameGraphMap" -"flameGraphMerge" -"flameGraphNull" -"flameGraphOrDefault" -"flameGraphOrNull" -"flameGraphResample" -"flameGraphSimpleState" -"flameGraphState" -"flatten" -"flattenTuple" -"floor" -"format" -"formatDateTime" -"formatDateTimeInJodaSyntax" -"formatQuery" -"formatQueryOrNull" -"formatQuerySingleLine" -"formatQuerySingleLineOrNull" -"formatReadableDecimalSize" -"formatReadableQuantity" -"formatReadableSize" -"formatReadableTimeDelta" -"formatRow" -"formatRowNoNewline" -"fragment" -"fromDaysSinceYearZero" -"fromDaysSinceYearZero32" -"fromModifiedJulianDay" -"fromModifiedJulianDayOrNull" -"fromUTCTimestamp" -"fromUnixTimestamp" -"fromUnixTimestamp64Micro" -"fromUnixTimestamp64Milli" -"fromUnixTimestamp64Nano" -"fromUnixTimestampInJodaSyntax" -"from_utc_timestamp" -"fullHostName" -"fuzzBits" -"gccMurmurHash" -"gcd" -"generateRandomStructure" -"generateSnowflakeID" -"generateULID" -"generateUUIDv4" -"generateUUIDv7" -"geoDistance" -"geoToH3" -"geoToS2" -"geohashDecode" -"geohashEncode" -"geohashesInBox" -"getClientHTTPHeader" -"getMacro" -"getOSKernelVersion" -"getServerPort" -"getSetting" -"getSizeOfEnumType" -"getSubcolumn" -"getTypeSerializationStreams" -"globalIn" -"globalInIgnoreSet" -"globalNotIn" -"globalNotInIgnoreSet" -"globalNotNullIn" -"globalNotNullInIgnoreSet" -"globalNullIn" -"globalNullInIgnoreSet" -"globalVariable" -"greatCircleAngle" -"greatCircleDistance" -"greater" -"greaterOrEquals" -"greatest" -"groupArray" -"groupArrayArgMax" -"groupArrayArgMin" -"groupArrayArray" -"groupArrayDistinct" -"groupArrayForEach" -"groupArrayIf" -"groupArrayInsertAt" -"groupArrayInsertAtArgMax" -"groupArrayInsertAtArgMin" -"groupArrayInsertAtArray" -"groupArrayInsertAtDistinct" -"groupArrayInsertAtForEach" -"groupArrayInsertAtIf" -"groupArrayInsertAtMap" -"groupArrayInsertAtMerge" -"groupArrayInsertAtNull" -"groupArrayInsertAtOrDefault" -"groupArrayInsertAtOrNull" -"groupArrayInsertAtResample" -"groupArrayInsertAtSimpleState" -"groupArrayInsertAtState" -"groupArrayIntersect" -"groupArrayIntersectArgMax" -"groupArrayIntersectArgMin" -"groupArrayIntersectArray" -"groupArrayIntersectDistinct" -"groupArrayIntersectForEach" -"groupArrayIntersectIf" -"groupArrayIntersectMap" -"groupArrayIntersectMerge" -"groupArrayIntersectNull" -"groupArrayIntersectOrDefault" -"groupArrayIntersectOrNull" -"groupArrayIntersectResample" -"groupArrayIntersectSimpleState" -"groupArrayIntersectState" -"groupArrayLast" -"groupArrayLastArgMax" -"groupArrayLastArgMin" -"groupArrayLastArray" -"groupArrayLastDistinct" -"groupArrayLastForEach" -"groupArrayLastIf" -"groupArrayLastMap" -"groupArrayLastMerge" -"groupArrayLastNull" -"groupArrayLastOrDefault" -"groupArrayLastOrNull" -"groupArrayLastResample" -"groupArrayLastSimpleState" -"groupArrayLastState" -"groupArrayMap" -"groupArrayMerge" -"groupArrayMovingAvg" -"groupArrayMovingAvgArgMax" -"groupArrayMovingAvgArgMin" -"groupArrayMovingAvgArray" -"groupArrayMovingAvgDistinct" -"groupArrayMovingAvgForEach" -"groupArrayMovingAvgIf" -"groupArrayMovingAvgMap" -"groupArrayMovingAvgMerge" -"groupArrayMovingAvgNull" -"groupArrayMovingAvgOrDefault" -"groupArrayMovingAvgOrNull" -"groupArrayMovingAvgResample" -"groupArrayMovingAvgSimpleState" -"groupArrayMovingAvgState" -"groupArrayMovingSum" -"groupArrayMovingSumArgMax" -"groupArrayMovingSumArgMin" -"groupArrayMovingSumArray" -"groupArrayMovingSumDistinct" -"groupArrayMovingSumForEach" -"groupArrayMovingSumIf" -"groupArrayMovingSumMap" -"groupArrayMovingSumMerge" -"groupArrayMovingSumNull" -"groupArrayMovingSumOrDefault" -"groupArrayMovingSumOrNull" -"groupArrayMovingSumResample" -"groupArrayMovingSumSimpleState" -"groupArrayMovingSumState" -"groupArrayNull" -"groupArrayOrDefault" -"groupArrayOrNull" -"groupArrayResample" -"groupArraySample" -"groupArraySampleArgMax" -"groupArraySampleArgMin" -"groupArraySampleArray" -"groupArraySampleDistinct" -"groupArraySampleForEach" -"groupArraySampleIf" -"groupArraySampleMap" -"groupArraySampleMerge" -"groupArraySampleNull" -"groupArraySampleOrDefault" -"groupArraySampleOrNull" -"groupArraySampleResample" -"groupArraySampleSimpleState" -"groupArraySampleState" -"groupArraySimpleState" -"groupArraySorted" -"groupArraySortedArgMax" -"groupArraySortedArgMin" -"groupArraySortedArray" -"groupArraySortedDistinct" -"groupArraySortedForEach" -"groupArraySortedIf" -"groupArraySortedMap" -"groupArraySortedMerge" -"groupArraySortedNull" -"groupArraySortedOrDefault" -"groupArraySortedOrNull" -"groupArraySortedResample" -"groupArraySortedSimpleState" -"groupArraySortedState" -"groupArrayState" -"groupBitAnd" -"groupBitAndArgMax" -"groupBitAndArgMin" -"groupBitAndArray" -"groupBitAndDistinct" -"groupBitAndForEach" -"groupBitAndIf" -"groupBitAndMap" -"groupBitAndMerge" -"groupBitAndNull" -"groupBitAndOrDefault" -"groupBitAndOrNull" -"groupBitAndResample" -"groupBitAndSimpleState" -"groupBitAndState" -"groupBitOr" -"groupBitOrArgMax" -"groupBitOrArgMin" -"groupBitOrArray" -"groupBitOrDistinct" -"groupBitOrForEach" -"groupBitOrIf" -"groupBitOrMap" -"groupBitOrMerge" -"groupBitOrNull" -"groupBitOrOrDefault" -"groupBitOrOrNull" -"groupBitOrResample" -"groupBitOrSimpleState" -"groupBitOrState" -"groupBitXor" -"groupBitXorArgMax" -"groupBitXorArgMin" -"groupBitXorArray" -"groupBitXorDistinct" -"groupBitXorForEach" -"groupBitXorIf" -"groupBitXorMap" -"groupBitXorMerge" -"groupBitXorNull" -"groupBitXorOrDefault" -"groupBitXorOrNull" -"groupBitXorResample" -"groupBitXorSimpleState" -"groupBitXorState" -"groupBitmap" -"groupBitmapAnd" -"groupBitmapAndArgMax" -"groupBitmapAndArgMin" -"groupBitmapAndArray" -"groupBitmapAndDistinct" -"groupBitmapAndForEach" -"groupBitmapAndIf" -"groupBitmapAndMap" -"groupBitmapAndMerge" -"groupBitmapAndNull" -"groupBitmapAndOrDefault" -"groupBitmapAndOrNull" -"groupBitmapAndResample" -"groupBitmapAndSimpleState" -"groupBitmapAndState" -"groupBitmapArgMax" -"groupBitmapArgMin" -"groupBitmapArray" -"groupBitmapDistinct" -"groupBitmapForEach" -"groupBitmapIf" -"groupBitmapMap" -"groupBitmapMerge" -"groupBitmapNull" -"groupBitmapOr" -"groupBitmapOrArgMax" -"groupBitmapOrArgMin" -"groupBitmapOrArray" -"groupBitmapOrDefault" -"groupBitmapOrDistinct" -"groupBitmapOrForEach" -"groupBitmapOrIf" -"groupBitmapOrMap" -"groupBitmapOrMerge" -"groupBitmapOrNull" -"groupBitmapOrNull" -"groupBitmapOrOrDefault" -"groupBitmapOrOrNull" -"groupBitmapOrResample" -"groupBitmapOrSimpleState" -"groupBitmapOrState" -"groupBitmapResample" -"groupBitmapSimpleState" -"groupBitmapState" -"groupBitmapXor" -"groupBitmapXorArgMax" -"groupBitmapXorArgMin" -"groupBitmapXorArray" -"groupBitmapXorDistinct" -"groupBitmapXorForEach" -"groupBitmapXorIf" -"groupBitmapXorMap" -"groupBitmapXorMerge" -"groupBitmapXorNull" -"groupBitmapXorOrDefault" -"groupBitmapXorOrNull" -"groupBitmapXorResample" -"groupBitmapXorSimpleState" -"groupBitmapXorState" -"groupConcat" -"groupConcatArgMax" -"groupConcatArgMin" -"groupConcatArray" -"groupConcatDistinct" -"groupConcatForEach" -"groupConcatIf" -"groupConcatMap" -"groupConcatMerge" -"groupConcatNull" -"groupConcatOrDefault" -"groupConcatOrNull" -"groupConcatResample" -"groupConcatSimpleState" -"groupConcatState" -"groupUniqArray" -"groupUniqArrayArgMax" -"groupUniqArrayArgMin" -"groupUniqArrayArray" -"groupUniqArrayDistinct" -"groupUniqArrayForEach" -"groupUniqArrayIf" -"groupUniqArrayMap" -"groupUniqArrayMerge" -"groupUniqArrayNull" -"groupUniqArrayOrDefault" -"groupUniqArrayOrNull" -"groupUniqArrayResample" -"groupUniqArraySimpleState" -"groupUniqArrayState" -"group_concat" -"group_concatArgMax" -"group_concatArgMin" -"group_concatArray" -"group_concatDistinct" -"group_concatForEach" -"group_concatIf" -"group_concatMap" -"group_concatMerge" -"group_concatNull" -"group_concatOrDefault" -"group_concatOrNull" -"group_concatResample" -"group_concatSimpleState" -"group_concatState" -"h3CellAreaM2" -"h3CellAreaRads2" -"h3Distance" -"h3EdgeAngle" -"h3EdgeLengthKm" -"h3EdgeLengthM" -"h3ExactEdgeLengthKm" -"h3ExactEdgeLengthM" -"h3ExactEdgeLengthRads" -"h3GetBaseCell" -"h3GetDestinationIndexFromUnidirectionalEdge" -"h3GetFaces" -"h3GetIndexesFromUnidirectionalEdge" -"h3GetOriginIndexFromUnidirectionalEdge" -"h3GetPentagonIndexes" -"h3GetRes0Indexes" -"h3GetResolution" -"h3GetUnidirectionalEdge" -"h3GetUnidirectionalEdgeBoundary" -"h3GetUnidirectionalEdgesFromHexagon" -"h3HexAreaKm2" -"h3HexAreaM2" -"h3HexRing" -"h3IndexesAreNeighbors" -"h3IsPentagon" -"h3IsResClassIII" -"h3IsValid" -"h3Line" -"h3NumHexagons" -"h3PointDistKm" -"h3PointDistM" -"h3PointDistRads" -"h3ToCenterChild" -"h3ToChildren" -"h3ToGeo" -"h3ToGeoBoundary" -"h3ToParent" -"h3ToString" -"h3UnidirectionalEdgeIsValid" -"h3kRing" -"halfMD5" -"has" -"hasAll" -"hasAny" -"hasColumnInTable" -"hasSubsequence" -"hasSubsequenceCaseInsensitive" -"hasSubsequenceCaseInsensitiveUTF8" -"hasSubsequenceUTF8" -"hasSubstr" -"hasThreadFuzzer" -"hasToken" -"hasTokenCaseInsensitive" -"hasTokenCaseInsensitiveOrNull" -"hasTokenOrNull" -"hex" -"hilbertDecode" -"hilbertEncode" -"histogram" -"histogramArgMax" -"histogramArgMin" -"histogramArray" -"histogramDistinct" -"histogramForEach" -"histogramIf" -"histogramMap" -"histogramMerge" -"histogramNull" -"histogramOrDefault" -"histogramOrNull" -"histogramResample" -"histogramSimpleState" -"histogramState" -"hiveHash" -"hop" -"hopEnd" -"hopStart" -"hostName" -"hostname" -"hypot" -"identity" -"idnaDecode" -"idnaEncode" -"if" -"ifNotFinite" -"ifNull" -"ignore" -"ilike" -"in" -"inIgnoreSet" -"indexHint" -"indexOf" -"initcap" -"initcapUTF8" -"initialQueryID" -"initial_query_id" -"initializeAggregation" -"instr" -"intDiv" -"intDivOrZero" -"intExp10" -"intExp2" -"intHash32" -"intHash64" -"intervalLengthSum" -"intervalLengthSumArgMax" -"intervalLengthSumArgMin" -"intervalLengthSumArray" -"intervalLengthSumDistinct" -"intervalLengthSumForEach" -"intervalLengthSumIf" -"intervalLengthSumMap" -"intervalLengthSumMerge" -"intervalLengthSumNull" -"intervalLengthSumOrDefault" -"intervalLengthSumOrNull" -"intervalLengthSumResample" -"intervalLengthSumSimpleState" -"intervalLengthSumState" -"isConstant" -"isDecimalOverflow" -"isFinite" -"isIPAddressInRange" -"isIPv4String" -"isIPv6String" -"isInfinite" -"isNaN" -"isNotDistinctFrom" -"isNotNull" -"isNull" -"isNullable" -"isValidJSON" -"isValidUTF8" -"isZeroOrNull" -"jaroSimilarity" -"jaroWinklerSimilarity" -"javaHash" -"javaHashUTF16LE" -"joinGet" -"joinGetOrNull" -"jsonMergePatch" -"jumpConsistentHash" -"kafkaMurmurHash" -"kolmogorovSmirnovTest" -"kolmogorovSmirnovTestArgMax" -"kolmogorovSmirnovTestArgMin" -"kolmogorovSmirnovTestArray" -"kolmogorovSmirnovTestDistinct" -"kolmogorovSmirnovTestForEach" -"kolmogorovSmirnovTestIf" -"kolmogorovSmirnovTestMap" -"kolmogorovSmirnovTestMerge" -"kolmogorovSmirnovTestNull" -"kolmogorovSmirnovTestOrDefault" -"kolmogorovSmirnovTestOrNull" -"kolmogorovSmirnovTestResample" -"kolmogorovSmirnovTestSimpleState" -"kolmogorovSmirnovTestState" -"kostikConsistentHash" -"kql_array_sort_asc" -"kql_array_sort_desc" -"kurtPop" -"kurtPopArgMax" -"kurtPopArgMin" -"kurtPopArray" -"kurtPopDistinct" -"kurtPopForEach" -"kurtPopIf" -"kurtPopMap" -"kurtPopMerge" -"kurtPopNull" -"kurtPopOrDefault" -"kurtPopOrNull" -"kurtPopResample" -"kurtPopSimpleState" -"kurtPopState" -"kurtSamp" -"kurtSampArgMax" -"kurtSampArgMin" -"kurtSampArray" -"kurtSampDistinct" -"kurtSampForEach" -"kurtSampIf" -"kurtSampMap" -"kurtSampMerge" -"kurtSampNull" -"kurtSampOrDefault" -"kurtSampOrNull" -"kurtSampResample" -"kurtSampSimpleState" -"kurtSampState" -"lagInFrame" -"lagInFrameArgMax" -"lagInFrameArgMin" -"lagInFrameArray" -"lagInFrameDistinct" -"lagInFrameForEach" -"lagInFrameIf" -"lagInFrameMap" -"lagInFrameMerge" -"lagInFrameNull" -"lagInFrameOrDefault" -"lagInFrameOrNull" -"lagInFrameResample" -"lagInFrameSimpleState" -"lagInFrameState" -"largestTriangleThreeBuckets" -"largestTriangleThreeBucketsArgMax" -"largestTriangleThreeBucketsArgMin" -"largestTriangleThreeBucketsArray" -"largestTriangleThreeBucketsDistinct" -"largestTriangleThreeBucketsForEach" -"largestTriangleThreeBucketsIf" -"largestTriangleThreeBucketsMap" -"largestTriangleThreeBucketsMerge" -"largestTriangleThreeBucketsNull" -"largestTriangleThreeBucketsOrDefault" -"largestTriangleThreeBucketsOrNull" -"largestTriangleThreeBucketsResample" -"largestTriangleThreeBucketsSimpleState" -"largestTriangleThreeBucketsState" -"last_value" -"last_valueArgMax" -"last_valueArgMin" -"last_valueArray" -"last_valueDistinct" -"last_valueForEach" -"last_valueIf" -"last_valueMap" -"last_valueMerge" -"last_valueNull" -"last_valueOrDefault" -"last_valueOrNull" -"last_valueResample" -"last_valueSimpleState" -"last_valueState" -"last_value_respect_nulls" -"last_value_respect_nullsArgMax" -"last_value_respect_nullsArgMin" -"last_value_respect_nullsArray" -"last_value_respect_nullsDistinct" -"last_value_respect_nullsForEach" -"last_value_respect_nullsIf" -"last_value_respect_nullsMap" -"last_value_respect_nullsMerge" -"last_value_respect_nullsNull" -"last_value_respect_nullsOrDefault" -"last_value_respect_nullsOrNull" -"last_value_respect_nullsResample" -"last_value_respect_nullsSimpleState" -"last_value_respect_nullsState" -"lcase" -"lcm" -"leadInFrame" -"leadInFrameArgMax" -"leadInFrameArgMin" -"leadInFrameArray" -"leadInFrameDistinct" -"leadInFrameForEach" -"leadInFrameIf" -"leadInFrameMap" -"leadInFrameMerge" -"leadInFrameNull" -"leadInFrameOrDefault" -"leadInFrameOrNull" -"leadInFrameResample" -"leadInFrameSimpleState" -"leadInFrameState" -"least" -"left" -"leftPad" -"leftPadUTF8" -"leftUTF8" -"lemmatize" -"length" -"lengthUTF8" -"less" -"lessOrEquals" -"levenshteinDistance" -"levenshteinDistanceUTF8" -"lgamma" -"like" -"ln" -"locate" -"log" -"log10" -"log1p" -"log2" -"logTrace" -"lowCardinalityIndices" -"lowCardinalityKeys" -"lower" -"lowerUTF8" -"lpad" -"ltrim" -"lttb" -"lttbArgMax" -"lttbArgMin" -"lttbArray" -"lttbDistinct" -"lttbForEach" -"lttbIf" -"lttbMap" -"lttbMerge" -"lttbNull" -"lttbOrDefault" -"lttbOrNull" -"lttbResample" -"lttbSimpleState" -"lttbState" -"makeDate" -"makeDate32" -"makeDateTime" -"makeDateTime64" -"mannWhitneyUTest" -"mannWhitneyUTestArgMax" -"mannWhitneyUTestArgMin" -"mannWhitneyUTestArray" -"mannWhitneyUTestDistinct" -"mannWhitneyUTestForEach" -"mannWhitneyUTestIf" -"mannWhitneyUTestMap" -"mannWhitneyUTestMerge" -"mannWhitneyUTestNull" -"mannWhitneyUTestOrDefault" -"mannWhitneyUTestOrNull" -"mannWhitneyUTestResample" -"mannWhitneyUTestSimpleState" -"mannWhitneyUTestState" -"map" -"mapAdd" -"mapAll" -"mapApply" -"mapConcat" -"mapContains" -"mapContainsKeyLike" -"mapExists" -"mapExtractKeyLike" -"mapFilter" -"mapFromArrays" -"mapFromString" -"mapKeys" -"mapPartialReverseSort" -"mapPartialSort" -"mapPopulateSeries" -"mapReverseSort" -"mapSort" -"mapSubtract" -"mapUpdate" -"mapValues" -"match" -"materialize" -"max" -"max2" -"maxArgMax" -"maxArgMin" -"maxArray" -"maxDistinct" -"maxForEach" -"maxIf" -"maxIntersections" -"maxIntersectionsArgMax" -"maxIntersectionsArgMin" -"maxIntersectionsArray" -"maxIntersectionsDistinct" -"maxIntersectionsForEach" -"maxIntersectionsIf" -"maxIntersectionsMap" -"maxIntersectionsMerge" -"maxIntersectionsNull" -"maxIntersectionsOrDefault" -"maxIntersectionsOrNull" -"maxIntersectionsPosition" -"maxIntersectionsPositionArgMax" -"maxIntersectionsPositionArgMin" -"maxIntersectionsPositionArray" -"maxIntersectionsPositionDistinct" -"maxIntersectionsPositionForEach" -"maxIntersectionsPositionIf" -"maxIntersectionsPositionMap" -"maxIntersectionsPositionMerge" -"maxIntersectionsPositionNull" -"maxIntersectionsPositionOrDefault" -"maxIntersectionsPositionOrNull" -"maxIntersectionsPositionResample" -"maxIntersectionsPositionSimpleState" -"maxIntersectionsPositionState" -"maxIntersectionsResample" -"maxIntersectionsSimpleState" -"maxIntersectionsState" -"maxMap" -"maxMappedArrays" -"maxMappedArraysArgMax" -"maxMappedArraysArgMin" -"maxMappedArraysArray" -"maxMappedArraysDistinct" -"maxMappedArraysForEach" -"maxMappedArraysIf" -"maxMappedArraysMap" -"maxMappedArraysMerge" -"maxMappedArraysNull" -"maxMappedArraysOrDefault" -"maxMappedArraysOrNull" -"maxMappedArraysResample" -"maxMappedArraysSimpleState" -"maxMappedArraysState" -"maxMerge" -"maxNull" -"maxOrDefault" -"maxOrNull" -"maxResample" -"maxSimpleState" -"maxState" -"meanZTest" -"meanZTestArgMax" -"meanZTestArgMin" -"meanZTestArray" -"meanZTestDistinct" -"meanZTestForEach" -"meanZTestIf" -"meanZTestMap" -"meanZTestMerge" -"meanZTestNull" -"meanZTestOrDefault" -"meanZTestOrNull" -"meanZTestResample" -"meanZTestSimpleState" -"meanZTestState" -"median" -"medianArgMax" -"medianArgMin" -"medianArray" -"medianBFloat16" -"medianBFloat16ArgMax" -"medianBFloat16ArgMin" -"medianBFloat16Array" -"medianBFloat16Distinct" -"medianBFloat16ForEach" -"medianBFloat16If" -"medianBFloat16Map" -"medianBFloat16Merge" -"medianBFloat16Null" -"medianBFloat16OrDefault" -"medianBFloat16OrNull" -"medianBFloat16Resample" -"medianBFloat16SimpleState" -"medianBFloat16State" -"medianBFloat16Weighted" -"medianBFloat16WeightedArgMax" -"medianBFloat16WeightedArgMin" -"medianBFloat16WeightedArray" -"medianBFloat16WeightedDistinct" -"medianBFloat16WeightedForEach" -"medianBFloat16WeightedIf" -"medianBFloat16WeightedMap" -"medianBFloat16WeightedMerge" -"medianBFloat16WeightedNull" -"medianBFloat16WeightedOrDefault" -"medianBFloat16WeightedOrNull" -"medianBFloat16WeightedResample" -"medianBFloat16WeightedSimpleState" -"medianBFloat16WeightedState" -"medianDD" -"medianDDArgMax" -"medianDDArgMin" -"medianDDArray" -"medianDDDistinct" -"medianDDForEach" -"medianDDIf" -"medianDDMap" -"medianDDMerge" -"medianDDNull" -"medianDDOrDefault" -"medianDDOrNull" -"medianDDResample" -"medianDDSimpleState" -"medianDDState" -"medianDeterministic" -"medianDeterministicArgMax" -"medianDeterministicArgMin" -"medianDeterministicArray" -"medianDeterministicDistinct" -"medianDeterministicForEach" -"medianDeterministicIf" -"medianDeterministicMap" -"medianDeterministicMerge" -"medianDeterministicNull" -"medianDeterministicOrDefault" -"medianDeterministicOrNull" -"medianDeterministicResample" -"medianDeterministicSimpleState" -"medianDeterministicState" -"medianDistinct" -"medianExact" -"medianExactArgMax" -"medianExactArgMin" -"medianExactArray" -"medianExactDistinct" -"medianExactForEach" -"medianExactHigh" -"medianExactHighArgMax" -"medianExactHighArgMin" -"medianExactHighArray" -"medianExactHighDistinct" -"medianExactHighForEach" -"medianExactHighIf" -"medianExactHighMap" -"medianExactHighMerge" -"medianExactHighNull" -"medianExactHighOrDefault" -"medianExactHighOrNull" -"medianExactHighResample" -"medianExactHighSimpleState" -"medianExactHighState" -"medianExactIf" -"medianExactLow" -"medianExactLowArgMax" -"medianExactLowArgMin" -"medianExactLowArray" -"medianExactLowDistinct" -"medianExactLowForEach" -"medianExactLowIf" -"medianExactLowMap" -"medianExactLowMerge" -"medianExactLowNull" -"medianExactLowOrDefault" -"medianExactLowOrNull" -"medianExactLowResample" -"medianExactLowSimpleState" -"medianExactLowState" -"medianExactMap" -"medianExactMerge" -"medianExactNull" -"medianExactOrDefault" -"medianExactOrNull" -"medianExactResample" -"medianExactSimpleState" -"medianExactState" -"medianExactWeighted" -"medianExactWeightedArgMax" -"medianExactWeightedArgMin" -"medianExactWeightedArray" -"medianExactWeightedDistinct" -"medianExactWeightedForEach" -"medianExactWeightedIf" -"medianExactWeightedMap" -"medianExactWeightedMerge" -"medianExactWeightedNull" -"medianExactWeightedOrDefault" -"medianExactWeightedOrNull" -"medianExactWeightedResample" -"medianExactWeightedSimpleState" -"medianExactWeightedState" -"medianForEach" -"medianGK" -"medianGKArgMax" -"medianGKArgMin" -"medianGKArray" -"medianGKDistinct" -"medianGKForEach" -"medianGKIf" -"medianGKMap" -"medianGKMerge" -"medianGKNull" -"medianGKOrDefault" -"medianGKOrNull" -"medianGKResample" -"medianGKSimpleState" -"medianGKState" -"medianIf" -"medianInterpolatedWeighted" -"medianInterpolatedWeightedArgMax" -"medianInterpolatedWeightedArgMin" -"medianInterpolatedWeightedArray" -"medianInterpolatedWeightedDistinct" -"medianInterpolatedWeightedForEach" -"medianInterpolatedWeightedIf" -"medianInterpolatedWeightedMap" -"medianInterpolatedWeightedMerge" -"medianInterpolatedWeightedNull" -"medianInterpolatedWeightedOrDefault" -"medianInterpolatedWeightedOrNull" -"medianInterpolatedWeightedResample" -"medianInterpolatedWeightedSimpleState" -"medianInterpolatedWeightedState" -"medianMap" -"medianMerge" -"medianNull" -"medianOrDefault" -"medianOrNull" -"medianResample" -"medianSimpleState" -"medianState" -"medianTDigest" -"medianTDigestArgMax" -"medianTDigestArgMin" -"medianTDigestArray" -"medianTDigestDistinct" -"medianTDigestForEach" -"medianTDigestIf" -"medianTDigestMap" -"medianTDigestMerge" -"medianTDigestNull" -"medianTDigestOrDefault" -"medianTDigestOrNull" -"medianTDigestResample" -"medianTDigestSimpleState" -"medianTDigestState" -"medianTDigestWeighted" -"medianTDigestWeightedArgMax" -"medianTDigestWeightedArgMin" -"medianTDigestWeightedArray" -"medianTDigestWeightedDistinct" -"medianTDigestWeightedForEach" -"medianTDigestWeightedIf" -"medianTDigestWeightedMap" -"medianTDigestWeightedMerge" -"medianTDigestWeightedNull" -"medianTDigestWeightedOrDefault" -"medianTDigestWeightedOrNull" -"medianTDigestWeightedResample" -"medianTDigestWeightedSimpleState" -"medianTDigestWeightedState" -"medianTiming" -"medianTimingArgMax" -"medianTimingArgMin" -"medianTimingArray" -"medianTimingDistinct" -"medianTimingForEach" -"medianTimingIf" -"medianTimingMap" -"medianTimingMerge" -"medianTimingNull" -"medianTimingOrDefault" -"medianTimingOrNull" -"medianTimingResample" -"medianTimingSimpleState" -"medianTimingState" -"medianTimingWeighted" -"medianTimingWeightedArgMax" -"medianTimingWeightedArgMin" -"medianTimingWeightedArray" -"medianTimingWeightedDistinct" -"medianTimingWeightedForEach" -"medianTimingWeightedIf" -"medianTimingWeightedMap" -"medianTimingWeightedMerge" -"medianTimingWeightedNull" -"medianTimingWeightedOrDefault" -"medianTimingWeightedOrNull" -"medianTimingWeightedResample" -"medianTimingWeightedSimpleState" -"medianTimingWeightedState" -"metroHash64" -"mid" -"min" -"min2" -"minArgMax" -"minArgMin" -"minArray" -"minDistinct" -"minForEach" -"minIf" -"minMap" -"minMappedArrays" -"minMappedArraysArgMax" -"minMappedArraysArgMin" -"minMappedArraysArray" -"minMappedArraysDistinct" -"minMappedArraysForEach" -"minMappedArraysIf" -"minMappedArraysMap" -"minMappedArraysMerge" -"minMappedArraysNull" -"minMappedArraysOrDefault" -"minMappedArraysOrNull" -"minMappedArraysResample" -"minMappedArraysSimpleState" -"minMappedArraysState" -"minMerge" -"minNull" -"minOrDefault" -"minOrNull" -"minResample" -"minSampleSizeContinous" -"minSampleSizeContinuous" -"minSampleSizeConversion" -"minSimpleState" -"minState" -"minus" -"mismatches" -"mod" -"modulo" -"moduloLegacy" -"moduloOrZero" -"monthName" -"mortonDecode" -"mortonEncode" -"multiFuzzyMatchAllIndices" -"multiFuzzyMatchAny" -"multiFuzzyMatchAnyIndex" -"multiIf" -"multiMatchAllIndices" -"multiMatchAny" -"multiMatchAnyIndex" -"multiSearchAllPositions" -"multiSearchAllPositionsCaseInsensitive" -"multiSearchAllPositionsCaseInsensitiveUTF8" -"multiSearchAllPositionsUTF8" -"multiSearchAny" -"multiSearchAnyCaseInsensitive" -"multiSearchAnyCaseInsensitiveUTF8" -"multiSearchAnyUTF8" -"multiSearchFirstIndex" -"multiSearchFirstIndexCaseInsensitive" -"multiSearchFirstIndexCaseInsensitiveUTF8" -"multiSearchFirstIndexUTF8" -"multiSearchFirstPosition" -"multiSearchFirstPositionCaseInsensitive" -"multiSearchFirstPositionCaseInsensitiveUTF8" -"multiSearchFirstPositionUTF8" -"multiply" -"multiplyDecimal" -"murmurHash2_32" -"murmurHash2_64" -"murmurHash3_128" -"murmurHash3_32" -"murmurHash3_64" -"negate" -"neighbor" -"nested" -"netloc" -"ngramDistance" -"ngramDistanceCaseInsensitive" -"ngramDistanceCaseInsensitiveUTF8" -"ngramDistanceUTF8" -"ngramMinHash" -"ngramMinHashArg" -"ngramMinHashArgCaseInsensitive" -"ngramMinHashArgCaseInsensitiveUTF8" -"ngramMinHashArgUTF8" -"ngramMinHashCaseInsensitive" -"ngramMinHashCaseInsensitiveUTF8" -"ngramMinHashUTF8" -"ngramSearch" -"ngramSearchCaseInsensitive" -"ngramSearchCaseInsensitiveUTF8" -"ngramSearchUTF8" -"ngramSimHash" -"ngramSimHashCaseInsensitive" -"ngramSimHashCaseInsensitiveUTF8" -"ngramSimHashUTF8" -"ngrams" -"nonNegativeDerivative" -"nonNegativeDerivativeArgMax" -"nonNegativeDerivativeArgMin" -"nonNegativeDerivativeArray" -"nonNegativeDerivativeDistinct" -"nonNegativeDerivativeForEach" -"nonNegativeDerivativeIf" -"nonNegativeDerivativeMap" -"nonNegativeDerivativeMerge" -"nonNegativeDerivativeNull" -"nonNegativeDerivativeOrDefault" -"nonNegativeDerivativeOrNull" -"nonNegativeDerivativeResample" -"nonNegativeDerivativeSimpleState" -"nonNegativeDerivativeState" -"normL1" -"normL2" -"normL2Squared" -"normLinf" -"normLp" -"normalizeL1" -"normalizeL2" -"normalizeLinf" -"normalizeLp" -"normalizeQuery" -"normalizeQueryKeepNames" -"normalizeUTF8NFC" -"normalizeUTF8NFD" -"normalizeUTF8NFKC" -"normalizeUTF8NFKD" -"normalizedQueryHash" -"normalizedQueryHashKeepNames" -"not" -"notEmpty" -"notEquals" -"notILike" -"notIn" -"notInIgnoreSet" -"notLike" -"notNullIn" -"notNullInIgnoreSet" -"nothing" -"nothingArgMax" -"nothingArgMin" -"nothingArray" -"nothingDistinct" -"nothingForEach" -"nothingIf" -"nothingMap" -"nothingMerge" -"nothingNull" -"nothingNull" -"nothingNullArgMax" -"nothingNullArgMin" -"nothingNullArray" -"nothingNullDistinct" -"nothingNullForEach" -"nothingNullIf" -"nothingNullMap" -"nothingNullMerge" -"nothingNullNull" -"nothingNullOrDefault" -"nothingNullOrNull" -"nothingNullResample" -"nothingNullSimpleState" -"nothingNullState" -"nothingOrDefault" -"nothingOrNull" -"nothingResample" -"nothingSimpleState" -"nothingState" -"nothingUInt64" -"nothingUInt64ArgMax" -"nothingUInt64ArgMin" -"nothingUInt64Array" -"nothingUInt64Distinct" -"nothingUInt64ForEach" -"nothingUInt64If" -"nothingUInt64Map" -"nothingUInt64Merge" -"nothingUInt64Null" -"nothingUInt64OrDefault" -"nothingUInt64OrNull" -"nothingUInt64Resample" -"nothingUInt64SimpleState" -"nothingUInt64State" -"now" -"now64" -"nowInBlock" -"nth_value" -"nth_valueArgMax" -"nth_valueArgMin" -"nth_valueArray" -"nth_valueDistinct" -"nth_valueForEach" -"nth_valueIf" -"nth_valueMap" -"nth_valueMerge" -"nth_valueNull" -"nth_valueOrDefault" -"nth_valueOrNull" -"nth_valueResample" -"nth_valueSimpleState" -"nth_valueState" -"ntile" -"ntileArgMax" -"ntileArgMin" -"ntileArray" -"ntileDistinct" -"ntileForEach" -"ntileIf" -"ntileMap" -"ntileMerge" -"ntileNull" -"ntileOrDefault" -"ntileOrNull" -"ntileResample" -"ntileSimpleState" -"ntileState" -"nullIf" -"nullIn" -"nullInIgnoreSet" -"or" -"parseDateTime" -"parseDateTime32BestEffort" -"parseDateTime32BestEffortOrNull" -"parseDateTime32BestEffortOrZero" -"parseDateTime64BestEffort" -"parseDateTime64BestEffortOrNull" -"parseDateTime64BestEffortOrZero" -"parseDateTime64BestEffortUS" -"parseDateTime64BestEffortUSOrNull" -"parseDateTime64BestEffortUSOrZero" -"parseDateTimeBestEffort" -"parseDateTimeBestEffortOrNull" -"parseDateTimeBestEffortOrZero" -"parseDateTimeBestEffortUS" -"parseDateTimeBestEffortUSOrNull" -"parseDateTimeBestEffortUSOrZero" -"parseDateTimeInJodaSyntax" -"parseDateTimeInJodaSyntaxOrNull" -"parseDateTimeInJodaSyntaxOrZero" -"parseDateTimeOrNull" -"parseDateTimeOrZero" -"parseReadableSize" -"parseReadableSizeOrNull" -"parseReadableSizeOrZero" -"parseTimeDelta" -"partitionID" -"partitionId" -"path" -"pathFull" -"percentRank" -"percentRankArgMax" -"percentRankArgMin" -"percentRankArray" -"percentRankDistinct" -"percentRankForEach" -"percentRankIf" -"percentRankMap" -"percentRankMerge" -"percentRankNull" -"percentRankOrDefault" -"percentRankOrNull" -"percentRankResample" -"percentRankSimpleState" -"percentRankState" -"percent_rank" -"percent_rankArgMax" -"percent_rankArgMin" -"percent_rankArray" -"percent_rankDistinct" -"percent_rankForEach" -"percent_rankIf" -"percent_rankMap" -"percent_rankMerge" -"percent_rankNull" -"percent_rankOrDefault" -"percent_rankOrNull" -"percent_rankResample" -"percent_rankSimpleState" -"percent_rankState" -"pi" -"plus" -"pmod" -"pointInEllipses" -"pointInPolygon" -"polygonAreaCartesian" -"polygonAreaSpherical" -"polygonConvexHullCartesian" -"polygonPerimeterCartesian" -"polygonPerimeterSpherical" -"polygonsDistanceCartesian" -"polygonsDistanceSpherical" -"polygonsEqualsCartesian" -"polygonsIntersectionCartesian" -"polygonsIntersectionSpherical" -"polygonsSymDifferenceCartesian" -"polygonsSymDifferenceSpherical" -"polygonsUnionCartesian" -"polygonsUnionSpherical" -"polygonsWithinCartesian" -"polygonsWithinSpherical" -"port" -"portRFC" -"position" -"positionCaseInsensitive" -"positionCaseInsensitiveUTF8" -"positionUTF8" -"positiveModulo" -"positive_modulo" -"pow" -"power" -"printf" -"proportionsZTest" -"protocol" -"punycodeDecode" -"punycodeEncode" -"quantile" -"quantileArgMax" -"quantileArgMin" -"quantileArray" -"quantileBFloat16" -"quantileBFloat16ArgMax" -"quantileBFloat16ArgMin" -"quantileBFloat16Array" -"quantileBFloat16Distinct" -"quantileBFloat16ForEach" -"quantileBFloat16If" -"quantileBFloat16Map" -"quantileBFloat16Merge" -"quantileBFloat16Null" -"quantileBFloat16OrDefault" -"quantileBFloat16OrNull" -"quantileBFloat16Resample" -"quantileBFloat16SimpleState" -"quantileBFloat16State" -"quantileBFloat16Weighted" -"quantileBFloat16WeightedArgMax" -"quantileBFloat16WeightedArgMin" -"quantileBFloat16WeightedArray" -"quantileBFloat16WeightedDistinct" -"quantileBFloat16WeightedForEach" -"quantileBFloat16WeightedIf" -"quantileBFloat16WeightedMap" -"quantileBFloat16WeightedMerge" -"quantileBFloat16WeightedNull" -"quantileBFloat16WeightedOrDefault" -"quantileBFloat16WeightedOrNull" -"quantileBFloat16WeightedResample" -"quantileBFloat16WeightedSimpleState" -"quantileBFloat16WeightedState" -"quantileDD" -"quantileDDArgMax" -"quantileDDArgMin" -"quantileDDArray" -"quantileDDDistinct" -"quantileDDForEach" -"quantileDDIf" -"quantileDDMap" -"quantileDDMerge" -"quantileDDNull" -"quantileDDOrDefault" -"quantileDDOrNull" -"quantileDDResample" -"quantileDDSimpleState" -"quantileDDState" -"quantileDeterministic" -"quantileDeterministicArgMax" -"quantileDeterministicArgMin" -"quantileDeterministicArray" -"quantileDeterministicDistinct" -"quantileDeterministicForEach" -"quantileDeterministicIf" -"quantileDeterministicMap" -"quantileDeterministicMerge" -"quantileDeterministicNull" -"quantileDeterministicOrDefault" -"quantileDeterministicOrNull" -"quantileDeterministicResample" -"quantileDeterministicSimpleState" -"quantileDeterministicState" -"quantileDistinct" -"quantileExact" -"quantileExactArgMax" -"quantileExactArgMin" -"quantileExactArray" -"quantileExactDistinct" -"quantileExactExclusive" -"quantileExactExclusiveArgMax" -"quantileExactExclusiveArgMin" -"quantileExactExclusiveArray" -"quantileExactExclusiveDistinct" -"quantileExactExclusiveForEach" -"quantileExactExclusiveIf" -"quantileExactExclusiveMap" -"quantileExactExclusiveMerge" -"quantileExactExclusiveNull" -"quantileExactExclusiveOrDefault" -"quantileExactExclusiveOrNull" -"quantileExactExclusiveResample" -"quantileExactExclusiveSimpleState" -"quantileExactExclusiveState" -"quantileExactForEach" -"quantileExactHigh" -"quantileExactHighArgMax" -"quantileExactHighArgMin" -"quantileExactHighArray" -"quantileExactHighDistinct" -"quantileExactHighForEach" -"quantileExactHighIf" -"quantileExactHighMap" -"quantileExactHighMerge" -"quantileExactHighNull" -"quantileExactHighOrDefault" -"quantileExactHighOrNull" -"quantileExactHighResample" -"quantileExactHighSimpleState" -"quantileExactHighState" -"quantileExactIf" -"quantileExactInclusive" -"quantileExactInclusiveArgMax" -"quantileExactInclusiveArgMin" -"quantileExactInclusiveArray" -"quantileExactInclusiveDistinct" -"quantileExactInclusiveForEach" -"quantileExactInclusiveIf" -"quantileExactInclusiveMap" -"quantileExactInclusiveMerge" -"quantileExactInclusiveNull" -"quantileExactInclusiveOrDefault" -"quantileExactInclusiveOrNull" -"quantileExactInclusiveResample" -"quantileExactInclusiveSimpleState" -"quantileExactInclusiveState" -"quantileExactLow" -"quantileExactLowArgMax" -"quantileExactLowArgMin" -"quantileExactLowArray" -"quantileExactLowDistinct" -"quantileExactLowForEach" -"quantileExactLowIf" -"quantileExactLowMap" -"quantileExactLowMerge" -"quantileExactLowNull" -"quantileExactLowOrDefault" -"quantileExactLowOrNull" -"quantileExactLowResample" -"quantileExactLowSimpleState" -"quantileExactLowState" -"quantileExactMap" -"quantileExactMerge" -"quantileExactNull" -"quantileExactOrDefault" -"quantileExactOrNull" -"quantileExactResample" -"quantileExactSimpleState" -"quantileExactState" -"quantileExactWeighted" -"quantileExactWeightedArgMax" -"quantileExactWeightedArgMin" -"quantileExactWeightedArray" -"quantileExactWeightedDistinct" -"quantileExactWeightedForEach" -"quantileExactWeightedIf" -"quantileExactWeightedMap" -"quantileExactWeightedMerge" -"quantileExactWeightedNull" -"quantileExactWeightedOrDefault" -"quantileExactWeightedOrNull" -"quantileExactWeightedResample" -"quantileExactWeightedSimpleState" -"quantileExactWeightedState" -"quantileForEach" -"quantileGK" -"quantileGKArgMax" -"quantileGKArgMin" -"quantileGKArray" -"quantileGKDistinct" -"quantileGKForEach" -"quantileGKIf" -"quantileGKMap" -"quantileGKMerge" -"quantileGKNull" -"quantileGKOrDefault" -"quantileGKOrNull" -"quantileGKResample" -"quantileGKSimpleState" -"quantileGKState" -"quantileIf" -"quantileInterpolatedWeighted" -"quantileInterpolatedWeightedArgMax" -"quantileInterpolatedWeightedArgMin" -"quantileInterpolatedWeightedArray" -"quantileInterpolatedWeightedDistinct" -"quantileInterpolatedWeightedForEach" -"quantileInterpolatedWeightedIf" -"quantileInterpolatedWeightedMap" -"quantileInterpolatedWeightedMerge" -"quantileInterpolatedWeightedNull" -"quantileInterpolatedWeightedOrDefault" -"quantileInterpolatedWeightedOrNull" -"quantileInterpolatedWeightedResample" -"quantileInterpolatedWeightedSimpleState" -"quantileInterpolatedWeightedState" -"quantileMap" -"quantileMerge" -"quantileNull" -"quantileOrDefault" -"quantileOrNull" -"quantileResample" -"quantileSimpleState" -"quantileState" -"quantileTDigest" -"quantileTDigestArgMax" -"quantileTDigestArgMin" -"quantileTDigestArray" -"quantileTDigestDistinct" -"quantileTDigestForEach" -"quantileTDigestIf" -"quantileTDigestMap" -"quantileTDigestMerge" -"quantileTDigestNull" -"quantileTDigestOrDefault" -"quantileTDigestOrNull" -"quantileTDigestResample" -"quantileTDigestSimpleState" -"quantileTDigestState" -"quantileTDigestWeighted" -"quantileTDigestWeightedArgMax" -"quantileTDigestWeightedArgMin" -"quantileTDigestWeightedArray" -"quantileTDigestWeightedDistinct" -"quantileTDigestWeightedForEach" -"quantileTDigestWeightedIf" -"quantileTDigestWeightedMap" -"quantileTDigestWeightedMerge" -"quantileTDigestWeightedNull" -"quantileTDigestWeightedOrDefault" -"quantileTDigestWeightedOrNull" -"quantileTDigestWeightedResample" -"quantileTDigestWeightedSimpleState" -"quantileTDigestWeightedState" -"quantileTiming" -"quantileTimingArgMax" -"quantileTimingArgMin" -"quantileTimingArray" -"quantileTimingDistinct" -"quantileTimingForEach" -"quantileTimingIf" -"quantileTimingMap" -"quantileTimingMerge" -"quantileTimingNull" -"quantileTimingOrDefault" -"quantileTimingOrNull" -"quantileTimingResample" -"quantileTimingSimpleState" -"quantileTimingState" -"quantileTimingWeighted" -"quantileTimingWeightedArgMax" -"quantileTimingWeightedArgMin" -"quantileTimingWeightedArray" -"quantileTimingWeightedDistinct" -"quantileTimingWeightedForEach" -"quantileTimingWeightedIf" -"quantileTimingWeightedMap" -"quantileTimingWeightedMerge" -"quantileTimingWeightedNull" -"quantileTimingWeightedOrDefault" -"quantileTimingWeightedOrNull" -"quantileTimingWeightedResample" -"quantileTimingWeightedSimpleState" -"quantileTimingWeightedState" -"quantiles" -"quantilesArgMax" -"quantilesArgMin" -"quantilesArray" -"quantilesBFloat16" -"quantilesBFloat16ArgMax" -"quantilesBFloat16ArgMin" -"quantilesBFloat16Array" -"quantilesBFloat16Distinct" -"quantilesBFloat16ForEach" -"quantilesBFloat16If" -"quantilesBFloat16Map" -"quantilesBFloat16Merge" -"quantilesBFloat16Null" -"quantilesBFloat16OrDefault" -"quantilesBFloat16OrNull" -"quantilesBFloat16Resample" -"quantilesBFloat16SimpleState" -"quantilesBFloat16State" -"quantilesBFloat16Weighted" -"quantilesBFloat16WeightedArgMax" -"quantilesBFloat16WeightedArgMin" -"quantilesBFloat16WeightedArray" -"quantilesBFloat16WeightedDistinct" -"quantilesBFloat16WeightedForEach" -"quantilesBFloat16WeightedIf" -"quantilesBFloat16WeightedMap" -"quantilesBFloat16WeightedMerge" -"quantilesBFloat16WeightedNull" -"quantilesBFloat16WeightedOrDefault" -"quantilesBFloat16WeightedOrNull" -"quantilesBFloat16WeightedResample" -"quantilesBFloat16WeightedSimpleState" -"quantilesBFloat16WeightedState" -"quantilesDD" -"quantilesDDArgMax" -"quantilesDDArgMin" -"quantilesDDArray" -"quantilesDDDistinct" -"quantilesDDForEach" -"quantilesDDIf" -"quantilesDDMap" -"quantilesDDMerge" -"quantilesDDNull" -"quantilesDDOrDefault" -"quantilesDDOrNull" -"quantilesDDResample" -"quantilesDDSimpleState" -"quantilesDDState" -"quantilesDeterministic" -"quantilesDeterministicArgMax" -"quantilesDeterministicArgMin" -"quantilesDeterministicArray" -"quantilesDeterministicDistinct" -"quantilesDeterministicForEach" -"quantilesDeterministicIf" -"quantilesDeterministicMap" -"quantilesDeterministicMerge" -"quantilesDeterministicNull" -"quantilesDeterministicOrDefault" -"quantilesDeterministicOrNull" -"quantilesDeterministicResample" -"quantilesDeterministicSimpleState" -"quantilesDeterministicState" -"quantilesDistinct" -"quantilesExact" -"quantilesExactArgMax" -"quantilesExactArgMin" -"quantilesExactArray" -"quantilesExactDistinct" -"quantilesExactExclusive" -"quantilesExactExclusiveArgMax" -"quantilesExactExclusiveArgMin" -"quantilesExactExclusiveArray" -"quantilesExactExclusiveDistinct" -"quantilesExactExclusiveForEach" -"quantilesExactExclusiveIf" -"quantilesExactExclusiveMap" -"quantilesExactExclusiveMerge" -"quantilesExactExclusiveNull" -"quantilesExactExclusiveOrDefault" -"quantilesExactExclusiveOrNull" -"quantilesExactExclusiveResample" -"quantilesExactExclusiveSimpleState" -"quantilesExactExclusiveState" -"quantilesExactForEach" -"quantilesExactHigh" -"quantilesExactHighArgMax" -"quantilesExactHighArgMin" -"quantilesExactHighArray" -"quantilesExactHighDistinct" -"quantilesExactHighForEach" -"quantilesExactHighIf" -"quantilesExactHighMap" -"quantilesExactHighMerge" -"quantilesExactHighNull" -"quantilesExactHighOrDefault" -"quantilesExactHighOrNull" -"quantilesExactHighResample" -"quantilesExactHighSimpleState" -"quantilesExactHighState" -"quantilesExactIf" -"quantilesExactInclusive" -"quantilesExactInclusiveArgMax" -"quantilesExactInclusiveArgMin" -"quantilesExactInclusiveArray" -"quantilesExactInclusiveDistinct" -"quantilesExactInclusiveForEach" -"quantilesExactInclusiveIf" -"quantilesExactInclusiveMap" -"quantilesExactInclusiveMerge" -"quantilesExactInclusiveNull" -"quantilesExactInclusiveOrDefault" -"quantilesExactInclusiveOrNull" -"quantilesExactInclusiveResample" -"quantilesExactInclusiveSimpleState" -"quantilesExactInclusiveState" -"quantilesExactLow" -"quantilesExactLowArgMax" -"quantilesExactLowArgMin" -"quantilesExactLowArray" -"quantilesExactLowDistinct" -"quantilesExactLowForEach" -"quantilesExactLowIf" -"quantilesExactLowMap" -"quantilesExactLowMerge" -"quantilesExactLowNull" -"quantilesExactLowOrDefault" -"quantilesExactLowOrNull" -"quantilesExactLowResample" -"quantilesExactLowSimpleState" -"quantilesExactLowState" -"quantilesExactMap" -"quantilesExactMerge" -"quantilesExactNull" -"quantilesExactOrDefault" -"quantilesExactOrNull" -"quantilesExactResample" -"quantilesExactSimpleState" -"quantilesExactState" -"quantilesExactWeighted" -"quantilesExactWeightedArgMax" -"quantilesExactWeightedArgMin" -"quantilesExactWeightedArray" -"quantilesExactWeightedDistinct" -"quantilesExactWeightedForEach" -"quantilesExactWeightedIf" -"quantilesExactWeightedMap" -"quantilesExactWeightedMerge" -"quantilesExactWeightedNull" -"quantilesExactWeightedOrDefault" -"quantilesExactWeightedOrNull" -"quantilesExactWeightedResample" -"quantilesExactWeightedSimpleState" -"quantilesExactWeightedState" -"quantilesForEach" -"quantilesGK" -"quantilesGKArgMax" -"quantilesGKArgMin" -"quantilesGKArray" -"quantilesGKDistinct" -"quantilesGKForEach" -"quantilesGKIf" -"quantilesGKMap" -"quantilesGKMerge" -"quantilesGKNull" -"quantilesGKOrDefault" -"quantilesGKOrNull" -"quantilesGKResample" -"quantilesGKSimpleState" -"quantilesGKState" -"quantilesIf" -"quantilesInterpolatedWeighted" -"quantilesInterpolatedWeightedArgMax" -"quantilesInterpolatedWeightedArgMin" -"quantilesInterpolatedWeightedArray" -"quantilesInterpolatedWeightedDistinct" -"quantilesInterpolatedWeightedForEach" -"quantilesInterpolatedWeightedIf" -"quantilesInterpolatedWeightedMap" -"quantilesInterpolatedWeightedMerge" -"quantilesInterpolatedWeightedNull" -"quantilesInterpolatedWeightedOrDefault" -"quantilesInterpolatedWeightedOrNull" -"quantilesInterpolatedWeightedResample" -"quantilesInterpolatedWeightedSimpleState" -"quantilesInterpolatedWeightedState" -"quantilesMap" -"quantilesMerge" -"quantilesNull" -"quantilesOrDefault" -"quantilesOrNull" -"quantilesResample" -"quantilesSimpleState" -"quantilesState" -"quantilesTDigest" -"quantilesTDigestArgMax" -"quantilesTDigestArgMin" -"quantilesTDigestArray" -"quantilesTDigestDistinct" -"quantilesTDigestForEach" -"quantilesTDigestIf" -"quantilesTDigestMap" -"quantilesTDigestMerge" -"quantilesTDigestNull" -"quantilesTDigestOrDefault" -"quantilesTDigestOrNull" -"quantilesTDigestResample" -"quantilesTDigestSimpleState" -"quantilesTDigestState" -"quantilesTDigestWeighted" -"quantilesTDigestWeightedArgMax" -"quantilesTDigestWeightedArgMin" -"quantilesTDigestWeightedArray" -"quantilesTDigestWeightedDistinct" -"quantilesTDigestWeightedForEach" -"quantilesTDigestWeightedIf" -"quantilesTDigestWeightedMap" -"quantilesTDigestWeightedMerge" -"quantilesTDigestWeightedNull" -"quantilesTDigestWeightedOrDefault" -"quantilesTDigestWeightedOrNull" -"quantilesTDigestWeightedResample" -"quantilesTDigestWeightedSimpleState" -"quantilesTDigestWeightedState" -"quantilesTiming" -"quantilesTimingArgMax" -"quantilesTimingArgMin" -"quantilesTimingArray" -"quantilesTimingDistinct" -"quantilesTimingForEach" -"quantilesTimingIf" -"quantilesTimingMap" -"quantilesTimingMerge" -"quantilesTimingNull" -"quantilesTimingOrDefault" -"quantilesTimingOrNull" -"quantilesTimingResample" -"quantilesTimingSimpleState" -"quantilesTimingState" -"quantilesTimingWeighted" -"quantilesTimingWeightedArgMax" -"quantilesTimingWeightedArgMin" -"quantilesTimingWeightedArray" -"quantilesTimingWeightedDistinct" -"quantilesTimingWeightedForEach" -"quantilesTimingWeightedIf" -"quantilesTimingWeightedMap" -"quantilesTimingWeightedMerge" -"quantilesTimingWeightedNull" -"quantilesTimingWeightedOrDefault" -"quantilesTimingWeightedOrNull" -"quantilesTimingWeightedResample" -"quantilesTimingWeightedSimpleState" -"quantilesTimingWeightedState" -"queryID" -"queryString" -"queryStringAndFragment" -"query_id" -"radians" -"rand" -"rand32" -"rand64" -"randBernoulli" -"randBinomial" -"randCanonical" -"randChiSquared" -"randConstant" -"randExponential" -"randFisherF" -"randLogNormal" -"randNegativeBinomial" -"randNormal" -"randPoisson" -"randStudentT" -"randUniform" -"randomFixedString" -"randomPrintableASCII" -"randomString" -"randomStringUTF8" -"range" -"rank" -"rankArgMax" -"rankArgMin" -"rankArray" -"rankCorr" -"rankCorrArgMax" -"rankCorrArgMin" -"rankCorrArray" -"rankCorrDistinct" -"rankCorrForEach" -"rankCorrIf" -"rankCorrMap" -"rankCorrMerge" -"rankCorrNull" -"rankCorrOrDefault" -"rankCorrOrNull" -"rankCorrResample" -"rankCorrSimpleState" -"rankCorrState" -"rankDistinct" -"rankForEach" -"rankIf" -"rankMap" -"rankMerge" -"rankNull" -"rankOrDefault" -"rankOrNull" -"rankResample" -"rankSimpleState" -"rankState" -"readWKTLineString" -"readWKTMultiLineString" -"readWKTMultiPolygon" -"readWKTPoint" -"readWKTPolygon" -"readWKTRing" -"regexpExtract" -"regexpQuoteMeta" -"regionHierarchy" -"regionIn" -"regionToArea" -"regionToCity" -"regionToContinent" -"regionToCountry" -"regionToDistrict" -"regionToName" -"regionToPopulation" -"regionToTopContinent" -"reinterpret" -"reinterpretAsDate" -"reinterpretAsDateTime" -"reinterpretAsFixedString" -"reinterpretAsFloat32" -"reinterpretAsFloat64" -"reinterpretAsInt128" -"reinterpretAsInt16" -"reinterpretAsInt256" -"reinterpretAsInt32" -"reinterpretAsInt64" -"reinterpretAsInt8" -"reinterpretAsString" -"reinterpretAsUInt128" -"reinterpretAsUInt16" -"reinterpretAsUInt256" -"reinterpretAsUInt32" -"reinterpretAsUInt64" -"reinterpretAsUInt8" -"reinterpretAsUUID" -"repeat" -"replace" -"replaceAll" -"replaceOne" -"replaceRegexpAll" -"replaceRegexpOne" -"replicate" -"retention" -"retentionArgMax" -"retentionArgMin" -"retentionArray" -"retentionDistinct" -"retentionForEach" -"retentionIf" -"retentionMap" -"retentionMerge" -"retentionNull" -"retentionOrDefault" -"retentionOrNull" -"retentionResample" -"retentionSimpleState" -"retentionState" -"reverse" -"reverseUTF8" -"revision" -"right" -"rightPad" -"rightPadUTF8" -"rightUTF8" -"round" -"roundAge" -"roundBankers" -"roundDown" -"roundDuration" -"roundToExp2" -"rowNumberInAllBlocks" -"rowNumberInBlock" -"row_number" -"row_numberArgMax" -"row_numberArgMin" -"row_numberArray" -"row_numberDistinct" -"row_numberForEach" -"row_numberIf" -"row_numberMap" -"row_numberMerge" -"row_numberNull" -"row_numberOrDefault" -"row_numberOrNull" -"row_numberResample" -"row_numberSimpleState" -"row_numberState" -"rpad" -"rtrim" -"runningAccumulate" -"runningConcurrency" -"runningDifference" -"runningDifferenceStartingWithFirstValue" -"s2CapContains" -"s2CapUnion" -"s2CellsIntersect" -"s2GetNeighbors" -"s2RectAdd" -"s2RectContains" -"s2RectIntersection" -"s2RectUnion" -"s2ToGeo" -"scalarProduct" -"sequenceCount" -"sequenceCountArgMax" -"sequenceCountArgMin" -"sequenceCountArray" -"sequenceCountDistinct" -"sequenceCountForEach" -"sequenceCountIf" -"sequenceCountMap" -"sequenceCountMerge" -"sequenceCountNull" -"sequenceCountOrDefault" -"sequenceCountOrNull" -"sequenceCountResample" -"sequenceCountSimpleState" -"sequenceCountState" -"sequenceMatch" -"sequenceMatchArgMax" -"sequenceMatchArgMin" -"sequenceMatchArray" -"sequenceMatchDistinct" -"sequenceMatchForEach" -"sequenceMatchIf" -"sequenceMatchMap" -"sequenceMatchMerge" -"sequenceMatchNull" -"sequenceMatchOrDefault" -"sequenceMatchOrNull" -"sequenceMatchResample" -"sequenceMatchSimpleState" -"sequenceMatchState" -"sequenceNextNode" -"sequenceNextNodeArgMax" -"sequenceNextNodeArgMin" -"sequenceNextNodeArray" -"sequenceNextNodeDistinct" -"sequenceNextNodeForEach" -"sequenceNextNodeIf" -"sequenceNextNodeMap" -"sequenceNextNodeMerge" -"sequenceNextNodeNull" -"sequenceNextNodeOrDefault" -"sequenceNextNodeOrNull" -"sequenceNextNodeResample" -"sequenceNextNodeSimpleState" -"sequenceNextNodeState" -"seriesDecomposeSTL" -"seriesOutliersDetectTukey" -"seriesPeriodDetectFFT" -"serverTimeZone" -"serverTimezone" -"serverUUID" -"shardCount" -"shardNum" -"showCertificate" -"sigmoid" -"sign" -"simpleJSONExtractBool" -"simpleJSONExtractFloat" -"simpleJSONExtractInt" -"simpleJSONExtractRaw" -"simpleJSONExtractString" -"simpleJSONExtractUInt" -"simpleJSONHas" -"simpleLinearRegression" -"simpleLinearRegressionArgMax" -"simpleLinearRegressionArgMin" -"simpleLinearRegressionArray" -"simpleLinearRegressionDistinct" -"simpleLinearRegressionForEach" -"simpleLinearRegressionIf" -"simpleLinearRegressionMap" -"simpleLinearRegressionMerge" -"simpleLinearRegressionNull" -"simpleLinearRegressionOrDefault" -"simpleLinearRegressionOrNull" -"simpleLinearRegressionResample" -"simpleLinearRegressionSimpleState" -"simpleLinearRegressionState" -"sin" -"singleValueOrNull" -"singleValueOrNullArgMax" -"singleValueOrNullArgMin" -"singleValueOrNullArray" -"singleValueOrNullDistinct" -"singleValueOrNullForEach" -"singleValueOrNullIf" -"singleValueOrNullMap" -"singleValueOrNullMerge" -"singleValueOrNullNull" -"singleValueOrNullOrDefault" -"singleValueOrNullOrNull" -"singleValueOrNullResample" -"singleValueOrNullSimpleState" -"singleValueOrNullState" -"sinh" -"sipHash128" -"sipHash128Keyed" -"sipHash128Reference" -"sipHash128ReferenceKeyed" -"sipHash64" -"sipHash64Keyed" -"skewPop" -"skewPopArgMax" -"skewPopArgMin" -"skewPopArray" -"skewPopDistinct" -"skewPopForEach" -"skewPopIf" -"skewPopMap" -"skewPopMerge" -"skewPopNull" -"skewPopOrDefault" -"skewPopOrNull" -"skewPopResample" -"skewPopSimpleState" -"skewPopState" -"skewSamp" -"skewSampArgMax" -"skewSampArgMin" -"skewSampArray" -"skewSampDistinct" -"skewSampForEach" -"skewSampIf" -"skewSampMap" -"skewSampMerge" -"skewSampNull" -"skewSampOrDefault" -"skewSampOrNull" -"skewSampResample" -"skewSampSimpleState" -"skewSampState" -"sleep" -"sleepEachRow" -"snowflakeIDToDateTime" -"snowflakeIDToDateTime64" -"snowflakeToDateTime" -"snowflakeToDateTime64" -"soundex" -"space" -"sparkBar" -"sparkBarArgMax" -"sparkBarArgMin" -"sparkBarArray" -"sparkBarDistinct" -"sparkBarForEach" -"sparkBarIf" -"sparkBarMap" -"sparkBarMerge" -"sparkBarNull" -"sparkBarOrDefault" -"sparkBarOrNull" -"sparkBarResample" -"sparkBarSimpleState" -"sparkBarState" -"sparkbar" -"sparkbarArgMax" -"sparkbarArgMin" -"sparkbarArray" -"sparkbarDistinct" -"sparkbarForEach" -"sparkbarIf" -"sparkbarMap" -"sparkbarMerge" -"sparkbarNull" -"sparkbarOrDefault" -"sparkbarOrNull" -"sparkbarResample" -"sparkbarSimpleState" -"sparkbarState" -"splitByAlpha" -"splitByChar" -"splitByNonAlpha" -"splitByRegexp" -"splitByString" -"splitByWhitespace" -"sqid" -"sqidDecode" -"sqidEncode" -"sqrt" -"startsWith" -"startsWithUTF8" -"stddevPop" -"stddevPopArgMax" -"stddevPopArgMin" -"stddevPopArray" -"stddevPopDistinct" -"stddevPopForEach" -"stddevPopIf" -"stddevPopMap" -"stddevPopMerge" -"stddevPopNull" -"stddevPopOrDefault" -"stddevPopOrNull" -"stddevPopResample" -"stddevPopSimpleState" -"stddevPopStable" -"stddevPopStableArgMax" -"stddevPopStableArgMin" -"stddevPopStableArray" -"stddevPopStableDistinct" -"stddevPopStableForEach" -"stddevPopStableIf" -"stddevPopStableMap" -"stddevPopStableMerge" -"stddevPopStableNull" -"stddevPopStableOrDefault" -"stddevPopStableOrNull" -"stddevPopStableResample" -"stddevPopStableSimpleState" -"stddevPopStableState" -"stddevPopState" -"stddevSamp" -"stddevSampArgMax" -"stddevSampArgMin" -"stddevSampArray" -"stddevSampDistinct" -"stddevSampForEach" -"stddevSampIf" -"stddevSampMap" -"stddevSampMerge" -"stddevSampNull" -"stddevSampOrDefault" -"stddevSampOrNull" -"stddevSampResample" -"stddevSampSimpleState" -"stddevSampStable" -"stddevSampStableArgMax" -"stddevSampStableArgMin" -"stddevSampStableArray" -"stddevSampStableDistinct" -"stddevSampStableForEach" -"stddevSampStableIf" -"stddevSampStableMap" -"stddevSampStableMerge" -"stddevSampStableNull" -"stddevSampStableOrDefault" -"stddevSampStableOrNull" -"stddevSampStableResample" -"stddevSampStableSimpleState" -"stddevSampStableState" -"stddevSampState" -"stem" -"stochasticLinearRegression" -"stochasticLinearRegressionArgMax" -"stochasticLinearRegressionArgMin" -"stochasticLinearRegressionArray" -"stochasticLinearRegressionDistinct" -"stochasticLinearRegressionForEach" -"stochasticLinearRegressionIf" -"stochasticLinearRegressionMap" -"stochasticLinearRegressionMerge" -"stochasticLinearRegressionNull" -"stochasticLinearRegressionOrDefault" -"stochasticLinearRegressionOrNull" -"stochasticLinearRegressionResample" -"stochasticLinearRegressionSimpleState" -"stochasticLinearRegressionState" -"stochasticLogisticRegression" -"stochasticLogisticRegressionArgMax" -"stochasticLogisticRegressionArgMin" -"stochasticLogisticRegressionArray" -"stochasticLogisticRegressionDistinct" -"stochasticLogisticRegressionForEach" -"stochasticLogisticRegressionIf" -"stochasticLogisticRegressionMap" -"stochasticLogisticRegressionMerge" -"stochasticLogisticRegressionNull" -"stochasticLogisticRegressionOrDefault" -"stochasticLogisticRegressionOrNull" -"stochasticLogisticRegressionResample" -"stochasticLogisticRegressionSimpleState" -"stochasticLogisticRegressionState" -"str_to_date" -"str_to_map" -"stringJaccardIndex" -"stringJaccardIndexUTF8" -"stringToH3" -"structureToCapnProtoSchema" -"structureToProtobufSchema" -"studentTTest" -"studentTTestArgMax" -"studentTTestArgMin" -"studentTTestArray" -"studentTTestDistinct" -"studentTTestForEach" -"studentTTestIf" -"studentTTestMap" -"studentTTestMerge" -"studentTTestNull" -"studentTTestOrDefault" -"studentTTestOrNull" -"studentTTestResample" -"studentTTestSimpleState" -"studentTTestState" -"subBitmap" -"subDate" -"substr" -"substring" -"substringIndex" -"substringIndexUTF8" -"substringUTF8" -"subtractDays" -"subtractHours" -"subtractInterval" -"subtractMicroseconds" -"subtractMilliseconds" -"subtractMinutes" -"subtractMonths" -"subtractNanoseconds" -"subtractQuarters" -"subtractSeconds" -"subtractTupleOfIntervals" -"subtractWeeks" -"subtractYears" -"sum" -"sumArgMax" -"sumArgMin" -"sumArray" -"sumCount" -"sumCountArgMax" -"sumCountArgMin" -"sumCountArray" -"sumCountDistinct" -"sumCountForEach" -"sumCountIf" -"sumCountMap" -"sumCountMerge" -"sumCountNull" -"sumCountOrDefault" -"sumCountOrNull" -"sumCountResample" -"sumCountSimpleState" -"sumCountState" -"sumDistinct" -"sumForEach" -"sumIf" -"sumKahan" -"sumKahanArgMax" -"sumKahanArgMin" -"sumKahanArray" -"sumKahanDistinct" -"sumKahanForEach" -"sumKahanIf" -"sumKahanMap" -"sumKahanMerge" -"sumKahanNull" -"sumKahanOrDefault" -"sumKahanOrNull" -"sumKahanResample" -"sumKahanSimpleState" -"sumKahanState" -"sumMap" -"sumMapFiltered" -"sumMapFilteredArgMax" -"sumMapFilteredArgMin" -"sumMapFilteredArray" -"sumMapFilteredDistinct" -"sumMapFilteredForEach" -"sumMapFilteredIf" -"sumMapFilteredMap" -"sumMapFilteredMerge" -"sumMapFilteredNull" -"sumMapFilteredOrDefault" -"sumMapFilteredOrNull" -"sumMapFilteredResample" -"sumMapFilteredSimpleState" -"sumMapFilteredState" -"sumMapFilteredWithOverflow" -"sumMapFilteredWithOverflowArgMax" -"sumMapFilteredWithOverflowArgMin" -"sumMapFilteredWithOverflowArray" -"sumMapFilteredWithOverflowDistinct" -"sumMapFilteredWithOverflowForEach" -"sumMapFilteredWithOverflowIf" -"sumMapFilteredWithOverflowMap" -"sumMapFilteredWithOverflowMerge" -"sumMapFilteredWithOverflowNull" -"sumMapFilteredWithOverflowOrDefault" -"sumMapFilteredWithOverflowOrNull" -"sumMapFilteredWithOverflowResample" -"sumMapFilteredWithOverflowSimpleState" -"sumMapFilteredWithOverflowState" -"sumMapWithOverflow" -"sumMapWithOverflowArgMax" -"sumMapWithOverflowArgMin" -"sumMapWithOverflowArray" -"sumMapWithOverflowDistinct" -"sumMapWithOverflowForEach" -"sumMapWithOverflowIf" -"sumMapWithOverflowMap" -"sumMapWithOverflowMerge" -"sumMapWithOverflowNull" -"sumMapWithOverflowOrDefault" -"sumMapWithOverflowOrNull" -"sumMapWithOverflowResample" -"sumMapWithOverflowSimpleState" -"sumMapWithOverflowState" -"sumMappedArrays" -"sumMappedArraysArgMax" -"sumMappedArraysArgMin" -"sumMappedArraysArray" -"sumMappedArraysDistinct" -"sumMappedArraysForEach" -"sumMappedArraysIf" -"sumMappedArraysMap" -"sumMappedArraysMerge" -"sumMappedArraysNull" -"sumMappedArraysOrDefault" -"sumMappedArraysOrNull" -"sumMappedArraysResample" -"sumMappedArraysSimpleState" -"sumMappedArraysState" -"sumMerge" -"sumNull" -"sumOrDefault" -"sumOrNull" -"sumResample" -"sumSimpleState" -"sumState" -"sumWithOverflow" -"sumWithOverflowArgMax" -"sumWithOverflowArgMin" -"sumWithOverflowArray" -"sumWithOverflowDistinct" -"sumWithOverflowForEach" -"sumWithOverflowIf" -"sumWithOverflowMap" -"sumWithOverflowMerge" -"sumWithOverflowNull" -"sumWithOverflowOrDefault" -"sumWithOverflowOrNull" -"sumWithOverflowResample" -"sumWithOverflowSimpleState" -"sumWithOverflowState" -"svg" -"synonyms" -"tan" -"tanh" -"tcpPort" -"tgamma" -"theilsU" -"theilsUArgMax" -"theilsUArgMin" -"theilsUArray" -"theilsUDistinct" -"theilsUForEach" -"theilsUIf" -"theilsUMap" -"theilsUMerge" -"theilsUNull" -"theilsUOrDefault" -"theilsUOrNull" -"theilsUResample" -"theilsUSimpleState" -"theilsUState" -"throwIf" -"tid" -"timeDiff" -"timeSlot" -"timeSlots" -"timeZone" -"timeZoneOf" -"timeZoneOffset" -"timestamp" -"timestampDiff" -"timestamp_diff" -"timezone" -"timezoneOf" -"timezoneOffset" -"toBool" -"toColumnTypeName" -"toDate" -"toDate32" -"toDate32OrDefault" -"toDate32OrNull" -"toDate32OrZero" -"toDateOrDefault" -"toDateOrNull" -"toDateOrZero" -"toDateTime" -"toDateTime32" -"toDateTime64" -"toDateTime64OrDefault" -"toDateTime64OrNull" -"toDateTime64OrZero" -"toDateTimeOrDefault" -"toDateTimeOrNull" -"toDateTimeOrZero" -"toDayOfMonth" -"toDayOfWeek" -"toDayOfYear" -"toDaysSinceYearZero" -"toDecimal128" -"toDecimal128OrDefault" -"toDecimal128OrNull" -"toDecimal128OrZero" -"toDecimal256" -"toDecimal256OrDefault" -"toDecimal256OrNull" -"toDecimal256OrZero" -"toDecimal32" -"toDecimal32OrDefault" -"toDecimal32OrNull" -"toDecimal32OrZero" -"toDecimal64" -"toDecimal64OrDefault" -"toDecimal64OrNull" -"toDecimal64OrZero" -"toDecimalString" -"toFixedString" -"toFloat32" -"toFloat32OrDefault" -"toFloat32OrNull" -"toFloat32OrZero" -"toFloat64" -"toFloat64OrDefault" -"toFloat64OrNull" -"toFloat64OrZero" -"toHour" -"toIPv4" -"toIPv4OrDefault" -"toIPv4OrNull" -"toIPv4OrZero" -"toIPv6" -"toIPv6OrDefault" -"toIPv6OrNull" -"toIPv6OrZero" -"toISOWeek" -"toISOYear" -"toInt128" -"toInt128OrDefault" -"toInt128OrNull" -"toInt128OrZero" -"toInt16" -"toInt16OrDefault" -"toInt16OrNull" -"toInt16OrZero" -"toInt256" -"toInt256OrDefault" -"toInt256OrNull" -"toInt256OrZero" -"toInt32" -"toInt32OrDefault" -"toInt32OrNull" -"toInt32OrZero" -"toInt64" -"toInt64OrDefault" -"toInt64OrNull" -"toInt64OrZero" -"toInt8" -"toInt8OrDefault" -"toInt8OrNull" -"toInt8OrZero" -"toIntervalDay" -"toIntervalHour" -"toIntervalMicrosecond" -"toIntervalMillisecond" -"toIntervalMinute" -"toIntervalMonth" -"toIntervalNanosecond" -"toIntervalQuarter" -"toIntervalSecond" -"toIntervalWeek" -"toIntervalYear" -"toJSONString" -"toLastDayOfMonth" -"toLastDayOfWeek" -"toLowCardinality" -"toMillisecond" -"toMinute" -"toModifiedJulianDay" -"toModifiedJulianDayOrNull" -"toMonday" -"toMonth" -"toNullable" -"toQuarter" -"toRelativeDayNum" -"toRelativeHourNum" -"toRelativeMinuteNum" -"toRelativeMonthNum" -"toRelativeQuarterNum" -"toRelativeSecondNum" -"toRelativeWeekNum" -"toRelativeYearNum" -"toSecond" -"toStartOfDay" -"toStartOfFifteenMinutes" -"toStartOfFiveMinute" -"toStartOfFiveMinutes" -"toStartOfHour" -"toStartOfISOYear" -"toStartOfInterval" -"toStartOfMicrosecond" -"toStartOfMillisecond" -"toStartOfMinute" -"toStartOfMonth" -"toStartOfNanosecond" -"toStartOfQuarter" -"toStartOfSecond" -"toStartOfTenMinutes" -"toStartOfWeek" -"toStartOfYear" -"toString" -"toStringCutToZero" -"toTime" -"toTimeZone" -"toTimezone" -"toTypeName" -"toUInt128" -"toUInt128OrDefault" -"toUInt128OrNull" -"toUInt128OrZero" -"toUInt16" -"toUInt16OrDefault" -"toUInt16OrNull" -"toUInt16OrZero" -"toUInt256" -"toUInt256OrDefault" -"toUInt256OrNull" -"toUInt256OrZero" -"toUInt32" -"toUInt32OrDefault" -"toUInt32OrNull" -"toUInt32OrZero" -"toUInt64" -"toUInt64OrDefault" -"toUInt64OrNull" -"toUInt64OrZero" -"toUInt8" -"toUInt8OrDefault" -"toUInt8OrNull" -"toUInt8OrZero" -"toUTCTimestamp" -"toUUID" -"toUUIDOrDefault" -"toUUIDOrNull" -"toUUIDOrZero" -"toUnixTimestamp" -"toUnixTimestamp64Micro" -"toUnixTimestamp64Milli" -"toUnixTimestamp64Nano" -"toValidUTF8" -"toWeek" -"toYYYYMM" -"toYYYYMMDD" -"toYYYYMMDDhhmmss" -"toYear" -"toYearWeek" -"to_utc_timestamp" -"today" -"tokens" -"topK" -"topKArgMax" -"topKArgMin" -"topKArray" -"topKDistinct" -"topKForEach" -"topKIf" -"topKMap" -"topKMerge" -"topKNull" -"topKOrDefault" -"topKOrNull" -"topKResample" -"topKSimpleState" -"topKState" -"topKWeighted" -"topKWeightedArgMax" -"topKWeightedArgMin" -"topKWeightedArray" -"topKWeightedDistinct" -"topKWeightedForEach" -"topKWeightedIf" -"topKWeightedMap" -"topKWeightedMerge" -"topKWeightedNull" -"topKWeightedOrDefault" -"topKWeightedOrNull" -"topKWeightedResample" -"topKWeightedSimpleState" -"topKWeightedState" -"topLevelDomain" -"topLevelDomainRFC" -"transactionID" -"transactionLatestSnapshot" -"transactionOldestSnapshot" -"transform" -"translate" -"translateUTF8" -"trim" -"trimBoth" -"trimLeft" -"trimRight" -"trunc" -"truncate" -"tryBase58Decode" -"tryBase64Decode" -"tryBase64URLDecode" -"tryDecrypt" -"tryIdnaEncode" -"tryPunycodeDecode" -"tumble" -"tumbleEnd" -"tumbleStart" -"tuple" -"tupleConcat" -"tupleDivide" -"tupleDivideByNumber" -"tupleElement" -"tupleHammingDistance" -"tupleIntDiv" -"tupleIntDivByNumber" -"tupleIntDivOrZero" -"tupleIntDivOrZeroByNumber" -"tupleMinus" -"tupleModulo" -"tupleModuloByNumber" -"tupleMultiply" -"tupleMultiplyByNumber" -"tupleNames" -"tupleNegate" -"tuplePlus" -"tupleToNameValuePairs" -"ucase" -"unbin" -"unhex" -"uniq" -"uniqArgMax" -"uniqArgMin" -"uniqArray" -"uniqCombined" -"uniqCombined64" -"uniqCombined64ArgMax" -"uniqCombined64ArgMin" -"uniqCombined64Array" -"uniqCombined64Distinct" -"uniqCombined64ForEach" -"uniqCombined64If" -"uniqCombined64Map" -"uniqCombined64Merge" -"uniqCombined64Null" -"uniqCombined64OrDefault" -"uniqCombined64OrNull" -"uniqCombined64Resample" -"uniqCombined64SimpleState" -"uniqCombined64State" -"uniqCombinedArgMax" -"uniqCombinedArgMin" -"uniqCombinedArray" -"uniqCombinedDistinct" -"uniqCombinedForEach" -"uniqCombinedIf" -"uniqCombinedMap" -"uniqCombinedMerge" -"uniqCombinedNull" -"uniqCombinedOrDefault" -"uniqCombinedOrNull" -"uniqCombinedResample" -"uniqCombinedSimpleState" -"uniqCombinedState" -"uniqDistinct" -"uniqExact" -"uniqExactArgMax" -"uniqExactArgMin" -"uniqExactArray" -"uniqExactDistinct" -"uniqExactForEach" -"uniqExactIf" -"uniqExactMap" -"uniqExactMerge" -"uniqExactNull" -"uniqExactOrDefault" -"uniqExactOrNull" -"uniqExactResample" -"uniqExactSimpleState" -"uniqExactState" -"uniqForEach" -"uniqHLL12" -"uniqHLL12ArgMax" -"uniqHLL12ArgMin" -"uniqHLL12Array" -"uniqHLL12Distinct" -"uniqHLL12ForEach" -"uniqHLL12If" -"uniqHLL12Map" -"uniqHLL12Merge" -"uniqHLL12Null" -"uniqHLL12OrDefault" -"uniqHLL12OrNull" -"uniqHLL12Resample" -"uniqHLL12SimpleState" -"uniqHLL12State" -"uniqIf" -"uniqMap" -"uniqMerge" -"uniqNull" -"uniqOrDefault" -"uniqOrNull" -"uniqResample" -"uniqSimpleState" -"uniqState" -"uniqTheta" -"uniqThetaArgMax" -"uniqThetaArgMin" -"uniqThetaArray" -"uniqThetaDistinct" -"uniqThetaForEach" -"uniqThetaIf" -"uniqThetaIntersect" -"uniqThetaMap" -"uniqThetaMerge" -"uniqThetaNot" -"uniqThetaNull" -"uniqThetaOrDefault" -"uniqThetaOrNull" -"uniqThetaResample" -"uniqThetaSimpleState" -"uniqThetaState" -"uniqThetaUnion" -"uniqUpTo" -"uniqUpToArgMax" -"uniqUpToArgMin" -"uniqUpToArray" -"uniqUpToDistinct" -"uniqUpToForEach" -"uniqUpToIf" -"uniqUpToMap" -"uniqUpToMerge" -"uniqUpToNull" -"uniqUpToOrDefault" -"uniqUpToOrNull" -"uniqUpToResample" -"uniqUpToSimpleState" -"uniqUpToState" -"upper" -"upperUTF8" -"uptime" -"user" -"validateNestedArraySizes" -"varPop" -"varPopArgMax" -"varPopArgMin" -"varPopArray" -"varPopDistinct" -"varPopForEach" -"varPopIf" -"varPopMap" -"varPopMerge" -"varPopNull" -"varPopOrDefault" -"varPopOrNull" -"varPopResample" -"varPopSimpleState" -"varPopStable" -"varPopStableArgMax" -"varPopStableArgMin" -"varPopStableArray" -"varPopStableDistinct" -"varPopStableForEach" -"varPopStableIf" -"varPopStableMap" -"varPopStableMerge" -"varPopStableNull" -"varPopStableOrDefault" -"varPopStableOrNull" -"varPopStableResample" -"varPopStableSimpleState" -"varPopStableState" -"varPopState" -"varSamp" -"varSampArgMax" -"varSampArgMin" -"varSampArray" -"varSampDistinct" -"varSampForEach" -"varSampIf" -"varSampMap" -"varSampMerge" -"varSampNull" -"varSampOrDefault" -"varSampOrNull" -"varSampResample" -"varSampSimpleState" -"varSampStable" -"varSampStableArgMax" -"varSampStableArgMin" -"varSampStableArray" -"varSampStableDistinct" -"varSampStableForEach" -"varSampStableIf" -"varSampStableMap" -"varSampStableMerge" -"varSampStableNull" -"varSampStableOrDefault" -"varSampStableOrNull" -"varSampStableResample" -"varSampStableSimpleState" -"varSampStableState" -"varSampState" -"variantElement" -"variantType" -"vectorDifference" -"vectorSum" -"version" -"visibleWidth" -"visitParamExtractBool" -"visitParamExtractFloat" -"visitParamExtractInt" -"visitParamExtractRaw" -"visitParamExtractString" -"visitParamExtractUInt" -"visitParamHas" -"week" -"welchTTest" -"welchTTestArgMax" -"welchTTestArgMin" -"welchTTestArray" -"welchTTestDistinct" -"welchTTestForEach" -"welchTTestIf" -"welchTTestMap" -"welchTTestMerge" -"welchTTestNull" -"welchTTestOrDefault" -"welchTTestOrNull" -"welchTTestResample" -"welchTTestSimpleState" -"welchTTestState" -"widthBucket" -"width_bucket" -"windowFunnel" -"windowFunnelArgMax" -"windowFunnelArgMin" -"windowFunnelArray" -"windowFunnelDistinct" -"windowFunnelForEach" -"windowFunnelIf" -"windowFunnelMap" -"windowFunnelMerge" -"windowFunnelNull" -"windowFunnelOrDefault" -"windowFunnelOrNull" -"windowFunnelResample" -"windowFunnelSimpleState" -"windowFunnelState" -"windowID" -"wkt" -"wordShingleMinHash" -"wordShingleMinHashArg" -"wordShingleMinHashArgCaseInsensitive" -"wordShingleMinHashArgCaseInsensitiveUTF8" -"wordShingleMinHashArgUTF8" -"wordShingleMinHashCaseInsensitive" -"wordShingleMinHashCaseInsensitiveUTF8" -"wordShingleMinHashUTF8" -"wordShingleSimHash" -"wordShingleSimHashCaseInsensitive" -"wordShingleSimHashCaseInsensitiveUTF8" -"wordShingleSimHashUTF8" -"wyHash64" -"xor" -"xxHash32" -"xxHash64" -"xxh3" -"yandexConsistentHash" -"yearweek" -"yesterday" -"zookeeperSessionUptime" +"bool" +"boolean" diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index e562595fb67..e69de29bb2d 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -1,4283 +0,0 @@ -"BIT_AND" -"BIT_ANDArgMax" -"BIT_ANDArgMin" -"BIT_ANDArray" -"BIT_ANDDistinct" -"BIT_ANDForEach" -"BIT_ANDIf" -"BIT_ANDMap" -"BIT_ANDMerge" -"BIT_ANDNull" -"BIT_ANDOrDefault" -"BIT_ANDOrNull" -"BIT_ANDResample" -"BIT_ANDSimpleState" -"BIT_ANDState" -"BIT_OR" -"BIT_ORArgMax" -"BIT_ORArgMin" -"BIT_ORArray" -"BIT_ORDistinct" -"BIT_ORForEach" -"BIT_ORIf" -"BIT_ORMap" -"BIT_ORMerge" -"BIT_ORNull" -"BIT_OROrDefault" -"BIT_OROrNull" -"BIT_ORResample" -"BIT_ORSimpleState" -"BIT_ORState" -"BIT_XOR" -"BIT_XORArgMax" -"BIT_XORArgMin" -"BIT_XORArray" -"BIT_XORDistinct" -"BIT_XORForEach" -"BIT_XORIf" -"BIT_XORMap" -"BIT_XORMerge" -"BIT_XORNull" -"BIT_XOROrDefault" -"BIT_XOROrNull" -"BIT_XORResample" -"BIT_XORSimpleState" -"BIT_XORState" -"BLAKE3" -"CAST" -"CHARACTER_LENGTH" -"CHAR_LENGTH" -"COVAR_POP" -"COVAR_POPArgMax" -"COVAR_POPArgMin" -"COVAR_POPArray" -"COVAR_POPDistinct" -"COVAR_POPForEach" -"COVAR_POPIf" -"COVAR_POPMap" -"COVAR_POPMerge" -"COVAR_POPNull" -"COVAR_POPOrDefault" -"COVAR_POPOrNull" -"COVAR_POPResample" -"COVAR_POPSimpleState" -"COVAR_POPState" -"COVAR_SAMP" -"COVAR_SAMPArgMax" -"COVAR_SAMPArgMin" -"COVAR_SAMPArray" -"COVAR_SAMPDistinct" -"COVAR_SAMPForEach" -"COVAR_SAMPIf" -"COVAR_SAMPMap" -"COVAR_SAMPMerge" -"COVAR_SAMPNull" -"COVAR_SAMPOrDefault" -"COVAR_SAMPOrNull" -"COVAR_SAMPResample" -"COVAR_SAMPSimpleState" -"COVAR_SAMPState" -"CRC32" -"CRC32IEEE" -"CRC64" -"DATABASE" -"DATE" -"DATE_DIFF" -"DATE_FORMAT" -"DATE_TRUNC" -"DAY" -"DAYOFMONTH" -"DAYOFWEEK" -"DAYOFYEAR" -"FORMAT_BYTES" -"FQDN" -"FROM_BASE64" -"FROM_DAYS" -"FROM_UNIXTIME" -"HOUR" -"INET6_ATON" -"INET6_NTOA" -"INET_ATON" -"INET_NTOA" -"IPv4CIDRToRange" -"IPv4NumToString" -"IPv4NumToStringClassC" -"IPv4StringToNum" -"IPv4StringToNumOrDefault" -"IPv4StringToNumOrNull" -"IPv4ToIPv6" -"IPv6CIDRToRange" -"IPv6NumToString" -"IPv6StringToNum" -"IPv6StringToNumOrDefault" -"IPv6StringToNumOrNull" -"JSONArrayLength" -"JSONExtract" -"JSONExtractArrayRaw" -"JSONExtractBool" -"JSONExtractFloat" -"JSONExtractInt" -"JSONExtractKeys" -"JSONExtractKeysAndValues" -"JSONExtractKeysAndValuesRaw" -"JSONExtractRaw" -"JSONExtractString" -"JSONExtractUInt" -"JSONHas" -"JSONKey" -"JSONLength" -"JSONMergePatch" -"JSONType" -"JSON_ARRAY_LENGTH" -"JSON_EXISTS" -"JSON_QUERY" -"JSON_VALUE" -"L1Distance" -"L1Norm" -"L1Normalize" -"L2Distance" -"L2Norm" -"L2Normalize" -"L2SquaredDistance" -"L2SquaredNorm" -"LAST_DAY" -"LinfDistance" -"LinfNorm" -"LinfNormalize" -"LpDistance" -"LpNorm" -"LpNormalize" -"MACNumToString" -"MACStringToNum" -"MACStringToOUI" -"MAP_FROM_ARRAYS" -"MD4" -"MD5" -"MILLISECOND" -"MINUTE" -"MONTH" -"OCTET_LENGTH" -"QUARTER" -"REGEXP_EXTRACT" -"REGEXP_MATCHES" -"REGEXP_REPLACE" -"SCHEMA" -"SECOND" -"SHA1" -"SHA224" -"SHA256" -"SHA384" -"SHA512" -"SHA512_256" -"STD" -"STDArgMax" -"STDArgMin" -"STDArray" -"STDDEV_POP" -"STDDEV_POPArgMax" -"STDDEV_POPArgMin" -"STDDEV_POPArray" -"STDDEV_POPDistinct" -"STDDEV_POPForEach" -"STDDEV_POPIf" -"STDDEV_POPMap" -"STDDEV_POPMerge" -"STDDEV_POPNull" -"STDDEV_POPOrDefault" -"STDDEV_POPOrNull" -"STDDEV_POPResample" -"STDDEV_POPSimpleState" -"STDDEV_POPState" -"STDDEV_SAMP" -"STDDEV_SAMPArgMax" -"STDDEV_SAMPArgMin" -"STDDEV_SAMPArray" -"STDDEV_SAMPDistinct" -"STDDEV_SAMPForEach" -"STDDEV_SAMPIf" -"STDDEV_SAMPMap" -"STDDEV_SAMPMerge" -"STDDEV_SAMPNull" -"STDDEV_SAMPOrDefault" -"STDDEV_SAMPOrNull" -"STDDEV_SAMPResample" -"STDDEV_SAMPSimpleState" -"STDDEV_SAMPState" -"STDDistinct" -"STDForEach" -"STDIf" -"STDMap" -"STDMerge" -"STDNull" -"STDOrDefault" -"STDOrNull" -"STDResample" -"STDSimpleState" -"STDState" -"SUBSTRING_INDEX" -"SVG" -"TIMESTAMP_DIFF" -"TO_BASE64" -"TO_DAYS" -"TO_UNIXTIME" -"ULIDStringToDateTime" -"URLHash" -"URLHierarchy" -"URLPathHierarchy" -"UTCTimestamp" -"UTC_timestamp" -"UUIDNumToString" -"UUIDStringToNum" -"UUIDToNum" -"UUIDv7ToDateTime" -"VAR_POP" -"VAR_POPArgMax" -"VAR_POPArgMin" -"VAR_POPArray" -"VAR_POPDistinct" -"VAR_POPForEach" -"VAR_POPIf" -"VAR_POPMap" -"VAR_POPMerge" -"VAR_POPNull" -"VAR_POPOrDefault" -"VAR_POPOrNull" -"VAR_POPResample" -"VAR_POPSimpleState" -"VAR_POPState" -"VAR_SAMP" -"VAR_SAMPArgMax" -"VAR_SAMPArgMin" -"VAR_SAMPArray" -"VAR_SAMPDistinct" -"VAR_SAMPForEach" -"VAR_SAMPIf" -"VAR_SAMPMap" -"VAR_SAMPMerge" -"VAR_SAMPNull" -"VAR_SAMPOrDefault" -"VAR_SAMPOrNull" -"VAR_SAMPResample" -"VAR_SAMPSimpleState" -"VAR_SAMPState" -"YEAR" -"YYYYMMDDToDate" -"YYYYMMDDToDate32" -"YYYYMMDDhhmmssToDateTime" -"YYYYMMDDhhmmssToDateTime64" -"_CAST" -"__actionName" -"__bitBoolMaskAnd" -"__bitBoolMaskOr" -"__bitSwapLastTwo" -"__bitWrapperFunc" -"__getScalar" -"__scalarSubqueryResult" -"abs" -"accurateCast" -"accurateCastOrDefault" -"accurateCastOrNull" -"acos" -"acosh" -"addDate" -"addDays" -"addHours" -"addInterval" -"addMicroseconds" -"addMilliseconds" -"addMinutes" -"addMonths" -"addNanoseconds" -"addQuarters" -"addSeconds" -"addTupleOfIntervals" -"addWeeks" -"addYears" -"addressToLine" -"addressToLineWithInlines" -"addressToSymbol" -"aes_decrypt_mysql" -"aes_encrypt_mysql" -"age" -"aggThrow" -"aggThrowArgMax" -"aggThrowArgMin" -"aggThrowArray" -"aggThrowDistinct" -"aggThrowForEach" -"aggThrowIf" -"aggThrowMap" -"aggThrowMerge" -"aggThrowNull" -"aggThrowOrDefault" -"aggThrowOrNull" -"aggThrowResample" -"aggThrowSimpleState" -"aggThrowState" -"alphaTokens" -"analysisOfVariance" -"analysisOfVarianceArgMax" -"analysisOfVarianceArgMin" -"analysisOfVarianceArray" -"analysisOfVarianceDistinct" -"analysisOfVarianceForEach" -"analysisOfVarianceIf" -"analysisOfVarianceMap" -"analysisOfVarianceMerge" -"analysisOfVarianceNull" -"analysisOfVarianceOrDefault" -"analysisOfVarianceOrNull" -"analysisOfVarianceResample" -"analysisOfVarianceSimpleState" -"analysisOfVarianceState" -"and" -"anova" -"anovaArgMax" -"anovaArgMin" -"anovaArray" -"anovaDistinct" -"anovaForEach" -"anovaIf" -"anovaMap" -"anovaMerge" -"anovaNull" -"anovaOrDefault" -"anovaOrNull" -"anovaResample" -"anovaSimpleState" -"anovaState" -"any" -"anyArgMax" -"anyArgMin" -"anyArray" -"anyDistinct" -"anyForEach" -"anyHeavy" -"anyHeavyArgMax" -"anyHeavyArgMin" -"anyHeavyArray" -"anyHeavyDistinct" -"anyHeavyForEach" -"anyHeavyIf" -"anyHeavyMap" -"anyHeavyMerge" -"anyHeavyNull" -"anyHeavyOrDefault" -"anyHeavyOrNull" -"anyHeavyResample" -"anyHeavySimpleState" -"anyHeavyState" -"anyIf" -"anyLast" -"anyLastArgMax" -"anyLastArgMin" -"anyLastArray" -"anyLastDistinct" -"anyLastForEach" -"anyLastIf" -"anyLastMap" -"anyLastMerge" -"anyLastNull" -"anyLastOrDefault" -"anyLastOrNull" -"anyLastResample" -"anyLastSimpleState" -"anyLastState" -"anyLast_respect_nulls" -"anyLast_respect_nullsArgMax" -"anyLast_respect_nullsArgMin" -"anyLast_respect_nullsArray" -"anyLast_respect_nullsDistinct" -"anyLast_respect_nullsForEach" -"anyLast_respect_nullsIf" -"anyLast_respect_nullsMap" -"anyLast_respect_nullsMerge" -"anyLast_respect_nullsNull" -"anyLast_respect_nullsOrDefault" -"anyLast_respect_nullsOrNull" -"anyLast_respect_nullsResample" -"anyLast_respect_nullsSimpleState" -"anyLast_respect_nullsState" -"anyMap" -"anyMerge" -"anyNull" -"anyOrDefault" -"anyOrNull" -"anyResample" -"anySimpleState" -"anyState" -"any_respect_nulls" -"any_respect_nullsArgMax" -"any_respect_nullsArgMin" -"any_respect_nullsArray" -"any_respect_nullsDistinct" -"any_respect_nullsForEach" -"any_respect_nullsIf" -"any_respect_nullsMap" -"any_respect_nullsMerge" -"any_respect_nullsNull" -"any_respect_nullsOrDefault" -"any_respect_nullsOrNull" -"any_respect_nullsResample" -"any_respect_nullsSimpleState" -"any_respect_nullsState" -"any_value" -"any_valueArgMax" -"any_valueArgMin" -"any_valueArray" -"any_valueDistinct" -"any_valueForEach" -"any_valueIf" -"any_valueMap" -"any_valueMerge" -"any_valueNull" -"any_valueOrDefault" -"any_valueOrNull" -"any_valueResample" -"any_valueSimpleState" -"any_valueState" -"any_value_respect_nulls" -"any_value_respect_nullsArgMax" -"any_value_respect_nullsArgMin" -"any_value_respect_nullsArray" -"any_value_respect_nullsDistinct" -"any_value_respect_nullsForEach" -"any_value_respect_nullsIf" -"any_value_respect_nullsMap" -"any_value_respect_nullsMerge" -"any_value_respect_nullsNull" -"any_value_respect_nullsOrDefault" -"any_value_respect_nullsOrNull" -"any_value_respect_nullsResample" -"any_value_respect_nullsSimpleState" -"any_value_respect_nullsState" -"appendTrailingCharIfAbsent" -"approx_top_count" -"approx_top_countArgMax" -"approx_top_countArgMin" -"approx_top_countArray" -"approx_top_countDistinct" -"approx_top_countForEach" -"approx_top_countIf" -"approx_top_countMap" -"approx_top_countMerge" -"approx_top_countNull" -"approx_top_countOrDefault" -"approx_top_countOrNull" -"approx_top_countResample" -"approx_top_countSimpleState" -"approx_top_countState" -"approx_top_k" -"approx_top_kArgMax" -"approx_top_kArgMin" -"approx_top_kArray" -"approx_top_kDistinct" -"approx_top_kForEach" -"approx_top_kIf" -"approx_top_kMap" -"approx_top_kMerge" -"approx_top_kNull" -"approx_top_kOrDefault" -"approx_top_kOrNull" -"approx_top_kResample" -"approx_top_kSimpleState" -"approx_top_kState" -"approx_top_sum" -"approx_top_sumArgMax" -"approx_top_sumArgMin" -"approx_top_sumArray" -"approx_top_sumDistinct" -"approx_top_sumForEach" -"approx_top_sumIf" -"approx_top_sumMap" -"approx_top_sumMerge" -"approx_top_sumNull" -"approx_top_sumOrDefault" -"approx_top_sumOrNull" -"approx_top_sumResample" -"approx_top_sumSimpleState" -"approx_top_sumState" -"argMax" -"argMaxArgMax" -"argMaxArgMin" -"argMaxArray" -"argMaxDistinct" -"argMaxForEach" -"argMaxIf" -"argMaxMap" -"argMaxMerge" -"argMaxNull" -"argMaxOrDefault" -"argMaxOrNull" -"argMaxResample" -"argMaxSimpleState" -"argMaxState" -"argMin" -"argMinArgMax" -"argMinArgMin" -"argMinArray" -"argMinDistinct" -"argMinForEach" -"argMinIf" -"argMinMap" -"argMinMerge" -"argMinNull" -"argMinOrDefault" -"argMinOrNull" -"argMinResample" -"argMinSimpleState" -"argMinState" -"array" -"arrayAUC" -"arrayAll" -"arrayAvg" -"arrayCompact" -"arrayConcat" -"arrayCount" -"arrayCumSum" -"arrayCumSumNonNegative" -"arrayDifference" -"arrayDistinct" -"arrayDotProduct" -"arrayElement" -"arrayEnumerate" -"arrayEnumerateDense" -"arrayEnumerateDenseRanked" -"arrayEnumerateUniq" -"arrayEnumerateUniqRanked" -"arrayExists" -"arrayFill" -"arrayFilter" -"arrayFirst" -"arrayFirstIndex" -"arrayFirstOrNull" -"arrayFlatten" -"arrayFold" -"arrayIntersect" -"arrayJaccardIndex" -"arrayJoin" -"arrayLast" -"arrayLastIndex" -"arrayLastOrNull" -"arrayMap" -"arrayMax" -"arrayMin" -"arrayPartialReverseSort" -"arrayPartialShuffle" -"arrayPartialSort" -"arrayPopBack" -"arrayPopFront" -"arrayProduct" -"arrayPushBack" -"arrayPushFront" -"arrayRandomSample" -"arrayReduce" -"arrayReduceInRanges" -"arrayResize" -"arrayReverse" -"arrayReverseFill" -"arrayReverseSort" -"arrayReverseSplit" -"arrayRotateLeft" -"arrayRotateRight" -"arrayShiftLeft" -"arrayShiftRight" -"arrayShingles" -"arrayShuffle" -"arraySlice" -"arraySort" -"arraySplit" -"arrayStringConcat" -"arraySum" -"arrayUniq" -"arrayWithConstant" -"arrayZip" -"array_agg" -"array_aggArgMax" -"array_aggArgMin" -"array_aggArray" -"array_aggDistinct" -"array_aggForEach" -"array_aggIf" -"array_aggMap" -"array_aggMerge" -"array_aggNull" -"array_aggOrDefault" -"array_aggOrNull" -"array_aggResample" -"array_aggSimpleState" -"array_aggState" -"array_concat_agg" -"array_concat_aggArgMax" -"array_concat_aggArgMin" -"array_concat_aggArray" -"array_concat_aggDistinct" -"array_concat_aggForEach" -"array_concat_aggIf" -"array_concat_aggMap" -"array_concat_aggMerge" -"array_concat_aggNull" -"array_concat_aggOrDefault" -"array_concat_aggOrNull" -"array_concat_aggResample" -"array_concat_aggSimpleState" -"array_concat_aggState" -"ascii" -"asin" -"asinh" -"assumeNotNull" -"atan" -"atan2" -"atanh" -"avg" -"avgArgMax" -"avgArgMin" -"avgArray" -"avgDistinct" -"avgForEach" -"avgIf" -"avgMap" -"avgMerge" -"avgNull" -"avgOrDefault" -"avgOrNull" -"avgResample" -"avgSimpleState" -"avgState" -"avgWeighted" -"avgWeightedArgMax" -"avgWeightedArgMin" -"avgWeightedArray" -"avgWeightedDistinct" -"avgWeightedForEach" -"avgWeightedIf" -"avgWeightedMap" -"avgWeightedMerge" -"avgWeightedNull" -"avgWeightedOrDefault" -"avgWeightedOrNull" -"avgWeightedResample" -"avgWeightedSimpleState" -"avgWeightedState" -"bar" -"base58Decode" -"base58Encode" -"base64Decode" -"base64Encode" -"base64URLDecode" -"base64URLEncode" -"basename" -"bin" -"bitAnd" -"bitCount" -"bitHammingDistance" -"bitNot" -"bitOr" -"bitPositionsToArray" -"bitRotateLeft" -"bitRotateRight" -"bitShiftLeft" -"bitShiftRight" -"bitSlice" -"bitTest" -"bitTestAll" -"bitTestAny" -"bitXor" -"bitmapAnd" -"bitmapAndCardinality" -"bitmapAndnot" -"bitmapAndnotCardinality" -"bitmapBuild" -"bitmapCardinality" -"bitmapContains" -"bitmapHasAll" -"bitmapHasAny" -"bitmapMax" -"bitmapMin" -"bitmapOr" -"bitmapOrCardinality" -"bitmapSubsetInRange" -"bitmapSubsetLimit" -"bitmapToArray" -"bitmapTransform" -"bitmapXor" -"bitmapXorCardinality" -"bitmaskToArray" -"bitmaskToList" -"blockNumber" -"blockSerializedSize" -"blockSize" -"boundingRatio" -"boundingRatioArgMax" -"boundingRatioArgMin" -"boundingRatioArray" -"boundingRatioDistinct" -"boundingRatioForEach" -"boundingRatioIf" -"boundingRatioMap" -"boundingRatioMerge" -"boundingRatioNull" -"boundingRatioOrDefault" -"boundingRatioOrNull" -"boundingRatioResample" -"boundingRatioSimpleState" -"boundingRatioState" -"buildId" -"byteHammingDistance" -"byteSize" -"byteSlice" -"byteSwap" -"caseWithExpr" -"caseWithExpression" -"caseWithoutExpr" -"caseWithoutExpression" -"catboostEvaluate" -"categoricalInformationValue" -"categoricalInformationValueArgMax" -"categoricalInformationValueArgMin" -"categoricalInformationValueArray" -"categoricalInformationValueDistinct" -"categoricalInformationValueForEach" -"categoricalInformationValueIf" -"categoricalInformationValueMap" -"categoricalInformationValueMerge" -"categoricalInformationValueNull" -"categoricalInformationValueOrDefault" -"categoricalInformationValueOrNull" -"categoricalInformationValueResample" -"categoricalInformationValueSimpleState" -"categoricalInformationValueState" -"cbrt" -"ceil" -"ceiling" -"changeDay" -"changeHour" -"changeMinute" -"changeMonth" -"changeSecond" -"changeYear" -"char" -"cityHash64" -"clamp" -"coalesce" -"concat" -"concatAssumeInjective" -"concatWithSeparator" -"concatWithSeparatorAssumeInjective" -"concat_ws" -"connectionId" -"connection_id" -"contingency" -"contingencyArgMax" -"contingencyArgMin" -"contingencyArray" -"contingencyDistinct" -"contingencyForEach" -"contingencyIf" -"contingencyMap" -"contingencyMerge" -"contingencyNull" -"contingencyOrDefault" -"contingencyOrNull" -"contingencyResample" -"contingencySimpleState" -"contingencyState" -"convertCharset" -"corr" -"corrArgMax" -"corrArgMin" -"corrArray" -"corrDistinct" -"corrForEach" -"corrIf" -"corrMap" -"corrMatrix" -"corrMatrixArgMax" -"corrMatrixArgMin" -"corrMatrixArray" -"corrMatrixDistinct" -"corrMatrixForEach" -"corrMatrixIf" -"corrMatrixMap" -"corrMatrixMerge" -"corrMatrixNull" -"corrMatrixOrDefault" -"corrMatrixOrNull" -"corrMatrixResample" -"corrMatrixSimpleState" -"corrMatrixState" -"corrMerge" -"corrNull" -"corrOrDefault" -"corrOrNull" -"corrResample" -"corrSimpleState" -"corrStable" -"corrStableArgMax" -"corrStableArgMin" -"corrStableArray" -"corrStableDistinct" -"corrStableForEach" -"corrStableIf" -"corrStableMap" -"corrStableMerge" -"corrStableNull" -"corrStableOrDefault" -"corrStableOrNull" -"corrStableResample" -"corrStableSimpleState" -"corrStableState" -"corrState" -"cos" -"cosh" -"cosineDistance" -"count" -"countArgMax" -"countArgMin" -"countArray" -"countDigits" -"countDistinct" -"countEqual" -"countForEach" -"countIf" -"countMap" -"countMatches" -"countMatchesCaseInsensitive" -"countMerge" -"countNull" -"countOrDefault" -"countOrNull" -"countResample" -"countSimpleState" -"countState" -"countSubstrings" -"countSubstringsCaseInsensitive" -"countSubstringsCaseInsensitiveUTF8" -"covarPop" -"covarPopArgMax" -"covarPopArgMin" -"covarPopArray" -"covarPopDistinct" -"covarPopForEach" -"covarPopIf" -"covarPopMap" -"covarPopMatrix" -"covarPopMatrixArgMax" -"covarPopMatrixArgMin" -"covarPopMatrixArray" -"covarPopMatrixDistinct" -"covarPopMatrixForEach" -"covarPopMatrixIf" -"covarPopMatrixMap" -"covarPopMatrixMerge" -"covarPopMatrixNull" -"covarPopMatrixOrDefault" -"covarPopMatrixOrNull" -"covarPopMatrixResample" -"covarPopMatrixSimpleState" -"covarPopMatrixState" -"covarPopMerge" -"covarPopNull" -"covarPopOrDefault" -"covarPopOrNull" -"covarPopResample" -"covarPopSimpleState" -"covarPopStable" -"covarPopStableArgMax" -"covarPopStableArgMin" -"covarPopStableArray" -"covarPopStableDistinct" -"covarPopStableForEach" -"covarPopStableIf" -"covarPopStableMap" -"covarPopStableMerge" -"covarPopStableNull" -"covarPopStableOrDefault" -"covarPopStableOrNull" -"covarPopStableResample" -"covarPopStableSimpleState" -"covarPopStableState" -"covarPopState" -"covarSamp" -"covarSampArgMax" -"covarSampArgMin" -"covarSampArray" -"covarSampDistinct" -"covarSampForEach" -"covarSampIf" -"covarSampMap" -"covarSampMatrix" -"covarSampMatrixArgMax" -"covarSampMatrixArgMin" -"covarSampMatrixArray" -"covarSampMatrixDistinct" -"covarSampMatrixForEach" -"covarSampMatrixIf" -"covarSampMatrixMap" -"covarSampMatrixMerge" -"covarSampMatrixNull" -"covarSampMatrixOrDefault" -"covarSampMatrixOrNull" -"covarSampMatrixResample" -"covarSampMatrixSimpleState" -"covarSampMatrixState" -"covarSampMerge" -"covarSampNull" -"covarSampOrDefault" -"covarSampOrNull" -"covarSampResample" -"covarSampSimpleState" -"covarSampStable" -"covarSampStableArgMax" -"covarSampStableArgMin" -"covarSampStableArray" -"covarSampStableDistinct" -"covarSampStableForEach" -"covarSampStableIf" -"covarSampStableMap" -"covarSampStableMerge" -"covarSampStableNull" -"covarSampStableOrDefault" -"covarSampStableOrNull" -"covarSampStableResample" -"covarSampStableSimpleState" -"covarSampStableState" -"covarSampState" -"cramersV" -"cramersVArgMax" -"cramersVArgMin" -"cramersVArray" -"cramersVBiasCorrected" -"cramersVBiasCorrectedArgMax" -"cramersVBiasCorrectedArgMin" -"cramersVBiasCorrectedArray" -"cramersVBiasCorrectedDistinct" -"cramersVBiasCorrectedForEach" -"cramersVBiasCorrectedIf" -"cramersVBiasCorrectedMap" -"cramersVBiasCorrectedMerge" -"cramersVBiasCorrectedNull" -"cramersVBiasCorrectedOrDefault" -"cramersVBiasCorrectedOrNull" -"cramersVBiasCorrectedResample" -"cramersVBiasCorrectedSimpleState" -"cramersVBiasCorrectedState" -"cramersVDistinct" -"cramersVForEach" -"cramersVIf" -"cramersVMap" -"cramersVMerge" -"cramersVNull" -"cramersVOrDefault" -"cramersVOrNull" -"cramersVResample" -"cramersVSimpleState" -"cramersVState" -"curdate" -"currentDatabase" -"currentProfiles" -"currentRoles" -"currentSchemas" -"currentUser" -"current_database" -"current_date" -"current_schemas" -"current_timestamp" -"current_user" -"cutFragment" -"cutIPv6" -"cutQueryString" -"cutQueryStringAndFragment" -"cutToFirstSignificantSubdomain" -"cutToFirstSignificantSubdomainCustom" -"cutToFirstSignificantSubdomainCustomRFC" -"cutToFirstSignificantSubdomainCustomWithWWW" -"cutToFirstSignificantSubdomainCustomWithWWWRFC" -"cutToFirstSignificantSubdomainRFC" -"cutToFirstSignificantSubdomainWithWWW" -"cutToFirstSignificantSubdomainWithWWWRFC" -"cutURLParameter" -"cutWWW" -"damerauLevenshteinDistance" -"dateDiff" -"dateName" -"dateTime64ToSnowflake" -"dateTime64ToSnowflakeID" -"dateTimeToSnowflake" -"dateTimeToSnowflakeID" -"dateTrunc" -"date_diff" -"decodeHTMLComponent" -"decodeURLComponent" -"decodeURLFormComponent" -"decodeXMLComponent" -"decrypt" -"defaultProfiles" -"defaultRoles" -"defaultValueOfArgumentType" -"defaultValueOfTypeName" -"degrees" -"deltaSum" -"deltaSumArgMax" -"deltaSumArgMin" -"deltaSumArray" -"deltaSumDistinct" -"deltaSumForEach" -"deltaSumIf" -"deltaSumMap" -"deltaSumMerge" -"deltaSumNull" -"deltaSumOrDefault" -"deltaSumOrNull" -"deltaSumResample" -"deltaSumSimpleState" -"deltaSumState" -"deltaSumTimestamp" -"deltaSumTimestampArgMax" -"deltaSumTimestampArgMin" -"deltaSumTimestampArray" -"deltaSumTimestampDistinct" -"deltaSumTimestampForEach" -"deltaSumTimestampIf" -"deltaSumTimestampMap" -"deltaSumTimestampMerge" -"deltaSumTimestampNull" -"deltaSumTimestampOrDefault" -"deltaSumTimestampOrNull" -"deltaSumTimestampResample" -"deltaSumTimestampSimpleState" -"deltaSumTimestampState" -"demangle" -"denseRank" -"denseRankArgMax" -"denseRankArgMin" -"denseRankArray" -"denseRankDistinct" -"denseRankForEach" -"denseRankIf" -"denseRankMap" -"denseRankMerge" -"denseRankNull" -"denseRankOrDefault" -"denseRankOrNull" -"denseRankResample" -"denseRankSimpleState" -"denseRankState" -"dense_rank" -"dense_rankArgMax" -"dense_rankArgMin" -"dense_rankArray" -"dense_rankDistinct" -"dense_rankForEach" -"dense_rankIf" -"dense_rankMap" -"dense_rankMerge" -"dense_rankNull" -"dense_rankOrDefault" -"dense_rankOrNull" -"dense_rankResample" -"dense_rankSimpleState" -"dense_rankState" -"detectCharset" -"detectLanguage" -"detectLanguageMixed" -"detectLanguageUnknown" -"detectProgrammingLanguage" -"detectTonality" -"dictGet" -"dictGetAll" -"dictGetChildren" -"dictGetDate" -"dictGetDateOrDefault" -"dictGetDateTime" -"dictGetDateTimeOrDefault" -"dictGetDescendants" -"dictGetFloat32" -"dictGetFloat32OrDefault" -"dictGetFloat64" -"dictGetFloat64OrDefault" -"dictGetHierarchy" -"dictGetIPv4" -"dictGetIPv4OrDefault" -"dictGetIPv6" -"dictGetIPv6OrDefault" -"dictGetInt16" -"dictGetInt16OrDefault" -"dictGetInt32" -"dictGetInt32OrDefault" -"dictGetInt64" -"dictGetInt64OrDefault" -"dictGetInt8" -"dictGetInt8OrDefault" -"dictGetOrDefault" -"dictGetOrNull" -"dictGetString" -"dictGetStringOrDefault" -"dictGetUInt16" -"dictGetUInt16OrDefault" -"dictGetUInt32" -"dictGetUInt32OrDefault" -"dictGetUInt64" -"dictGetUInt64OrDefault" -"dictGetUInt8" -"dictGetUInt8OrDefault" -"dictGetUUID" -"dictGetUUIDOrDefault" -"dictHas" -"dictIsIn" -"displayName" -"distanceL1" -"distanceL2" -"distanceL2Squared" -"distanceLinf" -"distanceLp" -"divide" -"divideDecimal" -"domain" -"domainRFC" -"domainWithoutWWW" -"domainWithoutWWWRFC" -"dotProduct" -"dumpColumnStructure" -"dynamicElement" -"dynamicType" -"e" -"editDistance" -"editDistanceUTF8" -"empty" -"emptyArrayDate" -"emptyArrayDateTime" -"emptyArrayFloat32" -"emptyArrayFloat64" -"emptyArrayInt16" -"emptyArrayInt32" -"emptyArrayInt64" -"emptyArrayInt8" -"emptyArrayString" -"emptyArrayToSingle" -"emptyArrayUInt16" -"emptyArrayUInt32" -"emptyArrayUInt64" -"emptyArrayUInt8" -"enabledProfiles" -"enabledRoles" -"encodeURLComponent" -"encodeURLFormComponent" -"encodeXMLComponent" -"encrypt" -"endsWith" -"endsWithUTF8" -"entropy" -"entropyArgMax" -"entropyArgMin" -"entropyArray" -"entropyDistinct" -"entropyForEach" -"entropyIf" -"entropyMap" -"entropyMerge" -"entropyNull" -"entropyOrDefault" -"entropyOrNull" -"entropyResample" -"entropySimpleState" -"entropyState" -"equals" -"erf" -"erfc" -"errorCodeToName" -"evalMLMethod" -"exp" -"exp10" -"exp2" -"exponentialMovingAverage" -"exponentialMovingAverageArgMax" -"exponentialMovingAverageArgMin" -"exponentialMovingAverageArray" -"exponentialMovingAverageDistinct" -"exponentialMovingAverageForEach" -"exponentialMovingAverageIf" -"exponentialMovingAverageMap" -"exponentialMovingAverageMerge" -"exponentialMovingAverageNull" -"exponentialMovingAverageOrDefault" -"exponentialMovingAverageOrNull" -"exponentialMovingAverageResample" -"exponentialMovingAverageSimpleState" -"exponentialMovingAverageState" -"exponentialTimeDecayedAvg" -"exponentialTimeDecayedAvgArgMax" -"exponentialTimeDecayedAvgArgMin" -"exponentialTimeDecayedAvgArray" -"exponentialTimeDecayedAvgDistinct" -"exponentialTimeDecayedAvgForEach" -"exponentialTimeDecayedAvgIf" -"exponentialTimeDecayedAvgMap" -"exponentialTimeDecayedAvgMerge" -"exponentialTimeDecayedAvgNull" -"exponentialTimeDecayedAvgOrDefault" -"exponentialTimeDecayedAvgOrNull" -"exponentialTimeDecayedAvgResample" -"exponentialTimeDecayedAvgSimpleState" -"exponentialTimeDecayedAvgState" -"exponentialTimeDecayedCount" -"exponentialTimeDecayedCountArgMax" -"exponentialTimeDecayedCountArgMin" -"exponentialTimeDecayedCountArray" -"exponentialTimeDecayedCountDistinct" -"exponentialTimeDecayedCountForEach" -"exponentialTimeDecayedCountIf" -"exponentialTimeDecayedCountMap" -"exponentialTimeDecayedCountMerge" -"exponentialTimeDecayedCountNull" -"exponentialTimeDecayedCountOrDefault" -"exponentialTimeDecayedCountOrNull" -"exponentialTimeDecayedCountResample" -"exponentialTimeDecayedCountSimpleState" -"exponentialTimeDecayedCountState" -"exponentialTimeDecayedMax" -"exponentialTimeDecayedMaxArgMax" -"exponentialTimeDecayedMaxArgMin" -"exponentialTimeDecayedMaxArray" -"exponentialTimeDecayedMaxDistinct" -"exponentialTimeDecayedMaxForEach" -"exponentialTimeDecayedMaxIf" -"exponentialTimeDecayedMaxMap" -"exponentialTimeDecayedMaxMerge" -"exponentialTimeDecayedMaxNull" -"exponentialTimeDecayedMaxOrDefault" -"exponentialTimeDecayedMaxOrNull" -"exponentialTimeDecayedMaxResample" -"exponentialTimeDecayedMaxSimpleState" -"exponentialTimeDecayedMaxState" -"exponentialTimeDecayedSum" -"exponentialTimeDecayedSumArgMax" -"exponentialTimeDecayedSumArgMin" -"exponentialTimeDecayedSumArray" -"exponentialTimeDecayedSumDistinct" -"exponentialTimeDecayedSumForEach" -"exponentialTimeDecayedSumIf" -"exponentialTimeDecayedSumMap" -"exponentialTimeDecayedSumMerge" -"exponentialTimeDecayedSumNull" -"exponentialTimeDecayedSumOrDefault" -"exponentialTimeDecayedSumOrNull" -"exponentialTimeDecayedSumResample" -"exponentialTimeDecayedSumSimpleState" -"exponentialTimeDecayedSumState" -"extract" -"extractAll" -"extractAllGroups" -"extractAllGroupsHorizontal" -"extractAllGroupsVertical" -"extractGroups" -"extractKeyValuePairs" -"extractKeyValuePairsWithEscaping" -"extractTextFromHTML" -"extractURLParameter" -"extractURLParameterNames" -"extractURLParameters" -"factorial" -"farmFingerprint64" -"farmHash64" -"file" -"filesystemAvailable" -"filesystemCapacity" -"filesystemUnreserved" -"finalizeAggregation" -"firstLine" -"firstSignificantSubdomain" -"firstSignificantSubdomainCustom" -"firstSignificantSubdomainCustomRFC" -"firstSignificantSubdomainRFC" -"first_value" -"first_valueArgMax" -"first_valueArgMin" -"first_valueArray" -"first_valueDistinct" -"first_valueForEach" -"first_valueIf" -"first_valueMap" -"first_valueMerge" -"first_valueNull" -"first_valueOrDefault" -"first_valueOrNull" -"first_valueResample" -"first_valueSimpleState" -"first_valueState" -"first_value_respect_nulls" -"first_value_respect_nullsArgMax" -"first_value_respect_nullsArgMin" -"first_value_respect_nullsArray" -"first_value_respect_nullsDistinct" -"first_value_respect_nullsForEach" -"first_value_respect_nullsIf" -"first_value_respect_nullsMap" -"first_value_respect_nullsMerge" -"first_value_respect_nullsNull" -"first_value_respect_nullsOrDefault" -"first_value_respect_nullsOrNull" -"first_value_respect_nullsResample" -"first_value_respect_nullsSimpleState" -"first_value_respect_nullsState" -"flameGraph" -"flameGraphArgMax" -"flameGraphArgMin" -"flameGraphArray" -"flameGraphDistinct" -"flameGraphForEach" -"flameGraphIf" -"flameGraphMap" -"flameGraphMerge" -"flameGraphNull" -"flameGraphOrDefault" -"flameGraphOrNull" -"flameGraphResample" -"flameGraphSimpleState" -"flameGraphState" -"flatten" -"flattenTuple" -"floor" -"format" -"formatDateTime" -"formatDateTimeInJodaSyntax" -"formatQuery" -"formatQueryOrNull" -"formatQuerySingleLine" -"formatQuerySingleLineOrNull" -"formatReadableDecimalSize" -"formatReadableQuantity" -"formatReadableSize" -"formatReadableTimeDelta" -"formatRow" -"formatRowNoNewline" -"fragment" -"fromDaysSinceYearZero" -"fromDaysSinceYearZero32" -"fromModifiedJulianDay" -"fromModifiedJulianDayOrNull" -"fromUTCTimestamp" -"fromUnixTimestamp" -"fromUnixTimestamp64Micro" -"fromUnixTimestamp64Milli" -"fromUnixTimestamp64Nano" -"fromUnixTimestampInJodaSyntax" -"from_utc_timestamp" -"fullHostName" -"fuzzBits" -"gccMurmurHash" -"gcd" -"generateRandomStructure" -"generateSnowflakeID" -"generateULID" -"generateUUIDv4" -"generateUUIDv7" -"geoDistance" -"geoToH3" -"geoToS2" -"geohashDecode" -"geohashEncode" -"geohashesInBox" -"getClientHTTPHeader" -"getMacro" -"getOSKernelVersion" -"getServerPort" -"getSetting" -"getSizeOfEnumType" -"getSubcolumn" -"getTypeSerializationStreams" -"globalIn" -"globalInIgnoreSet" -"globalNotIn" -"globalNotInIgnoreSet" -"globalNotNullIn" -"globalNotNullInIgnoreSet" -"globalNullIn" -"globalNullInIgnoreSet" -"globalVariable" -"greatCircleAngle" -"greatCircleDistance" -"greater" -"greaterOrEquals" -"greatest" -"groupArray" -"groupArrayArgMax" -"groupArrayArgMin" -"groupArrayArray" -"groupArrayDistinct" -"groupArrayForEach" -"groupArrayIf" -"groupArrayInsertAt" -"groupArrayInsertAtArgMax" -"groupArrayInsertAtArgMin" -"groupArrayInsertAtArray" -"groupArrayInsertAtDistinct" -"groupArrayInsertAtForEach" -"groupArrayInsertAtIf" -"groupArrayInsertAtMap" -"groupArrayInsertAtMerge" -"groupArrayInsertAtNull" -"groupArrayInsertAtOrDefault" -"groupArrayInsertAtOrNull" -"groupArrayInsertAtResample" -"groupArrayInsertAtSimpleState" -"groupArrayInsertAtState" -"groupArrayIntersect" -"groupArrayIntersectArgMax" -"groupArrayIntersectArgMin" -"groupArrayIntersectArray" -"groupArrayIntersectDistinct" -"groupArrayIntersectForEach" -"groupArrayIntersectIf" -"groupArrayIntersectMap" -"groupArrayIntersectMerge" -"groupArrayIntersectNull" -"groupArrayIntersectOrDefault" -"groupArrayIntersectOrNull" -"groupArrayIntersectResample" -"groupArrayIntersectSimpleState" -"groupArrayIntersectState" -"groupArrayLast" -"groupArrayLastArgMax" -"groupArrayLastArgMin" -"groupArrayLastArray" -"groupArrayLastDistinct" -"groupArrayLastForEach" -"groupArrayLastIf" -"groupArrayLastMap" -"groupArrayLastMerge" -"groupArrayLastNull" -"groupArrayLastOrDefault" -"groupArrayLastOrNull" -"groupArrayLastResample" -"groupArrayLastSimpleState" -"groupArrayLastState" -"groupArrayMap" -"groupArrayMerge" -"groupArrayMovingAvg" -"groupArrayMovingAvgArgMax" -"groupArrayMovingAvgArgMin" -"groupArrayMovingAvgArray" -"groupArrayMovingAvgDistinct" -"groupArrayMovingAvgForEach" -"groupArrayMovingAvgIf" -"groupArrayMovingAvgMap" -"groupArrayMovingAvgMerge" -"groupArrayMovingAvgNull" -"groupArrayMovingAvgOrDefault" -"groupArrayMovingAvgOrNull" -"groupArrayMovingAvgResample" -"groupArrayMovingAvgSimpleState" -"groupArrayMovingAvgState" -"groupArrayMovingSum" -"groupArrayMovingSumArgMax" -"groupArrayMovingSumArgMin" -"groupArrayMovingSumArray" -"groupArrayMovingSumDistinct" -"groupArrayMovingSumForEach" -"groupArrayMovingSumIf" -"groupArrayMovingSumMap" -"groupArrayMovingSumMerge" -"groupArrayMovingSumNull" -"groupArrayMovingSumOrDefault" -"groupArrayMovingSumOrNull" -"groupArrayMovingSumResample" -"groupArrayMovingSumSimpleState" -"groupArrayMovingSumState" -"groupArrayNull" -"groupArrayOrDefault" -"groupArrayOrNull" -"groupArrayResample" -"groupArraySample" -"groupArraySampleArgMax" -"groupArraySampleArgMin" -"groupArraySampleArray" -"groupArraySampleDistinct" -"groupArraySampleForEach" -"groupArraySampleIf" -"groupArraySampleMap" -"groupArraySampleMerge" -"groupArraySampleNull" -"groupArraySampleOrDefault" -"groupArraySampleOrNull" -"groupArraySampleResample" -"groupArraySampleSimpleState" -"groupArraySampleState" -"groupArraySimpleState" -"groupArraySorted" -"groupArraySortedArgMax" -"groupArraySortedArgMin" -"groupArraySortedArray" -"groupArraySortedDistinct" -"groupArraySortedForEach" -"groupArraySortedIf" -"groupArraySortedMap" -"groupArraySortedMerge" -"groupArraySortedNull" -"groupArraySortedOrDefault" -"groupArraySortedOrNull" -"groupArraySortedResample" -"groupArraySortedSimpleState" -"groupArraySortedState" -"groupArrayState" -"groupBitAnd" -"groupBitAndArgMax" -"groupBitAndArgMin" -"groupBitAndArray" -"groupBitAndDistinct" -"groupBitAndForEach" -"groupBitAndIf" -"groupBitAndMap" -"groupBitAndMerge" -"groupBitAndNull" -"groupBitAndOrDefault" -"groupBitAndOrNull" -"groupBitAndResample" -"groupBitAndSimpleState" -"groupBitAndState" -"groupBitOr" -"groupBitOrArgMax" -"groupBitOrArgMin" -"groupBitOrArray" -"groupBitOrDistinct" -"groupBitOrForEach" -"groupBitOrIf" -"groupBitOrMap" -"groupBitOrMerge" -"groupBitOrNull" -"groupBitOrOrDefault" -"groupBitOrOrNull" -"groupBitOrResample" -"groupBitOrSimpleState" -"groupBitOrState" -"groupBitXor" -"groupBitXorArgMax" -"groupBitXorArgMin" -"groupBitXorArray" -"groupBitXorDistinct" -"groupBitXorForEach" -"groupBitXorIf" -"groupBitXorMap" -"groupBitXorMerge" -"groupBitXorNull" -"groupBitXorOrDefault" -"groupBitXorOrNull" -"groupBitXorResample" -"groupBitXorSimpleState" -"groupBitXorState" -"groupBitmap" -"groupBitmapAnd" -"groupBitmapAndArgMax" -"groupBitmapAndArgMin" -"groupBitmapAndArray" -"groupBitmapAndDistinct" -"groupBitmapAndForEach" -"groupBitmapAndIf" -"groupBitmapAndMap" -"groupBitmapAndMerge" -"groupBitmapAndNull" -"groupBitmapAndOrDefault" -"groupBitmapAndOrNull" -"groupBitmapAndResample" -"groupBitmapAndSimpleState" -"groupBitmapAndState" -"groupBitmapArgMax" -"groupBitmapArgMin" -"groupBitmapArray" -"groupBitmapDistinct" -"groupBitmapForEach" -"groupBitmapIf" -"groupBitmapMap" -"groupBitmapMerge" -"groupBitmapNull" -"groupBitmapOr" -"groupBitmapOrArgMax" -"groupBitmapOrArgMin" -"groupBitmapOrArray" -"groupBitmapOrDefault" -"groupBitmapOrDistinct" -"groupBitmapOrForEach" -"groupBitmapOrIf" -"groupBitmapOrMap" -"groupBitmapOrMerge" -"groupBitmapOrNull" -"groupBitmapOrNull" -"groupBitmapOrOrDefault" -"groupBitmapOrOrNull" -"groupBitmapOrResample" -"groupBitmapOrSimpleState" -"groupBitmapOrState" -"groupBitmapResample" -"groupBitmapSimpleState" -"groupBitmapState" -"groupBitmapXor" -"groupBitmapXorArgMax" -"groupBitmapXorArgMin" -"groupBitmapXorArray" -"groupBitmapXorDistinct" -"groupBitmapXorForEach" -"groupBitmapXorIf" -"groupBitmapXorMap" -"groupBitmapXorMerge" -"groupBitmapXorNull" -"groupBitmapXorOrDefault" -"groupBitmapXorOrNull" -"groupBitmapXorResample" -"groupBitmapXorSimpleState" -"groupBitmapXorState" -"groupConcat" -"groupConcatArgMax" -"groupConcatArgMin" -"groupConcatArray" -"groupConcatDistinct" -"groupConcatForEach" -"groupConcatIf" -"groupConcatMap" -"groupConcatMerge" -"groupConcatNull" -"groupConcatOrDefault" -"groupConcatOrNull" -"groupConcatResample" -"groupConcatSimpleState" -"groupConcatState" -"groupUniqArray" -"groupUniqArrayArgMax" -"groupUniqArrayArgMin" -"groupUniqArrayArray" -"groupUniqArrayDistinct" -"groupUniqArrayForEach" -"groupUniqArrayIf" -"groupUniqArrayMap" -"groupUniqArrayMerge" -"groupUniqArrayNull" -"groupUniqArrayOrDefault" -"groupUniqArrayOrNull" -"groupUniqArrayResample" -"groupUniqArraySimpleState" -"groupUniqArrayState" -"group_concat" -"group_concatArgMax" -"group_concatArgMin" -"group_concatArray" -"group_concatDistinct" -"group_concatForEach" -"group_concatIf" -"group_concatMap" -"group_concatMerge" -"group_concatNull" -"group_concatOrDefault" -"group_concatOrNull" -"group_concatResample" -"group_concatSimpleState" -"group_concatState" -"h3CellAreaM2" -"h3CellAreaRads2" -"h3Distance" -"h3EdgeAngle" -"h3EdgeLengthKm" -"h3EdgeLengthM" -"h3ExactEdgeLengthKm" -"h3ExactEdgeLengthM" -"h3ExactEdgeLengthRads" -"h3GetBaseCell" -"h3GetDestinationIndexFromUnidirectionalEdge" -"h3GetFaces" -"h3GetIndexesFromUnidirectionalEdge" -"h3GetOriginIndexFromUnidirectionalEdge" -"h3GetPentagonIndexes" -"h3GetRes0Indexes" -"h3GetResolution" -"h3GetUnidirectionalEdge" -"h3GetUnidirectionalEdgeBoundary" -"h3GetUnidirectionalEdgesFromHexagon" -"h3HexAreaKm2" -"h3HexAreaM2" -"h3HexRing" -"h3IndexesAreNeighbors" -"h3IsPentagon" -"h3IsResClassIII" -"h3IsValid" -"h3Line" -"h3NumHexagons" -"h3PointDistKm" -"h3PointDistM" -"h3PointDistRads" -"h3ToCenterChild" -"h3ToChildren" -"h3ToGeo" -"h3ToGeoBoundary" -"h3ToParent" -"h3ToString" -"h3UnidirectionalEdgeIsValid" -"h3kRing" -"halfMD5" -"has" -"hasAll" -"hasAny" -"hasColumnInTable" -"hasSubsequence" -"hasSubsequenceCaseInsensitive" -"hasSubsequenceCaseInsensitiveUTF8" -"hasSubsequenceUTF8" -"hasSubstr" -"hasThreadFuzzer" -"hasToken" -"hasTokenCaseInsensitive" -"hasTokenCaseInsensitiveOrNull" -"hasTokenOrNull" -"hex" -"hilbertDecode" -"hilbertEncode" -"histogram" -"histogramArgMax" -"histogramArgMin" -"histogramArray" -"histogramDistinct" -"histogramForEach" -"histogramIf" -"histogramMap" -"histogramMerge" -"histogramNull" -"histogramOrDefault" -"histogramOrNull" -"histogramResample" -"histogramSimpleState" -"histogramState" -"hiveHash" -"hop" -"hopEnd" -"hopStart" -"hostName" -"hostname" -"hypot" -"identity" -"idnaDecode" -"idnaEncode" -"if" -"ifNotFinite" -"ifNull" -"ignore" -"ilike" -"in" -"inIgnoreSet" -"indexHint" -"indexOf" -"initcap" -"initcapUTF8" -"initialQueryID" -"initial_query_id" -"initializeAggregation" -"instr" -"intDiv" -"intDivOrZero" -"intExp10" -"intExp2" -"intHash32" -"intHash64" -"intervalLengthSum" -"intervalLengthSumArgMax" -"intervalLengthSumArgMin" -"intervalLengthSumArray" -"intervalLengthSumDistinct" -"intervalLengthSumForEach" -"intervalLengthSumIf" -"intervalLengthSumMap" -"intervalLengthSumMerge" -"intervalLengthSumNull" -"intervalLengthSumOrDefault" -"intervalLengthSumOrNull" -"intervalLengthSumResample" -"intervalLengthSumSimpleState" -"intervalLengthSumState" -"isConstant" -"isDecimalOverflow" -"isFinite" -"isIPAddressInRange" -"isIPv4String" -"isIPv6String" -"isInfinite" -"isNaN" -"isNotDistinctFrom" -"isNotNull" -"isNull" -"isNullable" -"isValidJSON" -"isValidUTF8" -"isZeroOrNull" -"jaroSimilarity" -"jaroWinklerSimilarity" -"javaHash" -"javaHashUTF16LE" -"joinGet" -"joinGetOrNull" -"jsonMergePatch" -"jumpConsistentHash" -"kafkaMurmurHash" -"kolmogorovSmirnovTest" -"kolmogorovSmirnovTestArgMax" -"kolmogorovSmirnovTestArgMin" -"kolmogorovSmirnovTestArray" -"kolmogorovSmirnovTestDistinct" -"kolmogorovSmirnovTestForEach" -"kolmogorovSmirnovTestIf" -"kolmogorovSmirnovTestMap" -"kolmogorovSmirnovTestMerge" -"kolmogorovSmirnovTestNull" -"kolmogorovSmirnovTestOrDefault" -"kolmogorovSmirnovTestOrNull" -"kolmogorovSmirnovTestResample" -"kolmogorovSmirnovTestSimpleState" -"kolmogorovSmirnovTestState" -"kostikConsistentHash" -"kql_array_sort_asc" -"kql_array_sort_desc" -"kurtPop" -"kurtPopArgMax" -"kurtPopArgMin" -"kurtPopArray" -"kurtPopDistinct" -"kurtPopForEach" -"kurtPopIf" -"kurtPopMap" -"kurtPopMerge" -"kurtPopNull" -"kurtPopOrDefault" -"kurtPopOrNull" -"kurtPopResample" -"kurtPopSimpleState" -"kurtPopState" -"kurtSamp" -"kurtSampArgMax" -"kurtSampArgMin" -"kurtSampArray" -"kurtSampDistinct" -"kurtSampForEach" -"kurtSampIf" -"kurtSampMap" -"kurtSampMerge" -"kurtSampNull" -"kurtSampOrDefault" -"kurtSampOrNull" -"kurtSampResample" -"kurtSampSimpleState" -"kurtSampState" -"lagInFrame" -"lagInFrameArgMax" -"lagInFrameArgMin" -"lagInFrameArray" -"lagInFrameDistinct" -"lagInFrameForEach" -"lagInFrameIf" -"lagInFrameMap" -"lagInFrameMerge" -"lagInFrameNull" -"lagInFrameOrDefault" -"lagInFrameOrNull" -"lagInFrameResample" -"lagInFrameSimpleState" -"lagInFrameState" -"largestTriangleThreeBuckets" -"largestTriangleThreeBucketsArgMax" -"largestTriangleThreeBucketsArgMin" -"largestTriangleThreeBucketsArray" -"largestTriangleThreeBucketsDistinct" -"largestTriangleThreeBucketsForEach" -"largestTriangleThreeBucketsIf" -"largestTriangleThreeBucketsMap" -"largestTriangleThreeBucketsMerge" -"largestTriangleThreeBucketsNull" -"largestTriangleThreeBucketsOrDefault" -"largestTriangleThreeBucketsOrNull" -"largestTriangleThreeBucketsResample" -"largestTriangleThreeBucketsSimpleState" -"largestTriangleThreeBucketsState" -"last_value" -"last_valueArgMax" -"last_valueArgMin" -"last_valueArray" -"last_valueDistinct" -"last_valueForEach" -"last_valueIf" -"last_valueMap" -"last_valueMerge" -"last_valueNull" -"last_valueOrDefault" -"last_valueOrNull" -"last_valueResample" -"last_valueSimpleState" -"last_valueState" -"last_value_respect_nulls" -"last_value_respect_nullsArgMax" -"last_value_respect_nullsArgMin" -"last_value_respect_nullsArray" -"last_value_respect_nullsDistinct" -"last_value_respect_nullsForEach" -"last_value_respect_nullsIf" -"last_value_respect_nullsMap" -"last_value_respect_nullsMerge" -"last_value_respect_nullsNull" -"last_value_respect_nullsOrDefault" -"last_value_respect_nullsOrNull" -"last_value_respect_nullsResample" -"last_value_respect_nullsSimpleState" -"last_value_respect_nullsState" -"lcase" -"lcm" -"leadInFrame" -"leadInFrameArgMax" -"leadInFrameArgMin" -"leadInFrameArray" -"leadInFrameDistinct" -"leadInFrameForEach" -"leadInFrameIf" -"leadInFrameMap" -"leadInFrameMerge" -"leadInFrameNull" -"leadInFrameOrDefault" -"leadInFrameOrNull" -"leadInFrameResample" -"leadInFrameSimpleState" -"leadInFrameState" -"least" -"left" -"leftPad" -"leftPadUTF8" -"leftUTF8" -"lemmatize" -"length" -"lengthUTF8" -"less" -"lessOrEquals" -"levenshteinDistance" -"levenshteinDistanceUTF8" -"lgamma" -"like" -"ln" -"locate" -"log" -"log10" -"log1p" -"log2" -"logTrace" -"lowCardinalityIndices" -"lowCardinalityKeys" -"lower" -"lowerUTF8" -"lpad" -"ltrim" -"lttb" -"lttbArgMax" -"lttbArgMin" -"lttbArray" -"lttbDistinct" -"lttbForEach" -"lttbIf" -"lttbMap" -"lttbMerge" -"lttbNull" -"lttbOrDefault" -"lttbOrNull" -"lttbResample" -"lttbSimpleState" -"lttbState" -"makeDate" -"makeDate32" -"makeDateTime" -"makeDateTime64" -"mannWhitneyUTest" -"mannWhitneyUTestArgMax" -"mannWhitneyUTestArgMin" -"mannWhitneyUTestArray" -"mannWhitneyUTestDistinct" -"mannWhitneyUTestForEach" -"mannWhitneyUTestIf" -"mannWhitneyUTestMap" -"mannWhitneyUTestMerge" -"mannWhitneyUTestNull" -"mannWhitneyUTestOrDefault" -"mannWhitneyUTestOrNull" -"mannWhitneyUTestResample" -"mannWhitneyUTestSimpleState" -"mannWhitneyUTestState" -"map" -"mapAdd" -"mapAll" -"mapApply" -"mapConcat" -"mapContains" -"mapContainsKeyLike" -"mapExists" -"mapExtractKeyLike" -"mapFilter" -"mapFromArrays" -"mapFromString" -"mapKeys" -"mapPartialReverseSort" -"mapPartialSort" -"mapPopulateSeries" -"mapReverseSort" -"mapSort" -"mapSubtract" -"mapUpdate" -"mapValues" -"match" -"materialize" -"max" -"max2" -"maxArgMax" -"maxArgMin" -"maxArray" -"maxDistinct" -"maxForEach" -"maxIf" -"maxIntersections" -"maxIntersectionsArgMax" -"maxIntersectionsArgMin" -"maxIntersectionsArray" -"maxIntersectionsDistinct" -"maxIntersectionsForEach" -"maxIntersectionsIf" -"maxIntersectionsMap" -"maxIntersectionsMerge" -"maxIntersectionsNull" -"maxIntersectionsOrDefault" -"maxIntersectionsOrNull" -"maxIntersectionsPosition" -"maxIntersectionsPositionArgMax" -"maxIntersectionsPositionArgMin" -"maxIntersectionsPositionArray" -"maxIntersectionsPositionDistinct" -"maxIntersectionsPositionForEach" -"maxIntersectionsPositionIf" -"maxIntersectionsPositionMap" -"maxIntersectionsPositionMerge" -"maxIntersectionsPositionNull" -"maxIntersectionsPositionOrDefault" -"maxIntersectionsPositionOrNull" -"maxIntersectionsPositionResample" -"maxIntersectionsPositionSimpleState" -"maxIntersectionsPositionState" -"maxIntersectionsResample" -"maxIntersectionsSimpleState" -"maxIntersectionsState" -"maxMap" -"maxMappedArrays" -"maxMappedArraysArgMax" -"maxMappedArraysArgMin" -"maxMappedArraysArray" -"maxMappedArraysDistinct" -"maxMappedArraysForEach" -"maxMappedArraysIf" -"maxMappedArraysMap" -"maxMappedArraysMerge" -"maxMappedArraysNull" -"maxMappedArraysOrDefault" -"maxMappedArraysOrNull" -"maxMappedArraysResample" -"maxMappedArraysSimpleState" -"maxMappedArraysState" -"maxMerge" -"maxNull" -"maxOrDefault" -"maxOrNull" -"maxResample" -"maxSimpleState" -"maxState" -"meanZTest" -"meanZTestArgMax" -"meanZTestArgMin" -"meanZTestArray" -"meanZTestDistinct" -"meanZTestForEach" -"meanZTestIf" -"meanZTestMap" -"meanZTestMerge" -"meanZTestNull" -"meanZTestOrDefault" -"meanZTestOrNull" -"meanZTestResample" -"meanZTestSimpleState" -"meanZTestState" -"median" -"medianArgMax" -"medianArgMin" -"medianArray" -"medianBFloat16" -"medianBFloat16ArgMax" -"medianBFloat16ArgMin" -"medianBFloat16Array" -"medianBFloat16Distinct" -"medianBFloat16ForEach" -"medianBFloat16If" -"medianBFloat16Map" -"medianBFloat16Merge" -"medianBFloat16Null" -"medianBFloat16OrDefault" -"medianBFloat16OrNull" -"medianBFloat16Resample" -"medianBFloat16SimpleState" -"medianBFloat16State" -"medianBFloat16Weighted" -"medianBFloat16WeightedArgMax" -"medianBFloat16WeightedArgMin" -"medianBFloat16WeightedArray" -"medianBFloat16WeightedDistinct" -"medianBFloat16WeightedForEach" -"medianBFloat16WeightedIf" -"medianBFloat16WeightedMap" -"medianBFloat16WeightedMerge" -"medianBFloat16WeightedNull" -"medianBFloat16WeightedOrDefault" -"medianBFloat16WeightedOrNull" -"medianBFloat16WeightedResample" -"medianBFloat16WeightedSimpleState" -"medianBFloat16WeightedState" -"medianDD" -"medianDDArgMax" -"medianDDArgMin" -"medianDDArray" -"medianDDDistinct" -"medianDDForEach" -"medianDDIf" -"medianDDMap" -"medianDDMerge" -"medianDDNull" -"medianDDOrDefault" -"medianDDOrNull" -"medianDDResample" -"medianDDSimpleState" -"medianDDState" -"medianDeterministic" -"medianDeterministicArgMax" -"medianDeterministicArgMin" -"medianDeterministicArray" -"medianDeterministicDistinct" -"medianDeterministicForEach" -"medianDeterministicIf" -"medianDeterministicMap" -"medianDeterministicMerge" -"medianDeterministicNull" -"medianDeterministicOrDefault" -"medianDeterministicOrNull" -"medianDeterministicResample" -"medianDeterministicSimpleState" -"medianDeterministicState" -"medianDistinct" -"medianExact" -"medianExactArgMax" -"medianExactArgMin" -"medianExactArray" -"medianExactDistinct" -"medianExactForEach" -"medianExactHigh" -"medianExactHighArgMax" -"medianExactHighArgMin" -"medianExactHighArray" -"medianExactHighDistinct" -"medianExactHighForEach" -"medianExactHighIf" -"medianExactHighMap" -"medianExactHighMerge" -"medianExactHighNull" -"medianExactHighOrDefault" -"medianExactHighOrNull" -"medianExactHighResample" -"medianExactHighSimpleState" -"medianExactHighState" -"medianExactIf" -"medianExactLow" -"medianExactLowArgMax" -"medianExactLowArgMin" -"medianExactLowArray" -"medianExactLowDistinct" -"medianExactLowForEach" -"medianExactLowIf" -"medianExactLowMap" -"medianExactLowMerge" -"medianExactLowNull" -"medianExactLowOrDefault" -"medianExactLowOrNull" -"medianExactLowResample" -"medianExactLowSimpleState" -"medianExactLowState" -"medianExactMap" -"medianExactMerge" -"medianExactNull" -"medianExactOrDefault" -"medianExactOrNull" -"medianExactResample" -"medianExactSimpleState" -"medianExactState" -"medianExactWeighted" -"medianExactWeightedArgMax" -"medianExactWeightedArgMin" -"medianExactWeightedArray" -"medianExactWeightedDistinct" -"medianExactWeightedForEach" -"medianExactWeightedIf" -"medianExactWeightedMap" -"medianExactWeightedMerge" -"medianExactWeightedNull" -"medianExactWeightedOrDefault" -"medianExactWeightedOrNull" -"medianExactWeightedResample" -"medianExactWeightedSimpleState" -"medianExactWeightedState" -"medianForEach" -"medianGK" -"medianGKArgMax" -"medianGKArgMin" -"medianGKArray" -"medianGKDistinct" -"medianGKForEach" -"medianGKIf" -"medianGKMap" -"medianGKMerge" -"medianGKNull" -"medianGKOrDefault" -"medianGKOrNull" -"medianGKResample" -"medianGKSimpleState" -"medianGKState" -"medianIf" -"medianInterpolatedWeighted" -"medianInterpolatedWeightedArgMax" -"medianInterpolatedWeightedArgMin" -"medianInterpolatedWeightedArray" -"medianInterpolatedWeightedDistinct" -"medianInterpolatedWeightedForEach" -"medianInterpolatedWeightedIf" -"medianInterpolatedWeightedMap" -"medianInterpolatedWeightedMerge" -"medianInterpolatedWeightedNull" -"medianInterpolatedWeightedOrDefault" -"medianInterpolatedWeightedOrNull" -"medianInterpolatedWeightedResample" -"medianInterpolatedWeightedSimpleState" -"medianInterpolatedWeightedState" -"medianMap" -"medianMerge" -"medianNull" -"medianOrDefault" -"medianOrNull" -"medianResample" -"medianSimpleState" -"medianState" -"medianTDigest" -"medianTDigestArgMax" -"medianTDigestArgMin" -"medianTDigestArray" -"medianTDigestDistinct" -"medianTDigestForEach" -"medianTDigestIf" -"medianTDigestMap" -"medianTDigestMerge" -"medianTDigestNull" -"medianTDigestOrDefault" -"medianTDigestOrNull" -"medianTDigestResample" -"medianTDigestSimpleState" -"medianTDigestState" -"medianTDigestWeighted" -"medianTDigestWeightedArgMax" -"medianTDigestWeightedArgMin" -"medianTDigestWeightedArray" -"medianTDigestWeightedDistinct" -"medianTDigestWeightedForEach" -"medianTDigestWeightedIf" -"medianTDigestWeightedMap" -"medianTDigestWeightedMerge" -"medianTDigestWeightedNull" -"medianTDigestWeightedOrDefault" -"medianTDigestWeightedOrNull" -"medianTDigestWeightedResample" -"medianTDigestWeightedSimpleState" -"medianTDigestWeightedState" -"medianTiming" -"medianTimingArgMax" -"medianTimingArgMin" -"medianTimingArray" -"medianTimingDistinct" -"medianTimingForEach" -"medianTimingIf" -"medianTimingMap" -"medianTimingMerge" -"medianTimingNull" -"medianTimingOrDefault" -"medianTimingOrNull" -"medianTimingResample" -"medianTimingSimpleState" -"medianTimingState" -"medianTimingWeighted" -"medianTimingWeightedArgMax" -"medianTimingWeightedArgMin" -"medianTimingWeightedArray" -"medianTimingWeightedDistinct" -"medianTimingWeightedForEach" -"medianTimingWeightedIf" -"medianTimingWeightedMap" -"medianTimingWeightedMerge" -"medianTimingWeightedNull" -"medianTimingWeightedOrDefault" -"medianTimingWeightedOrNull" -"medianTimingWeightedResample" -"medianTimingWeightedSimpleState" -"medianTimingWeightedState" -"metroHash64" -"mid" -"min" -"min2" -"minArgMax" -"minArgMin" -"minArray" -"minDistinct" -"minForEach" -"minIf" -"minMap" -"minMappedArrays" -"minMappedArraysArgMax" -"minMappedArraysArgMin" -"minMappedArraysArray" -"minMappedArraysDistinct" -"minMappedArraysForEach" -"minMappedArraysIf" -"minMappedArraysMap" -"minMappedArraysMerge" -"minMappedArraysNull" -"minMappedArraysOrDefault" -"minMappedArraysOrNull" -"minMappedArraysResample" -"minMappedArraysSimpleState" -"minMappedArraysState" -"minMerge" -"minNull" -"minOrDefault" -"minOrNull" -"minResample" -"minSampleSizeContinous" -"minSampleSizeContinuous" -"minSampleSizeConversion" -"minSimpleState" -"minState" -"minus" -"mismatches" -"mod" -"modulo" -"moduloLegacy" -"moduloOrZero" -"monthName" -"mortonDecode" -"mortonEncode" -"multiFuzzyMatchAllIndices" -"multiFuzzyMatchAny" -"multiFuzzyMatchAnyIndex" -"multiIf" -"multiMatchAllIndices" -"multiMatchAny" -"multiMatchAnyIndex" -"multiSearchAllPositions" -"multiSearchAllPositionsCaseInsensitive" -"multiSearchAllPositionsCaseInsensitiveUTF8" -"multiSearchAllPositionsUTF8" -"multiSearchAny" -"multiSearchAnyCaseInsensitive" -"multiSearchAnyCaseInsensitiveUTF8" -"multiSearchAnyUTF8" -"multiSearchFirstIndex" -"multiSearchFirstIndexCaseInsensitive" -"multiSearchFirstIndexCaseInsensitiveUTF8" -"multiSearchFirstIndexUTF8" -"multiSearchFirstPosition" -"multiSearchFirstPositionCaseInsensitive" -"multiSearchFirstPositionCaseInsensitiveUTF8" -"multiSearchFirstPositionUTF8" -"multiply" -"multiplyDecimal" -"murmurHash2_32" -"murmurHash2_64" -"murmurHash3_128" -"murmurHash3_32" -"murmurHash3_64" -"negate" -"neighbor" -"nested" -"netloc" -"ngramDistance" -"ngramDistanceCaseInsensitive" -"ngramDistanceCaseInsensitiveUTF8" -"ngramDistanceUTF8" -"ngramMinHash" -"ngramMinHashArg" -"ngramMinHashArgCaseInsensitive" -"ngramMinHashArgCaseInsensitiveUTF8" -"ngramMinHashArgUTF8" -"ngramMinHashCaseInsensitive" -"ngramMinHashCaseInsensitiveUTF8" -"ngramMinHashUTF8" -"ngramSearch" -"ngramSearchCaseInsensitive" -"ngramSearchCaseInsensitiveUTF8" -"ngramSearchUTF8" -"ngramSimHash" -"ngramSimHashCaseInsensitive" -"ngramSimHashCaseInsensitiveUTF8" -"ngramSimHashUTF8" -"ngrams" -"nonNegativeDerivative" -"nonNegativeDerivativeArgMax" -"nonNegativeDerivativeArgMin" -"nonNegativeDerivativeArray" -"nonNegativeDerivativeDistinct" -"nonNegativeDerivativeForEach" -"nonNegativeDerivativeIf" -"nonNegativeDerivativeMap" -"nonNegativeDerivativeMerge" -"nonNegativeDerivativeNull" -"nonNegativeDerivativeOrDefault" -"nonNegativeDerivativeOrNull" -"nonNegativeDerivativeResample" -"nonNegativeDerivativeSimpleState" -"nonNegativeDerivativeState" -"normL1" -"normL2" -"normL2Squared" -"normLinf" -"normLp" -"normalizeL1" -"normalizeL2" -"normalizeLinf" -"normalizeLp" -"normalizeQuery" -"normalizeQueryKeepNames" -"normalizeUTF8NFC" -"normalizeUTF8NFD" -"normalizeUTF8NFKC" -"normalizeUTF8NFKD" -"normalizedQueryHash" -"normalizedQueryHashKeepNames" -"not" -"notEmpty" -"notEquals" -"notILike" -"notIn" -"notInIgnoreSet" -"notLike" -"notNullIn" -"notNullInIgnoreSet" -"nothing" -"nothingArgMax" -"nothingArgMin" -"nothingArray" -"nothingDistinct" -"nothingForEach" -"nothingIf" -"nothingMap" -"nothingMerge" -"nothingNull" -"nothingNull" -"nothingNullArgMax" -"nothingNullArgMin" -"nothingNullArray" -"nothingNullDistinct" -"nothingNullForEach" -"nothingNullIf" -"nothingNullMap" -"nothingNullMerge" -"nothingNullNull" -"nothingNullOrDefault" -"nothingNullOrNull" -"nothingNullResample" -"nothingNullSimpleState" -"nothingNullState" -"nothingOrDefault" -"nothingOrNull" -"nothingResample" -"nothingSimpleState" -"nothingState" -"nothingUInt64" -"nothingUInt64ArgMax" -"nothingUInt64ArgMin" -"nothingUInt64Array" -"nothingUInt64Distinct" -"nothingUInt64ForEach" -"nothingUInt64If" -"nothingUInt64Map" -"nothingUInt64Merge" -"nothingUInt64Null" -"nothingUInt64OrDefault" -"nothingUInt64OrNull" -"nothingUInt64Resample" -"nothingUInt64SimpleState" -"nothingUInt64State" -"now" -"now64" -"nowInBlock" -"nth_value" -"nth_valueArgMax" -"nth_valueArgMin" -"nth_valueArray" -"nth_valueDistinct" -"nth_valueForEach" -"nth_valueIf" -"nth_valueMap" -"nth_valueMerge" -"nth_valueNull" -"nth_valueOrDefault" -"nth_valueOrNull" -"nth_valueResample" -"nth_valueSimpleState" -"nth_valueState" -"ntile" -"ntileArgMax" -"ntileArgMin" -"ntileArray" -"ntileDistinct" -"ntileForEach" -"ntileIf" -"ntileMap" -"ntileMerge" -"ntileNull" -"ntileOrDefault" -"ntileOrNull" -"ntileResample" -"ntileSimpleState" -"ntileState" -"nullIf" -"nullIn" -"nullInIgnoreSet" -"or" -"parseDateTime" -"parseDateTime32BestEffort" -"parseDateTime32BestEffortOrNull" -"parseDateTime32BestEffortOrZero" -"parseDateTime64BestEffort" -"parseDateTime64BestEffortOrNull" -"parseDateTime64BestEffortOrZero" -"parseDateTime64BestEffortUS" -"parseDateTime64BestEffortUSOrNull" -"parseDateTime64BestEffortUSOrZero" -"parseDateTimeBestEffort" -"parseDateTimeBestEffortOrNull" -"parseDateTimeBestEffortOrZero" -"parseDateTimeBestEffortUS" -"parseDateTimeBestEffortUSOrNull" -"parseDateTimeBestEffortUSOrZero" -"parseDateTimeInJodaSyntax" -"parseDateTimeInJodaSyntaxOrNull" -"parseDateTimeInJodaSyntaxOrZero" -"parseDateTimeOrNull" -"parseDateTimeOrZero" -"parseReadableSize" -"parseReadableSizeOrNull" -"parseReadableSizeOrZero" -"parseTimeDelta" -"partitionID" -"partitionId" -"path" -"pathFull" -"percentRank" -"percentRankArgMax" -"percentRankArgMin" -"percentRankArray" -"percentRankDistinct" -"percentRankForEach" -"percentRankIf" -"percentRankMap" -"percentRankMerge" -"percentRankNull" -"percentRankOrDefault" -"percentRankOrNull" -"percentRankResample" -"percentRankSimpleState" -"percentRankState" -"percent_rank" -"percent_rankArgMax" -"percent_rankArgMin" -"percent_rankArray" -"percent_rankDistinct" -"percent_rankForEach" -"percent_rankIf" -"percent_rankMap" -"percent_rankMerge" -"percent_rankNull" -"percent_rankOrDefault" -"percent_rankOrNull" -"percent_rankResample" -"percent_rankSimpleState" -"percent_rankState" -"pi" -"plus" -"pmod" -"pointInEllipses" -"pointInPolygon" -"polygonAreaCartesian" -"polygonAreaSpherical" -"polygonConvexHullCartesian" -"polygonPerimeterCartesian" -"polygonPerimeterSpherical" -"polygonsDistanceCartesian" -"polygonsDistanceSpherical" -"polygonsEqualsCartesian" -"polygonsIntersectionCartesian" -"polygonsIntersectionSpherical" -"polygonsSymDifferenceCartesian" -"polygonsSymDifferenceSpherical" -"polygonsUnionCartesian" -"polygonsUnionSpherical" -"polygonsWithinCartesian" -"polygonsWithinSpherical" -"port" -"portRFC" -"position" -"positionCaseInsensitive" -"positionCaseInsensitiveUTF8" -"positionUTF8" -"positiveModulo" -"positive_modulo" -"pow" -"power" -"printf" -"proportionsZTest" -"protocol" -"punycodeDecode" -"punycodeEncode" -"quantile" -"quantileArgMax" -"quantileArgMin" -"quantileArray" -"quantileBFloat16" -"quantileBFloat16ArgMax" -"quantileBFloat16ArgMin" -"quantileBFloat16Array" -"quantileBFloat16Distinct" -"quantileBFloat16ForEach" -"quantileBFloat16If" -"quantileBFloat16Map" -"quantileBFloat16Merge" -"quantileBFloat16Null" -"quantileBFloat16OrDefault" -"quantileBFloat16OrNull" -"quantileBFloat16Resample" -"quantileBFloat16SimpleState" -"quantileBFloat16State" -"quantileBFloat16Weighted" -"quantileBFloat16WeightedArgMax" -"quantileBFloat16WeightedArgMin" -"quantileBFloat16WeightedArray" -"quantileBFloat16WeightedDistinct" -"quantileBFloat16WeightedForEach" -"quantileBFloat16WeightedIf" -"quantileBFloat16WeightedMap" -"quantileBFloat16WeightedMerge" -"quantileBFloat16WeightedNull" -"quantileBFloat16WeightedOrDefault" -"quantileBFloat16WeightedOrNull" -"quantileBFloat16WeightedResample" -"quantileBFloat16WeightedSimpleState" -"quantileBFloat16WeightedState" -"quantileDD" -"quantileDDArgMax" -"quantileDDArgMin" -"quantileDDArray" -"quantileDDDistinct" -"quantileDDForEach" -"quantileDDIf" -"quantileDDMap" -"quantileDDMerge" -"quantileDDNull" -"quantileDDOrDefault" -"quantileDDOrNull" -"quantileDDResample" -"quantileDDSimpleState" -"quantileDDState" -"quantileDeterministic" -"quantileDeterministicArgMax" -"quantileDeterministicArgMin" -"quantileDeterministicArray" -"quantileDeterministicDistinct" -"quantileDeterministicForEach" -"quantileDeterministicIf" -"quantileDeterministicMap" -"quantileDeterministicMerge" -"quantileDeterministicNull" -"quantileDeterministicOrDefault" -"quantileDeterministicOrNull" -"quantileDeterministicResample" -"quantileDeterministicSimpleState" -"quantileDeterministicState" -"quantileDistinct" -"quantileExact" -"quantileExactArgMax" -"quantileExactArgMin" -"quantileExactArray" -"quantileExactDistinct" -"quantileExactExclusive" -"quantileExactExclusiveArgMax" -"quantileExactExclusiveArgMin" -"quantileExactExclusiveArray" -"quantileExactExclusiveDistinct" -"quantileExactExclusiveForEach" -"quantileExactExclusiveIf" -"quantileExactExclusiveMap" -"quantileExactExclusiveMerge" -"quantileExactExclusiveNull" -"quantileExactExclusiveOrDefault" -"quantileExactExclusiveOrNull" -"quantileExactExclusiveResample" -"quantileExactExclusiveSimpleState" -"quantileExactExclusiveState" -"quantileExactForEach" -"quantileExactHigh" -"quantileExactHighArgMax" -"quantileExactHighArgMin" -"quantileExactHighArray" -"quantileExactHighDistinct" -"quantileExactHighForEach" -"quantileExactHighIf" -"quantileExactHighMap" -"quantileExactHighMerge" -"quantileExactHighNull" -"quantileExactHighOrDefault" -"quantileExactHighOrNull" -"quantileExactHighResample" -"quantileExactHighSimpleState" -"quantileExactHighState" -"quantileExactIf" -"quantileExactInclusive" -"quantileExactInclusiveArgMax" -"quantileExactInclusiveArgMin" -"quantileExactInclusiveArray" -"quantileExactInclusiveDistinct" -"quantileExactInclusiveForEach" -"quantileExactInclusiveIf" -"quantileExactInclusiveMap" -"quantileExactInclusiveMerge" -"quantileExactInclusiveNull" -"quantileExactInclusiveOrDefault" -"quantileExactInclusiveOrNull" -"quantileExactInclusiveResample" -"quantileExactInclusiveSimpleState" -"quantileExactInclusiveState" -"quantileExactLow" -"quantileExactLowArgMax" -"quantileExactLowArgMin" -"quantileExactLowArray" -"quantileExactLowDistinct" -"quantileExactLowForEach" -"quantileExactLowIf" -"quantileExactLowMap" -"quantileExactLowMerge" -"quantileExactLowNull" -"quantileExactLowOrDefault" -"quantileExactLowOrNull" -"quantileExactLowResample" -"quantileExactLowSimpleState" -"quantileExactLowState" -"quantileExactMap" -"quantileExactMerge" -"quantileExactNull" -"quantileExactOrDefault" -"quantileExactOrNull" -"quantileExactResample" -"quantileExactSimpleState" -"quantileExactState" -"quantileExactWeighted" -"quantileExactWeightedArgMax" -"quantileExactWeightedArgMin" -"quantileExactWeightedArray" -"quantileExactWeightedDistinct" -"quantileExactWeightedForEach" -"quantileExactWeightedIf" -"quantileExactWeightedMap" -"quantileExactWeightedMerge" -"quantileExactWeightedNull" -"quantileExactWeightedOrDefault" -"quantileExactWeightedOrNull" -"quantileExactWeightedResample" -"quantileExactWeightedSimpleState" -"quantileExactWeightedState" -"quantileForEach" -"quantileGK" -"quantileGKArgMax" -"quantileGKArgMin" -"quantileGKArray" -"quantileGKDistinct" -"quantileGKForEach" -"quantileGKIf" -"quantileGKMap" -"quantileGKMerge" -"quantileGKNull" -"quantileGKOrDefault" -"quantileGKOrNull" -"quantileGKResample" -"quantileGKSimpleState" -"quantileGKState" -"quantileIf" -"quantileInterpolatedWeighted" -"quantileInterpolatedWeightedArgMax" -"quantileInterpolatedWeightedArgMin" -"quantileInterpolatedWeightedArray" -"quantileInterpolatedWeightedDistinct" -"quantileInterpolatedWeightedForEach" -"quantileInterpolatedWeightedIf" -"quantileInterpolatedWeightedMap" -"quantileInterpolatedWeightedMerge" -"quantileInterpolatedWeightedNull" -"quantileInterpolatedWeightedOrDefault" -"quantileInterpolatedWeightedOrNull" -"quantileInterpolatedWeightedResample" -"quantileInterpolatedWeightedSimpleState" -"quantileInterpolatedWeightedState" -"quantileMap" -"quantileMerge" -"quantileNull" -"quantileOrDefault" -"quantileOrNull" -"quantileResample" -"quantileSimpleState" -"quantileState" -"quantileTDigest" -"quantileTDigestArgMax" -"quantileTDigestArgMin" -"quantileTDigestArray" -"quantileTDigestDistinct" -"quantileTDigestForEach" -"quantileTDigestIf" -"quantileTDigestMap" -"quantileTDigestMerge" -"quantileTDigestNull" -"quantileTDigestOrDefault" -"quantileTDigestOrNull" -"quantileTDigestResample" -"quantileTDigestSimpleState" -"quantileTDigestState" -"quantileTDigestWeighted" -"quantileTDigestWeightedArgMax" -"quantileTDigestWeightedArgMin" -"quantileTDigestWeightedArray" -"quantileTDigestWeightedDistinct" -"quantileTDigestWeightedForEach" -"quantileTDigestWeightedIf" -"quantileTDigestWeightedMap" -"quantileTDigestWeightedMerge" -"quantileTDigestWeightedNull" -"quantileTDigestWeightedOrDefault" -"quantileTDigestWeightedOrNull" -"quantileTDigestWeightedResample" -"quantileTDigestWeightedSimpleState" -"quantileTDigestWeightedState" -"quantileTiming" -"quantileTimingArgMax" -"quantileTimingArgMin" -"quantileTimingArray" -"quantileTimingDistinct" -"quantileTimingForEach" -"quantileTimingIf" -"quantileTimingMap" -"quantileTimingMerge" -"quantileTimingNull" -"quantileTimingOrDefault" -"quantileTimingOrNull" -"quantileTimingResample" -"quantileTimingSimpleState" -"quantileTimingState" -"quantileTimingWeighted" -"quantileTimingWeightedArgMax" -"quantileTimingWeightedArgMin" -"quantileTimingWeightedArray" -"quantileTimingWeightedDistinct" -"quantileTimingWeightedForEach" -"quantileTimingWeightedIf" -"quantileTimingWeightedMap" -"quantileTimingWeightedMerge" -"quantileTimingWeightedNull" -"quantileTimingWeightedOrDefault" -"quantileTimingWeightedOrNull" -"quantileTimingWeightedResample" -"quantileTimingWeightedSimpleState" -"quantileTimingWeightedState" -"quantiles" -"quantilesArgMax" -"quantilesArgMin" -"quantilesArray" -"quantilesBFloat16" -"quantilesBFloat16ArgMax" -"quantilesBFloat16ArgMin" -"quantilesBFloat16Array" -"quantilesBFloat16Distinct" -"quantilesBFloat16ForEach" -"quantilesBFloat16If" -"quantilesBFloat16Map" -"quantilesBFloat16Merge" -"quantilesBFloat16Null" -"quantilesBFloat16OrDefault" -"quantilesBFloat16OrNull" -"quantilesBFloat16Resample" -"quantilesBFloat16SimpleState" -"quantilesBFloat16State" -"quantilesBFloat16Weighted" -"quantilesBFloat16WeightedArgMax" -"quantilesBFloat16WeightedArgMin" -"quantilesBFloat16WeightedArray" -"quantilesBFloat16WeightedDistinct" -"quantilesBFloat16WeightedForEach" -"quantilesBFloat16WeightedIf" -"quantilesBFloat16WeightedMap" -"quantilesBFloat16WeightedMerge" -"quantilesBFloat16WeightedNull" -"quantilesBFloat16WeightedOrDefault" -"quantilesBFloat16WeightedOrNull" -"quantilesBFloat16WeightedResample" -"quantilesBFloat16WeightedSimpleState" -"quantilesBFloat16WeightedState" -"quantilesDD" -"quantilesDDArgMax" -"quantilesDDArgMin" -"quantilesDDArray" -"quantilesDDDistinct" -"quantilesDDForEach" -"quantilesDDIf" -"quantilesDDMap" -"quantilesDDMerge" -"quantilesDDNull" -"quantilesDDOrDefault" -"quantilesDDOrNull" -"quantilesDDResample" -"quantilesDDSimpleState" -"quantilesDDState" -"quantilesDeterministic" -"quantilesDeterministicArgMax" -"quantilesDeterministicArgMin" -"quantilesDeterministicArray" -"quantilesDeterministicDistinct" -"quantilesDeterministicForEach" -"quantilesDeterministicIf" -"quantilesDeterministicMap" -"quantilesDeterministicMerge" -"quantilesDeterministicNull" -"quantilesDeterministicOrDefault" -"quantilesDeterministicOrNull" -"quantilesDeterministicResample" -"quantilesDeterministicSimpleState" -"quantilesDeterministicState" -"quantilesDistinct" -"quantilesExact" -"quantilesExactArgMax" -"quantilesExactArgMin" -"quantilesExactArray" -"quantilesExactDistinct" -"quantilesExactExclusive" -"quantilesExactExclusiveArgMax" -"quantilesExactExclusiveArgMin" -"quantilesExactExclusiveArray" -"quantilesExactExclusiveDistinct" -"quantilesExactExclusiveForEach" -"quantilesExactExclusiveIf" -"quantilesExactExclusiveMap" -"quantilesExactExclusiveMerge" -"quantilesExactExclusiveNull" -"quantilesExactExclusiveOrDefault" -"quantilesExactExclusiveOrNull" -"quantilesExactExclusiveResample" -"quantilesExactExclusiveSimpleState" -"quantilesExactExclusiveState" -"quantilesExactForEach" -"quantilesExactHigh" -"quantilesExactHighArgMax" -"quantilesExactHighArgMin" -"quantilesExactHighArray" -"quantilesExactHighDistinct" -"quantilesExactHighForEach" -"quantilesExactHighIf" -"quantilesExactHighMap" -"quantilesExactHighMerge" -"quantilesExactHighNull" -"quantilesExactHighOrDefault" -"quantilesExactHighOrNull" -"quantilesExactHighResample" -"quantilesExactHighSimpleState" -"quantilesExactHighState" -"quantilesExactIf" -"quantilesExactInclusive" -"quantilesExactInclusiveArgMax" -"quantilesExactInclusiveArgMin" -"quantilesExactInclusiveArray" -"quantilesExactInclusiveDistinct" -"quantilesExactInclusiveForEach" -"quantilesExactInclusiveIf" -"quantilesExactInclusiveMap" -"quantilesExactInclusiveMerge" -"quantilesExactInclusiveNull" -"quantilesExactInclusiveOrDefault" -"quantilesExactInclusiveOrNull" -"quantilesExactInclusiveResample" -"quantilesExactInclusiveSimpleState" -"quantilesExactInclusiveState" -"quantilesExactLow" -"quantilesExactLowArgMax" -"quantilesExactLowArgMin" -"quantilesExactLowArray" -"quantilesExactLowDistinct" -"quantilesExactLowForEach" -"quantilesExactLowIf" -"quantilesExactLowMap" -"quantilesExactLowMerge" -"quantilesExactLowNull" -"quantilesExactLowOrDefault" -"quantilesExactLowOrNull" -"quantilesExactLowResample" -"quantilesExactLowSimpleState" -"quantilesExactLowState" -"quantilesExactMap" -"quantilesExactMerge" -"quantilesExactNull" -"quantilesExactOrDefault" -"quantilesExactOrNull" -"quantilesExactResample" -"quantilesExactSimpleState" -"quantilesExactState" -"quantilesExactWeighted" -"quantilesExactWeightedArgMax" -"quantilesExactWeightedArgMin" -"quantilesExactWeightedArray" -"quantilesExactWeightedDistinct" -"quantilesExactWeightedForEach" -"quantilesExactWeightedIf" -"quantilesExactWeightedMap" -"quantilesExactWeightedMerge" -"quantilesExactWeightedNull" -"quantilesExactWeightedOrDefault" -"quantilesExactWeightedOrNull" -"quantilesExactWeightedResample" -"quantilesExactWeightedSimpleState" -"quantilesExactWeightedState" -"quantilesForEach" -"quantilesGK" -"quantilesGKArgMax" -"quantilesGKArgMin" -"quantilesGKArray" -"quantilesGKDistinct" -"quantilesGKForEach" -"quantilesGKIf" -"quantilesGKMap" -"quantilesGKMerge" -"quantilesGKNull" -"quantilesGKOrDefault" -"quantilesGKOrNull" -"quantilesGKResample" -"quantilesGKSimpleState" -"quantilesGKState" -"quantilesIf" -"quantilesInterpolatedWeighted" -"quantilesInterpolatedWeightedArgMax" -"quantilesInterpolatedWeightedArgMin" -"quantilesInterpolatedWeightedArray" -"quantilesInterpolatedWeightedDistinct" -"quantilesInterpolatedWeightedForEach" -"quantilesInterpolatedWeightedIf" -"quantilesInterpolatedWeightedMap" -"quantilesInterpolatedWeightedMerge" -"quantilesInterpolatedWeightedNull" -"quantilesInterpolatedWeightedOrDefault" -"quantilesInterpolatedWeightedOrNull" -"quantilesInterpolatedWeightedResample" -"quantilesInterpolatedWeightedSimpleState" -"quantilesInterpolatedWeightedState" -"quantilesMap" -"quantilesMerge" -"quantilesNull" -"quantilesOrDefault" -"quantilesOrNull" -"quantilesResample" -"quantilesSimpleState" -"quantilesState" -"quantilesTDigest" -"quantilesTDigestArgMax" -"quantilesTDigestArgMin" -"quantilesTDigestArray" -"quantilesTDigestDistinct" -"quantilesTDigestForEach" -"quantilesTDigestIf" -"quantilesTDigestMap" -"quantilesTDigestMerge" -"quantilesTDigestNull" -"quantilesTDigestOrDefault" -"quantilesTDigestOrNull" -"quantilesTDigestResample" -"quantilesTDigestSimpleState" -"quantilesTDigestState" -"quantilesTDigestWeighted" -"quantilesTDigestWeightedArgMax" -"quantilesTDigestWeightedArgMin" -"quantilesTDigestWeightedArray" -"quantilesTDigestWeightedDistinct" -"quantilesTDigestWeightedForEach" -"quantilesTDigestWeightedIf" -"quantilesTDigestWeightedMap" -"quantilesTDigestWeightedMerge" -"quantilesTDigestWeightedNull" -"quantilesTDigestWeightedOrDefault" -"quantilesTDigestWeightedOrNull" -"quantilesTDigestWeightedResample" -"quantilesTDigestWeightedSimpleState" -"quantilesTDigestWeightedState" -"quantilesTiming" -"quantilesTimingArgMax" -"quantilesTimingArgMin" -"quantilesTimingArray" -"quantilesTimingDistinct" -"quantilesTimingForEach" -"quantilesTimingIf" -"quantilesTimingMap" -"quantilesTimingMerge" -"quantilesTimingNull" -"quantilesTimingOrDefault" -"quantilesTimingOrNull" -"quantilesTimingResample" -"quantilesTimingSimpleState" -"quantilesTimingState" -"quantilesTimingWeighted" -"quantilesTimingWeightedArgMax" -"quantilesTimingWeightedArgMin" -"quantilesTimingWeightedArray" -"quantilesTimingWeightedDistinct" -"quantilesTimingWeightedForEach" -"quantilesTimingWeightedIf" -"quantilesTimingWeightedMap" -"quantilesTimingWeightedMerge" -"quantilesTimingWeightedNull" -"quantilesTimingWeightedOrDefault" -"quantilesTimingWeightedOrNull" -"quantilesTimingWeightedResample" -"quantilesTimingWeightedSimpleState" -"quantilesTimingWeightedState" -"queryID" -"queryString" -"queryStringAndFragment" -"query_id" -"radians" -"rand" -"rand32" -"rand64" -"randBernoulli" -"randBinomial" -"randCanonical" -"randChiSquared" -"randConstant" -"randExponential" -"randFisherF" -"randLogNormal" -"randNegativeBinomial" -"randNormal" -"randPoisson" -"randStudentT" -"randUniform" -"randomFixedString" -"randomPrintableASCII" -"randomString" -"randomStringUTF8" -"range" -"rank" -"rankArgMax" -"rankArgMin" -"rankArray" -"rankCorr" -"rankCorrArgMax" -"rankCorrArgMin" -"rankCorrArray" -"rankCorrDistinct" -"rankCorrForEach" -"rankCorrIf" -"rankCorrMap" -"rankCorrMerge" -"rankCorrNull" -"rankCorrOrDefault" -"rankCorrOrNull" -"rankCorrResample" -"rankCorrSimpleState" -"rankCorrState" -"rankDistinct" -"rankForEach" -"rankIf" -"rankMap" -"rankMerge" -"rankNull" -"rankOrDefault" -"rankOrNull" -"rankResample" -"rankSimpleState" -"rankState" -"readWKTLineString" -"readWKTMultiLineString" -"readWKTMultiPolygon" -"readWKTPoint" -"readWKTPolygon" -"readWKTRing" -"regexpExtract" -"regexpQuoteMeta" -"regionHierarchy" -"regionIn" -"regionToArea" -"regionToCity" -"regionToContinent" -"regionToCountry" -"regionToDistrict" -"regionToName" -"regionToPopulation" -"regionToTopContinent" -"reinterpret" -"reinterpretAsDate" -"reinterpretAsDateTime" -"reinterpretAsFixedString" -"reinterpretAsFloat32" -"reinterpretAsFloat64" -"reinterpretAsInt128" -"reinterpretAsInt16" -"reinterpretAsInt256" -"reinterpretAsInt32" -"reinterpretAsInt64" -"reinterpretAsInt8" -"reinterpretAsString" -"reinterpretAsUInt128" -"reinterpretAsUInt16" -"reinterpretAsUInt256" -"reinterpretAsUInt32" -"reinterpretAsUInt64" -"reinterpretAsUInt8" -"reinterpretAsUUID" -"repeat" -"replace" -"replaceAll" -"replaceOne" -"replaceRegexpAll" -"replaceRegexpOne" -"replicate" -"retention" -"retentionArgMax" -"retentionArgMin" -"retentionArray" -"retentionDistinct" -"retentionForEach" -"retentionIf" -"retentionMap" -"retentionMerge" -"retentionNull" -"retentionOrDefault" -"retentionOrNull" -"retentionResample" -"retentionSimpleState" -"retentionState" -"reverse" -"reverseUTF8" -"revision" -"right" -"rightPad" -"rightPadUTF8" -"rightUTF8" -"round" -"roundAge" -"roundBankers" -"roundDown" -"roundDuration" -"roundToExp2" -"rowNumberInAllBlocks" -"rowNumberInBlock" -"row_number" -"row_numberArgMax" -"row_numberArgMin" -"row_numberArray" -"row_numberDistinct" -"row_numberForEach" -"row_numberIf" -"row_numberMap" -"row_numberMerge" -"row_numberNull" -"row_numberOrDefault" -"row_numberOrNull" -"row_numberResample" -"row_numberSimpleState" -"row_numberState" -"rpad" -"rtrim" -"runningAccumulate" -"runningConcurrency" -"runningDifference" -"runningDifferenceStartingWithFirstValue" -"s2CapContains" -"s2CapUnion" -"s2CellsIntersect" -"s2GetNeighbors" -"s2RectAdd" -"s2RectContains" -"s2RectIntersection" -"s2RectUnion" -"s2ToGeo" -"scalarProduct" -"sequenceCount" -"sequenceCountArgMax" -"sequenceCountArgMin" -"sequenceCountArray" -"sequenceCountDistinct" -"sequenceCountForEach" -"sequenceCountIf" -"sequenceCountMap" -"sequenceCountMerge" -"sequenceCountNull" -"sequenceCountOrDefault" -"sequenceCountOrNull" -"sequenceCountResample" -"sequenceCountSimpleState" -"sequenceCountState" -"sequenceMatch" -"sequenceMatchArgMax" -"sequenceMatchArgMin" -"sequenceMatchArray" -"sequenceMatchDistinct" -"sequenceMatchForEach" -"sequenceMatchIf" -"sequenceMatchMap" -"sequenceMatchMerge" -"sequenceMatchNull" -"sequenceMatchOrDefault" -"sequenceMatchOrNull" -"sequenceMatchResample" -"sequenceMatchSimpleState" -"sequenceMatchState" -"sequenceNextNode" -"sequenceNextNodeArgMax" -"sequenceNextNodeArgMin" -"sequenceNextNodeArray" -"sequenceNextNodeDistinct" -"sequenceNextNodeForEach" -"sequenceNextNodeIf" -"sequenceNextNodeMap" -"sequenceNextNodeMerge" -"sequenceNextNodeNull" -"sequenceNextNodeOrDefault" -"sequenceNextNodeOrNull" -"sequenceNextNodeResample" -"sequenceNextNodeSimpleState" -"sequenceNextNodeState" -"seriesDecomposeSTL" -"seriesOutliersDetectTukey" -"seriesPeriodDetectFFT" -"serverTimeZone" -"serverTimezone" -"serverUUID" -"shardCount" -"shardNum" -"showCertificate" -"sigmoid" -"sign" -"simpleJSONExtractBool" -"simpleJSONExtractFloat" -"simpleJSONExtractInt" -"simpleJSONExtractRaw" -"simpleJSONExtractString" -"simpleJSONExtractUInt" -"simpleJSONHas" -"simpleLinearRegression" -"simpleLinearRegressionArgMax" -"simpleLinearRegressionArgMin" -"simpleLinearRegressionArray" -"simpleLinearRegressionDistinct" -"simpleLinearRegressionForEach" -"simpleLinearRegressionIf" -"simpleLinearRegressionMap" -"simpleLinearRegressionMerge" -"simpleLinearRegressionNull" -"simpleLinearRegressionOrDefault" -"simpleLinearRegressionOrNull" -"simpleLinearRegressionResample" -"simpleLinearRegressionSimpleState" -"simpleLinearRegressionState" -"sin" -"singleValueOrNull" -"singleValueOrNullArgMax" -"singleValueOrNullArgMin" -"singleValueOrNullArray" -"singleValueOrNullDistinct" -"singleValueOrNullForEach" -"singleValueOrNullIf" -"singleValueOrNullMap" -"singleValueOrNullMerge" -"singleValueOrNullNull" -"singleValueOrNullOrDefault" -"singleValueOrNullOrNull" -"singleValueOrNullResample" -"singleValueOrNullSimpleState" -"singleValueOrNullState" -"sinh" -"sipHash128" -"sipHash128Keyed" -"sipHash128Reference" -"sipHash128ReferenceKeyed" -"sipHash64" -"sipHash64Keyed" -"skewPop" -"skewPopArgMax" -"skewPopArgMin" -"skewPopArray" -"skewPopDistinct" -"skewPopForEach" -"skewPopIf" -"skewPopMap" -"skewPopMerge" -"skewPopNull" -"skewPopOrDefault" -"skewPopOrNull" -"skewPopResample" -"skewPopSimpleState" -"skewPopState" -"skewSamp" -"skewSampArgMax" -"skewSampArgMin" -"skewSampArray" -"skewSampDistinct" -"skewSampForEach" -"skewSampIf" -"skewSampMap" -"skewSampMerge" -"skewSampNull" -"skewSampOrDefault" -"skewSampOrNull" -"skewSampResample" -"skewSampSimpleState" -"skewSampState" -"sleep" -"sleepEachRow" -"snowflakeIDToDateTime" -"snowflakeIDToDateTime64" -"snowflakeToDateTime" -"snowflakeToDateTime64" -"soundex" -"space" -"sparkBar" -"sparkBarArgMax" -"sparkBarArgMin" -"sparkBarArray" -"sparkBarDistinct" -"sparkBarForEach" -"sparkBarIf" -"sparkBarMap" -"sparkBarMerge" -"sparkBarNull" -"sparkBarOrDefault" -"sparkBarOrNull" -"sparkBarResample" -"sparkBarSimpleState" -"sparkBarState" -"sparkbar" -"sparkbarArgMax" -"sparkbarArgMin" -"sparkbarArray" -"sparkbarDistinct" -"sparkbarForEach" -"sparkbarIf" -"sparkbarMap" -"sparkbarMerge" -"sparkbarNull" -"sparkbarOrDefault" -"sparkbarOrNull" -"sparkbarResample" -"sparkbarSimpleState" -"sparkbarState" -"splitByAlpha" -"splitByChar" -"splitByNonAlpha" -"splitByRegexp" -"splitByString" -"splitByWhitespace" -"sqid" -"sqidDecode" -"sqidEncode" -"sqrt" -"startsWith" -"startsWithUTF8" -"stddevPop" -"stddevPopArgMax" -"stddevPopArgMin" -"stddevPopArray" -"stddevPopDistinct" -"stddevPopForEach" -"stddevPopIf" -"stddevPopMap" -"stddevPopMerge" -"stddevPopNull" -"stddevPopOrDefault" -"stddevPopOrNull" -"stddevPopResample" -"stddevPopSimpleState" -"stddevPopStable" -"stddevPopStableArgMax" -"stddevPopStableArgMin" -"stddevPopStableArray" -"stddevPopStableDistinct" -"stddevPopStableForEach" -"stddevPopStableIf" -"stddevPopStableMap" -"stddevPopStableMerge" -"stddevPopStableNull" -"stddevPopStableOrDefault" -"stddevPopStableOrNull" -"stddevPopStableResample" -"stddevPopStableSimpleState" -"stddevPopStableState" -"stddevPopState" -"stddevSamp" -"stddevSampArgMax" -"stddevSampArgMin" -"stddevSampArray" -"stddevSampDistinct" -"stddevSampForEach" -"stddevSampIf" -"stddevSampMap" -"stddevSampMerge" -"stddevSampNull" -"stddevSampOrDefault" -"stddevSampOrNull" -"stddevSampResample" -"stddevSampSimpleState" -"stddevSampStable" -"stddevSampStableArgMax" -"stddevSampStableArgMin" -"stddevSampStableArray" -"stddevSampStableDistinct" -"stddevSampStableForEach" -"stddevSampStableIf" -"stddevSampStableMap" -"stddevSampStableMerge" -"stddevSampStableNull" -"stddevSampStableOrDefault" -"stddevSampStableOrNull" -"stddevSampStableResample" -"stddevSampStableSimpleState" -"stddevSampStableState" -"stddevSampState" -"stem" -"stochasticLinearRegression" -"stochasticLinearRegressionArgMax" -"stochasticLinearRegressionArgMin" -"stochasticLinearRegressionArray" -"stochasticLinearRegressionDistinct" -"stochasticLinearRegressionForEach" -"stochasticLinearRegressionIf" -"stochasticLinearRegressionMap" -"stochasticLinearRegressionMerge" -"stochasticLinearRegressionNull" -"stochasticLinearRegressionOrDefault" -"stochasticLinearRegressionOrNull" -"stochasticLinearRegressionResample" -"stochasticLinearRegressionSimpleState" -"stochasticLinearRegressionState" -"stochasticLogisticRegression" -"stochasticLogisticRegressionArgMax" -"stochasticLogisticRegressionArgMin" -"stochasticLogisticRegressionArray" -"stochasticLogisticRegressionDistinct" -"stochasticLogisticRegressionForEach" -"stochasticLogisticRegressionIf" -"stochasticLogisticRegressionMap" -"stochasticLogisticRegressionMerge" -"stochasticLogisticRegressionNull" -"stochasticLogisticRegressionOrDefault" -"stochasticLogisticRegressionOrNull" -"stochasticLogisticRegressionResample" -"stochasticLogisticRegressionSimpleState" -"stochasticLogisticRegressionState" -"str_to_date" -"str_to_map" -"stringJaccardIndex" -"stringJaccardIndexUTF8" -"stringToH3" -"structureToCapnProtoSchema" -"structureToProtobufSchema" -"studentTTest" -"studentTTestArgMax" -"studentTTestArgMin" -"studentTTestArray" -"studentTTestDistinct" -"studentTTestForEach" -"studentTTestIf" -"studentTTestMap" -"studentTTestMerge" -"studentTTestNull" -"studentTTestOrDefault" -"studentTTestOrNull" -"studentTTestResample" -"studentTTestSimpleState" -"studentTTestState" -"subBitmap" -"subDate" -"substr" -"substring" -"substringIndex" -"substringIndexUTF8" -"substringUTF8" -"subtractDays" -"subtractHours" -"subtractInterval" -"subtractMicroseconds" -"subtractMilliseconds" -"subtractMinutes" -"subtractMonths" -"subtractNanoseconds" -"subtractQuarters" -"subtractSeconds" -"subtractTupleOfIntervals" -"subtractWeeks" -"subtractYears" -"sum" -"sumArgMax" -"sumArgMin" -"sumArray" -"sumCount" -"sumCountArgMax" -"sumCountArgMin" -"sumCountArray" -"sumCountDistinct" -"sumCountForEach" -"sumCountIf" -"sumCountMap" -"sumCountMerge" -"sumCountNull" -"sumCountOrDefault" -"sumCountOrNull" -"sumCountResample" -"sumCountSimpleState" -"sumCountState" -"sumDistinct" -"sumForEach" -"sumIf" -"sumKahan" -"sumKahanArgMax" -"sumKahanArgMin" -"sumKahanArray" -"sumKahanDistinct" -"sumKahanForEach" -"sumKahanIf" -"sumKahanMap" -"sumKahanMerge" -"sumKahanNull" -"sumKahanOrDefault" -"sumKahanOrNull" -"sumKahanResample" -"sumKahanSimpleState" -"sumKahanState" -"sumMap" -"sumMapFiltered" -"sumMapFilteredArgMax" -"sumMapFilteredArgMin" -"sumMapFilteredArray" -"sumMapFilteredDistinct" -"sumMapFilteredForEach" -"sumMapFilteredIf" -"sumMapFilteredMap" -"sumMapFilteredMerge" -"sumMapFilteredNull" -"sumMapFilteredOrDefault" -"sumMapFilteredOrNull" -"sumMapFilteredResample" -"sumMapFilteredSimpleState" -"sumMapFilteredState" -"sumMapFilteredWithOverflow" -"sumMapFilteredWithOverflowArgMax" -"sumMapFilteredWithOverflowArgMin" -"sumMapFilteredWithOverflowArray" -"sumMapFilteredWithOverflowDistinct" -"sumMapFilteredWithOverflowForEach" -"sumMapFilteredWithOverflowIf" -"sumMapFilteredWithOverflowMap" -"sumMapFilteredWithOverflowMerge" -"sumMapFilteredWithOverflowNull" -"sumMapFilteredWithOverflowOrDefault" -"sumMapFilteredWithOverflowOrNull" -"sumMapFilteredWithOverflowResample" -"sumMapFilteredWithOverflowSimpleState" -"sumMapFilteredWithOverflowState" -"sumMapWithOverflow" -"sumMapWithOverflowArgMax" -"sumMapWithOverflowArgMin" -"sumMapWithOverflowArray" -"sumMapWithOverflowDistinct" -"sumMapWithOverflowForEach" -"sumMapWithOverflowIf" -"sumMapWithOverflowMap" -"sumMapWithOverflowMerge" -"sumMapWithOverflowNull" -"sumMapWithOverflowOrDefault" -"sumMapWithOverflowOrNull" -"sumMapWithOverflowResample" -"sumMapWithOverflowSimpleState" -"sumMapWithOverflowState" -"sumMappedArrays" -"sumMappedArraysArgMax" -"sumMappedArraysArgMin" -"sumMappedArraysArray" -"sumMappedArraysDistinct" -"sumMappedArraysForEach" -"sumMappedArraysIf" -"sumMappedArraysMap" -"sumMappedArraysMerge" -"sumMappedArraysNull" -"sumMappedArraysOrDefault" -"sumMappedArraysOrNull" -"sumMappedArraysResample" -"sumMappedArraysSimpleState" -"sumMappedArraysState" -"sumMerge" -"sumNull" -"sumOrDefault" -"sumOrNull" -"sumResample" -"sumSimpleState" -"sumState" -"sumWithOverflow" -"sumWithOverflowArgMax" -"sumWithOverflowArgMin" -"sumWithOverflowArray" -"sumWithOverflowDistinct" -"sumWithOverflowForEach" -"sumWithOverflowIf" -"sumWithOverflowMap" -"sumWithOverflowMerge" -"sumWithOverflowNull" -"sumWithOverflowOrDefault" -"sumWithOverflowOrNull" -"sumWithOverflowResample" -"sumWithOverflowSimpleState" -"sumWithOverflowState" -"svg" -"synonyms" -"tan" -"tanh" -"tcpPort" -"tgamma" -"theilsU" -"theilsUArgMax" -"theilsUArgMin" -"theilsUArray" -"theilsUDistinct" -"theilsUForEach" -"theilsUIf" -"theilsUMap" -"theilsUMerge" -"theilsUNull" -"theilsUOrDefault" -"theilsUOrNull" -"theilsUResample" -"theilsUSimpleState" -"theilsUState" -"throwIf" -"tid" -"timeDiff" -"timeSlot" -"timeSlots" -"timeZone" -"timeZoneOf" -"timeZoneOffset" -"timestamp" -"timestampDiff" -"timestamp_diff" -"timezone" -"timezoneOf" -"timezoneOffset" -"toBool" -"toColumnTypeName" -"toDate" -"toDate32" -"toDate32OrDefault" -"toDate32OrNull" -"toDate32OrZero" -"toDateOrDefault" -"toDateOrNull" -"toDateOrZero" -"toDateTime" -"toDateTime32" -"toDateTime64" -"toDateTime64OrDefault" -"toDateTime64OrNull" -"toDateTime64OrZero" -"toDateTimeOrDefault" -"toDateTimeOrNull" -"toDateTimeOrZero" -"toDayOfMonth" -"toDayOfWeek" -"toDayOfYear" -"toDaysSinceYearZero" -"toDecimal128" -"toDecimal128OrDefault" -"toDecimal128OrNull" -"toDecimal128OrZero" -"toDecimal256" -"toDecimal256OrDefault" -"toDecimal256OrNull" -"toDecimal256OrZero" -"toDecimal32" -"toDecimal32OrDefault" -"toDecimal32OrNull" -"toDecimal32OrZero" -"toDecimal64" -"toDecimal64OrDefault" -"toDecimal64OrNull" -"toDecimal64OrZero" -"toDecimalString" -"toFixedString" -"toFloat32" -"toFloat32OrDefault" -"toFloat32OrNull" -"toFloat32OrZero" -"toFloat64" -"toFloat64OrDefault" -"toFloat64OrNull" -"toFloat64OrZero" -"toHour" -"toIPv4" -"toIPv4OrDefault" -"toIPv4OrNull" -"toIPv4OrZero" -"toIPv6" -"toIPv6OrDefault" -"toIPv6OrNull" -"toIPv6OrZero" -"toISOWeek" -"toISOYear" -"toInt128" -"toInt128OrDefault" -"toInt128OrNull" -"toInt128OrZero" -"toInt16" -"toInt16OrDefault" -"toInt16OrNull" -"toInt16OrZero" -"toInt256" -"toInt256OrDefault" -"toInt256OrNull" -"toInt256OrZero" -"toInt32" -"toInt32OrDefault" -"toInt32OrNull" -"toInt32OrZero" -"toInt64" -"toInt64OrDefault" -"toInt64OrNull" -"toInt64OrZero" -"toInt8" -"toInt8OrDefault" -"toInt8OrNull" -"toInt8OrZero" -"toIntervalDay" -"toIntervalHour" -"toIntervalMicrosecond" -"toIntervalMillisecond" -"toIntervalMinute" -"toIntervalMonth" -"toIntervalNanosecond" -"toIntervalQuarter" -"toIntervalSecond" -"toIntervalWeek" -"toIntervalYear" -"toJSONString" -"toLastDayOfMonth" -"toLastDayOfWeek" -"toLowCardinality" -"toMillisecond" -"toMinute" -"toModifiedJulianDay" -"toModifiedJulianDayOrNull" -"toMonday" -"toMonth" -"toNullable" -"toQuarter" -"toRelativeDayNum" -"toRelativeHourNum" -"toRelativeMinuteNum" -"toRelativeMonthNum" -"toRelativeQuarterNum" -"toRelativeSecondNum" -"toRelativeWeekNum" -"toRelativeYearNum" -"toSecond" -"toStartOfDay" -"toStartOfFifteenMinutes" -"toStartOfFiveMinute" -"toStartOfFiveMinutes" -"toStartOfHour" -"toStartOfISOYear" -"toStartOfInterval" -"toStartOfMicrosecond" -"toStartOfMillisecond" -"toStartOfMinute" -"toStartOfMonth" -"toStartOfNanosecond" -"toStartOfQuarter" -"toStartOfSecond" -"toStartOfTenMinutes" -"toStartOfWeek" -"toStartOfYear" -"toString" -"toStringCutToZero" -"toTime" -"toTimeZone" -"toTimezone" -"toTypeName" -"toUInt128" -"toUInt128OrDefault" -"toUInt128OrNull" -"toUInt128OrZero" -"toUInt16" -"toUInt16OrDefault" -"toUInt16OrNull" -"toUInt16OrZero" -"toUInt256" -"toUInt256OrDefault" -"toUInt256OrNull" -"toUInt256OrZero" -"toUInt32" -"toUInt32OrDefault" -"toUInt32OrNull" -"toUInt32OrZero" -"toUInt64" -"toUInt64OrDefault" -"toUInt64OrNull" -"toUInt64OrZero" -"toUInt8" -"toUInt8OrDefault" -"toUInt8OrNull" -"toUInt8OrZero" -"toUTCTimestamp" -"toUUID" -"toUUIDOrDefault" -"toUUIDOrNull" -"toUUIDOrZero" -"toUnixTimestamp" -"toUnixTimestamp64Micro" -"toUnixTimestamp64Milli" -"toUnixTimestamp64Nano" -"toValidUTF8" -"toWeek" -"toYYYYMM" -"toYYYYMMDD" -"toYYYYMMDDhhmmss" -"toYear" -"toYearWeek" -"to_utc_timestamp" -"today" -"tokens" -"topK" -"topKArgMax" -"topKArgMin" -"topKArray" -"topKDistinct" -"topKForEach" -"topKIf" -"topKMap" -"topKMerge" -"topKNull" -"topKOrDefault" -"topKOrNull" -"topKResample" -"topKSimpleState" -"topKState" -"topKWeighted" -"topKWeightedArgMax" -"topKWeightedArgMin" -"topKWeightedArray" -"topKWeightedDistinct" -"topKWeightedForEach" -"topKWeightedIf" -"topKWeightedMap" -"topKWeightedMerge" -"topKWeightedNull" -"topKWeightedOrDefault" -"topKWeightedOrNull" -"topKWeightedResample" -"topKWeightedSimpleState" -"topKWeightedState" -"topLevelDomain" -"topLevelDomainRFC" -"transactionID" -"transactionLatestSnapshot" -"transactionOldestSnapshot" -"transform" -"translate" -"translateUTF8" -"trim" -"trimBoth" -"trimLeft" -"trimRight" -"trunc" -"truncate" -"tryBase58Decode" -"tryBase64Decode" -"tryBase64URLDecode" -"tryDecrypt" -"tryIdnaEncode" -"tryPunycodeDecode" -"tumble" -"tumbleEnd" -"tumbleStart" -"tuple" -"tupleConcat" -"tupleDivide" -"tupleDivideByNumber" -"tupleElement" -"tupleHammingDistance" -"tupleIntDiv" -"tupleIntDivByNumber" -"tupleIntDivOrZero" -"tupleIntDivOrZeroByNumber" -"tupleMinus" -"tupleModulo" -"tupleModuloByNumber" -"tupleMultiply" -"tupleMultiplyByNumber" -"tupleNames" -"tupleNegate" -"tuplePlus" -"tupleToNameValuePairs" -"ucase" -"unbin" -"unhex" -"uniq" -"uniqArgMax" -"uniqArgMin" -"uniqArray" -"uniqCombined" -"uniqCombined64" -"uniqCombined64ArgMax" -"uniqCombined64ArgMin" -"uniqCombined64Array" -"uniqCombined64Distinct" -"uniqCombined64ForEach" -"uniqCombined64If" -"uniqCombined64Map" -"uniqCombined64Merge" -"uniqCombined64Null" -"uniqCombined64OrDefault" -"uniqCombined64OrNull" -"uniqCombined64Resample" -"uniqCombined64SimpleState" -"uniqCombined64State" -"uniqCombinedArgMax" -"uniqCombinedArgMin" -"uniqCombinedArray" -"uniqCombinedDistinct" -"uniqCombinedForEach" -"uniqCombinedIf" -"uniqCombinedMap" -"uniqCombinedMerge" -"uniqCombinedNull" -"uniqCombinedOrDefault" -"uniqCombinedOrNull" -"uniqCombinedResample" -"uniqCombinedSimpleState" -"uniqCombinedState" -"uniqDistinct" -"uniqExact" -"uniqExactArgMax" -"uniqExactArgMin" -"uniqExactArray" -"uniqExactDistinct" -"uniqExactForEach" -"uniqExactIf" -"uniqExactMap" -"uniqExactMerge" -"uniqExactNull" -"uniqExactOrDefault" -"uniqExactOrNull" -"uniqExactResample" -"uniqExactSimpleState" -"uniqExactState" -"uniqForEach" -"uniqHLL12" -"uniqHLL12ArgMax" -"uniqHLL12ArgMin" -"uniqHLL12Array" -"uniqHLL12Distinct" -"uniqHLL12ForEach" -"uniqHLL12If" -"uniqHLL12Map" -"uniqHLL12Merge" -"uniqHLL12Null" -"uniqHLL12OrDefault" -"uniqHLL12OrNull" -"uniqHLL12Resample" -"uniqHLL12SimpleState" -"uniqHLL12State" -"uniqIf" -"uniqMap" -"uniqMerge" -"uniqNull" -"uniqOrDefault" -"uniqOrNull" -"uniqResample" -"uniqSimpleState" -"uniqState" -"uniqTheta" -"uniqThetaArgMax" -"uniqThetaArgMin" -"uniqThetaArray" -"uniqThetaDistinct" -"uniqThetaForEach" -"uniqThetaIf" -"uniqThetaIntersect" -"uniqThetaMap" -"uniqThetaMerge" -"uniqThetaNot" -"uniqThetaNull" -"uniqThetaOrDefault" -"uniqThetaOrNull" -"uniqThetaResample" -"uniqThetaSimpleState" -"uniqThetaState" -"uniqThetaUnion" -"uniqUpTo" -"uniqUpToArgMax" -"uniqUpToArgMin" -"uniqUpToArray" -"uniqUpToDistinct" -"uniqUpToForEach" -"uniqUpToIf" -"uniqUpToMap" -"uniqUpToMerge" -"uniqUpToNull" -"uniqUpToOrDefault" -"uniqUpToOrNull" -"uniqUpToResample" -"uniqUpToSimpleState" -"uniqUpToState" -"upper" -"upperUTF8" -"uptime" -"user" -"validateNestedArraySizes" -"varPop" -"varPopArgMax" -"varPopArgMin" -"varPopArray" -"varPopDistinct" -"varPopForEach" -"varPopIf" -"varPopMap" -"varPopMerge" -"varPopNull" -"varPopOrDefault" -"varPopOrNull" -"varPopResample" -"varPopSimpleState" -"varPopStable" -"varPopStableArgMax" -"varPopStableArgMin" -"varPopStableArray" -"varPopStableDistinct" -"varPopStableForEach" -"varPopStableIf" -"varPopStableMap" -"varPopStableMerge" -"varPopStableNull" -"varPopStableOrDefault" -"varPopStableOrNull" -"varPopStableResample" -"varPopStableSimpleState" -"varPopStableState" -"varPopState" -"varSamp" -"varSampArgMax" -"varSampArgMin" -"varSampArray" -"varSampDistinct" -"varSampForEach" -"varSampIf" -"varSampMap" -"varSampMerge" -"varSampNull" -"varSampOrDefault" -"varSampOrNull" -"varSampResample" -"varSampSimpleState" -"varSampStable" -"varSampStableArgMax" -"varSampStableArgMin" -"varSampStableArray" -"varSampStableDistinct" -"varSampStableForEach" -"varSampStableIf" -"varSampStableMap" -"varSampStableMerge" -"varSampStableNull" -"varSampStableOrDefault" -"varSampStableOrNull" -"varSampStableResample" -"varSampStableSimpleState" -"varSampStableState" -"varSampState" -"variantElement" -"variantType" -"vectorDifference" -"vectorSum" -"version" -"visibleWidth" -"visitParamExtractBool" -"visitParamExtractFloat" -"visitParamExtractInt" -"visitParamExtractRaw" -"visitParamExtractString" -"visitParamExtractUInt" -"visitParamHas" -"week" -"welchTTest" -"welchTTestArgMax" -"welchTTestArgMin" -"welchTTestArray" -"welchTTestDistinct" -"welchTTestForEach" -"welchTTestIf" -"welchTTestMap" -"welchTTestMerge" -"welchTTestNull" -"welchTTestOrDefault" -"welchTTestOrNull" -"welchTTestResample" -"welchTTestSimpleState" -"welchTTestState" -"widthBucket" -"width_bucket" -"windowFunnel" -"windowFunnelArgMax" -"windowFunnelArgMin" -"windowFunnelArray" -"windowFunnelDistinct" -"windowFunnelForEach" -"windowFunnelIf" -"windowFunnelMap" -"windowFunnelMerge" -"windowFunnelNull" -"windowFunnelOrDefault" -"windowFunnelOrNull" -"windowFunnelResample" -"windowFunnelSimpleState" -"windowFunnelState" -"windowID" -"wkt" -"wordShingleMinHash" -"wordShingleMinHashArg" -"wordShingleMinHashArgCaseInsensitive" -"wordShingleMinHashArgCaseInsensitiveUTF8" -"wordShingleMinHashArgUTF8" -"wordShingleMinHashCaseInsensitive" -"wordShingleMinHashCaseInsensitiveUTF8" -"wordShingleMinHashUTF8" -"wordShingleSimHash" -"wordShingleSimHashCaseInsensitive" -"wordShingleSimHashCaseInsensitiveUTF8" -"wordShingleSimHashUTF8" -"wyHash64" -"xor" -"xxHash32" -"xxHash64" -"xxh3" -"yandexConsistentHash" -"yearweek" -"yesterday" -"zookeeperSessionUptime" diff --git a/tests/fuzz/update_dict.sh b/tests/fuzz/update_dict.sh new file mode 100755 index 00000000000..a83c9167129 --- /dev/null +++ b/tests/fuzz/update_dict.sh @@ -0,0 +1,20 @@ +#!/bin/bash + +set -euo pipefail + +SCRIPT_DIR=$(dirname "$(realpath "$0")") +ROOT_PATH="$(git rev-parse --show-toplevel)" +CLICKHOUSE_BIN="${CLICKHOUSE_BIN:-$ROOT_PATH/build/programs/clickhouse}" +DICTIONARIES_DIR="$SCRIPT_DIR/dictionaries" + +echo "Generating functions dict" +$CLICKHOUSE_BIN local -q "SELECT * FROM (SELECT DISTINCT concat('\"', name, '\"') as res FROM system.functions ORDER BY name UNION ALL SELECT concat('\"', a.name, b.name, '\"') as res FROM system.functions as a CROSS JOIN system.aggregate_function_combinators as b WHERE a.is_aggregate = 1) ORDER BY res" > "$DICTIONARIES_DIR/functions.dict" + +echo "Generating data types dict" +$CLICKHOUSE_BIN local -q "SELECT DISTINCT concat('\"', name, '\"') as res FROM system.data_type_families ORDER BY name" > "$DICTIONARIES_DIR/datatypes.dict" + +echo "Generating keywords dict" +$CLICKHOUSE_BIN local -q "SELECT DISTINCT concat('\"', keyword, '\"') as res FROM system.keywords ORDER BY keyword" > "$DICTIONARIES_DIR/keywords.dict" + +echo "Merging dictionaries into all.dict" +cat "$DICTIONARIES_DIR"/* | LC_ALL=C sort | uniq > "$SCRIPT_DIR/all.dict" \ No newline at end of file From 20e20b97c9bddf3a88b0ba8e3a388fe54496fcb2 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 13 Aug 2024 16:20:27 +0200 Subject: [PATCH 633/644] Fix test storage_join_direct_join --- tests/performance/storage_join_direct_join.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/performance/storage_join_direct_join.xml b/tests/performance/storage_join_direct_join.xml index 867108ac2b7..0e67abb275e 100644 --- a/tests/performance/storage_join_direct_join.xml +++ b/tests/performance/storage_join_direct_join.xml @@ -16,4 +16,7 @@ SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null SETTINGS allow_experimental_analyzer=1 + + DROP TABLE IF EXISTS keys + DROP TABLE IF EXISTS dict From 0593650565d717c773fec265056256de98f86f7f Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 13 Aug 2024 14:30:42 +0000 Subject: [PATCH 634/644] Update dict files Previous commit had the files from the script ran from the root directory, which returned an error. --- tests/fuzz/dictionaries/functions.dict | 4283 ++++++++++++++++++++++++ 1 file changed, 4283 insertions(+) diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index e69de29bb2d..e562595fb67 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -0,0 +1,4283 @@ +"BIT_AND" +"BIT_ANDArgMax" +"BIT_ANDArgMin" +"BIT_ANDArray" +"BIT_ANDDistinct" +"BIT_ANDForEach" +"BIT_ANDIf" +"BIT_ANDMap" +"BIT_ANDMerge" +"BIT_ANDNull" +"BIT_ANDOrDefault" +"BIT_ANDOrNull" +"BIT_ANDResample" +"BIT_ANDSimpleState" +"BIT_ANDState" +"BIT_OR" +"BIT_ORArgMax" +"BIT_ORArgMin" +"BIT_ORArray" +"BIT_ORDistinct" +"BIT_ORForEach" +"BIT_ORIf" +"BIT_ORMap" +"BIT_ORMerge" +"BIT_ORNull" +"BIT_OROrDefault" +"BIT_OROrNull" +"BIT_ORResample" +"BIT_ORSimpleState" +"BIT_ORState" +"BIT_XOR" +"BIT_XORArgMax" +"BIT_XORArgMin" +"BIT_XORArray" +"BIT_XORDistinct" +"BIT_XORForEach" +"BIT_XORIf" +"BIT_XORMap" +"BIT_XORMerge" +"BIT_XORNull" +"BIT_XOROrDefault" +"BIT_XOROrNull" +"BIT_XORResample" +"BIT_XORSimpleState" +"BIT_XORState" +"BLAKE3" +"CAST" +"CHARACTER_LENGTH" +"CHAR_LENGTH" +"COVAR_POP" +"COVAR_POPArgMax" +"COVAR_POPArgMin" +"COVAR_POPArray" +"COVAR_POPDistinct" +"COVAR_POPForEach" +"COVAR_POPIf" +"COVAR_POPMap" +"COVAR_POPMerge" +"COVAR_POPNull" +"COVAR_POPOrDefault" +"COVAR_POPOrNull" +"COVAR_POPResample" +"COVAR_POPSimpleState" +"COVAR_POPState" +"COVAR_SAMP" +"COVAR_SAMPArgMax" +"COVAR_SAMPArgMin" +"COVAR_SAMPArray" +"COVAR_SAMPDistinct" +"COVAR_SAMPForEach" +"COVAR_SAMPIf" +"COVAR_SAMPMap" +"COVAR_SAMPMerge" +"COVAR_SAMPNull" +"COVAR_SAMPOrDefault" +"COVAR_SAMPOrNull" +"COVAR_SAMPResample" +"COVAR_SAMPSimpleState" +"COVAR_SAMPState" +"CRC32" +"CRC32IEEE" +"CRC64" +"DATABASE" +"DATE" +"DATE_DIFF" +"DATE_FORMAT" +"DATE_TRUNC" +"DAY" +"DAYOFMONTH" +"DAYOFWEEK" +"DAYOFYEAR" +"FORMAT_BYTES" +"FQDN" +"FROM_BASE64" +"FROM_DAYS" +"FROM_UNIXTIME" +"HOUR" +"INET6_ATON" +"INET6_NTOA" +"INET_ATON" +"INET_NTOA" +"IPv4CIDRToRange" +"IPv4NumToString" +"IPv4NumToStringClassC" +"IPv4StringToNum" +"IPv4StringToNumOrDefault" +"IPv4StringToNumOrNull" +"IPv4ToIPv6" +"IPv6CIDRToRange" +"IPv6NumToString" +"IPv6StringToNum" +"IPv6StringToNumOrDefault" +"IPv6StringToNumOrNull" +"JSONArrayLength" +"JSONExtract" +"JSONExtractArrayRaw" +"JSONExtractBool" +"JSONExtractFloat" +"JSONExtractInt" +"JSONExtractKeys" +"JSONExtractKeysAndValues" +"JSONExtractKeysAndValuesRaw" +"JSONExtractRaw" +"JSONExtractString" +"JSONExtractUInt" +"JSONHas" +"JSONKey" +"JSONLength" +"JSONMergePatch" +"JSONType" +"JSON_ARRAY_LENGTH" +"JSON_EXISTS" +"JSON_QUERY" +"JSON_VALUE" +"L1Distance" +"L1Norm" +"L1Normalize" +"L2Distance" +"L2Norm" +"L2Normalize" +"L2SquaredDistance" +"L2SquaredNorm" +"LAST_DAY" +"LinfDistance" +"LinfNorm" +"LinfNormalize" +"LpDistance" +"LpNorm" +"LpNormalize" +"MACNumToString" +"MACStringToNum" +"MACStringToOUI" +"MAP_FROM_ARRAYS" +"MD4" +"MD5" +"MILLISECOND" +"MINUTE" +"MONTH" +"OCTET_LENGTH" +"QUARTER" +"REGEXP_EXTRACT" +"REGEXP_MATCHES" +"REGEXP_REPLACE" +"SCHEMA" +"SECOND" +"SHA1" +"SHA224" +"SHA256" +"SHA384" +"SHA512" +"SHA512_256" +"STD" +"STDArgMax" +"STDArgMin" +"STDArray" +"STDDEV_POP" +"STDDEV_POPArgMax" +"STDDEV_POPArgMin" +"STDDEV_POPArray" +"STDDEV_POPDistinct" +"STDDEV_POPForEach" +"STDDEV_POPIf" +"STDDEV_POPMap" +"STDDEV_POPMerge" +"STDDEV_POPNull" +"STDDEV_POPOrDefault" +"STDDEV_POPOrNull" +"STDDEV_POPResample" +"STDDEV_POPSimpleState" +"STDDEV_POPState" +"STDDEV_SAMP" +"STDDEV_SAMPArgMax" +"STDDEV_SAMPArgMin" +"STDDEV_SAMPArray" +"STDDEV_SAMPDistinct" +"STDDEV_SAMPForEach" +"STDDEV_SAMPIf" +"STDDEV_SAMPMap" +"STDDEV_SAMPMerge" +"STDDEV_SAMPNull" +"STDDEV_SAMPOrDefault" +"STDDEV_SAMPOrNull" +"STDDEV_SAMPResample" +"STDDEV_SAMPSimpleState" +"STDDEV_SAMPState" +"STDDistinct" +"STDForEach" +"STDIf" +"STDMap" +"STDMerge" +"STDNull" +"STDOrDefault" +"STDOrNull" +"STDResample" +"STDSimpleState" +"STDState" +"SUBSTRING_INDEX" +"SVG" +"TIMESTAMP_DIFF" +"TO_BASE64" +"TO_DAYS" +"TO_UNIXTIME" +"ULIDStringToDateTime" +"URLHash" +"URLHierarchy" +"URLPathHierarchy" +"UTCTimestamp" +"UTC_timestamp" +"UUIDNumToString" +"UUIDStringToNum" +"UUIDToNum" +"UUIDv7ToDateTime" +"VAR_POP" +"VAR_POPArgMax" +"VAR_POPArgMin" +"VAR_POPArray" +"VAR_POPDistinct" +"VAR_POPForEach" +"VAR_POPIf" +"VAR_POPMap" +"VAR_POPMerge" +"VAR_POPNull" +"VAR_POPOrDefault" +"VAR_POPOrNull" +"VAR_POPResample" +"VAR_POPSimpleState" +"VAR_POPState" +"VAR_SAMP" +"VAR_SAMPArgMax" +"VAR_SAMPArgMin" +"VAR_SAMPArray" +"VAR_SAMPDistinct" +"VAR_SAMPForEach" +"VAR_SAMPIf" +"VAR_SAMPMap" +"VAR_SAMPMerge" +"VAR_SAMPNull" +"VAR_SAMPOrDefault" +"VAR_SAMPOrNull" +"VAR_SAMPResample" +"VAR_SAMPSimpleState" +"VAR_SAMPState" +"YEAR" +"YYYYMMDDToDate" +"YYYYMMDDToDate32" +"YYYYMMDDhhmmssToDateTime" +"YYYYMMDDhhmmssToDateTime64" +"_CAST" +"__actionName" +"__bitBoolMaskAnd" +"__bitBoolMaskOr" +"__bitSwapLastTwo" +"__bitWrapperFunc" +"__getScalar" +"__scalarSubqueryResult" +"abs" +"accurateCast" +"accurateCastOrDefault" +"accurateCastOrNull" +"acos" +"acosh" +"addDate" +"addDays" +"addHours" +"addInterval" +"addMicroseconds" +"addMilliseconds" +"addMinutes" +"addMonths" +"addNanoseconds" +"addQuarters" +"addSeconds" +"addTupleOfIntervals" +"addWeeks" +"addYears" +"addressToLine" +"addressToLineWithInlines" +"addressToSymbol" +"aes_decrypt_mysql" +"aes_encrypt_mysql" +"age" +"aggThrow" +"aggThrowArgMax" +"aggThrowArgMin" +"aggThrowArray" +"aggThrowDistinct" +"aggThrowForEach" +"aggThrowIf" +"aggThrowMap" +"aggThrowMerge" +"aggThrowNull" +"aggThrowOrDefault" +"aggThrowOrNull" +"aggThrowResample" +"aggThrowSimpleState" +"aggThrowState" +"alphaTokens" +"analysisOfVariance" +"analysisOfVarianceArgMax" +"analysisOfVarianceArgMin" +"analysisOfVarianceArray" +"analysisOfVarianceDistinct" +"analysisOfVarianceForEach" +"analysisOfVarianceIf" +"analysisOfVarianceMap" +"analysisOfVarianceMerge" +"analysisOfVarianceNull" +"analysisOfVarianceOrDefault" +"analysisOfVarianceOrNull" +"analysisOfVarianceResample" +"analysisOfVarianceSimpleState" +"analysisOfVarianceState" +"and" +"anova" +"anovaArgMax" +"anovaArgMin" +"anovaArray" +"anovaDistinct" +"anovaForEach" +"anovaIf" +"anovaMap" +"anovaMerge" +"anovaNull" +"anovaOrDefault" +"anovaOrNull" +"anovaResample" +"anovaSimpleState" +"anovaState" +"any" +"anyArgMax" +"anyArgMin" +"anyArray" +"anyDistinct" +"anyForEach" +"anyHeavy" +"anyHeavyArgMax" +"anyHeavyArgMin" +"anyHeavyArray" +"anyHeavyDistinct" +"anyHeavyForEach" +"anyHeavyIf" +"anyHeavyMap" +"anyHeavyMerge" +"anyHeavyNull" +"anyHeavyOrDefault" +"anyHeavyOrNull" +"anyHeavyResample" +"anyHeavySimpleState" +"anyHeavyState" +"anyIf" +"anyLast" +"anyLastArgMax" +"anyLastArgMin" +"anyLastArray" +"anyLastDistinct" +"anyLastForEach" +"anyLastIf" +"anyLastMap" +"anyLastMerge" +"anyLastNull" +"anyLastOrDefault" +"anyLastOrNull" +"anyLastResample" +"anyLastSimpleState" +"anyLastState" +"anyLast_respect_nulls" +"anyLast_respect_nullsArgMax" +"anyLast_respect_nullsArgMin" +"anyLast_respect_nullsArray" +"anyLast_respect_nullsDistinct" +"anyLast_respect_nullsForEach" +"anyLast_respect_nullsIf" +"anyLast_respect_nullsMap" +"anyLast_respect_nullsMerge" +"anyLast_respect_nullsNull" +"anyLast_respect_nullsOrDefault" +"anyLast_respect_nullsOrNull" +"anyLast_respect_nullsResample" +"anyLast_respect_nullsSimpleState" +"anyLast_respect_nullsState" +"anyMap" +"anyMerge" +"anyNull" +"anyOrDefault" +"anyOrNull" +"anyResample" +"anySimpleState" +"anyState" +"any_respect_nulls" +"any_respect_nullsArgMax" +"any_respect_nullsArgMin" +"any_respect_nullsArray" +"any_respect_nullsDistinct" +"any_respect_nullsForEach" +"any_respect_nullsIf" +"any_respect_nullsMap" +"any_respect_nullsMerge" +"any_respect_nullsNull" +"any_respect_nullsOrDefault" +"any_respect_nullsOrNull" +"any_respect_nullsResample" +"any_respect_nullsSimpleState" +"any_respect_nullsState" +"any_value" +"any_valueArgMax" +"any_valueArgMin" +"any_valueArray" +"any_valueDistinct" +"any_valueForEach" +"any_valueIf" +"any_valueMap" +"any_valueMerge" +"any_valueNull" +"any_valueOrDefault" +"any_valueOrNull" +"any_valueResample" +"any_valueSimpleState" +"any_valueState" +"any_value_respect_nulls" +"any_value_respect_nullsArgMax" +"any_value_respect_nullsArgMin" +"any_value_respect_nullsArray" +"any_value_respect_nullsDistinct" +"any_value_respect_nullsForEach" +"any_value_respect_nullsIf" +"any_value_respect_nullsMap" +"any_value_respect_nullsMerge" +"any_value_respect_nullsNull" +"any_value_respect_nullsOrDefault" +"any_value_respect_nullsOrNull" +"any_value_respect_nullsResample" +"any_value_respect_nullsSimpleState" +"any_value_respect_nullsState" +"appendTrailingCharIfAbsent" +"approx_top_count" +"approx_top_countArgMax" +"approx_top_countArgMin" +"approx_top_countArray" +"approx_top_countDistinct" +"approx_top_countForEach" +"approx_top_countIf" +"approx_top_countMap" +"approx_top_countMerge" +"approx_top_countNull" +"approx_top_countOrDefault" +"approx_top_countOrNull" +"approx_top_countResample" +"approx_top_countSimpleState" +"approx_top_countState" +"approx_top_k" +"approx_top_kArgMax" +"approx_top_kArgMin" +"approx_top_kArray" +"approx_top_kDistinct" +"approx_top_kForEach" +"approx_top_kIf" +"approx_top_kMap" +"approx_top_kMerge" +"approx_top_kNull" +"approx_top_kOrDefault" +"approx_top_kOrNull" +"approx_top_kResample" +"approx_top_kSimpleState" +"approx_top_kState" +"approx_top_sum" +"approx_top_sumArgMax" +"approx_top_sumArgMin" +"approx_top_sumArray" +"approx_top_sumDistinct" +"approx_top_sumForEach" +"approx_top_sumIf" +"approx_top_sumMap" +"approx_top_sumMerge" +"approx_top_sumNull" +"approx_top_sumOrDefault" +"approx_top_sumOrNull" +"approx_top_sumResample" +"approx_top_sumSimpleState" +"approx_top_sumState" +"argMax" +"argMaxArgMax" +"argMaxArgMin" +"argMaxArray" +"argMaxDistinct" +"argMaxForEach" +"argMaxIf" +"argMaxMap" +"argMaxMerge" +"argMaxNull" +"argMaxOrDefault" +"argMaxOrNull" +"argMaxResample" +"argMaxSimpleState" +"argMaxState" +"argMin" +"argMinArgMax" +"argMinArgMin" +"argMinArray" +"argMinDistinct" +"argMinForEach" +"argMinIf" +"argMinMap" +"argMinMerge" +"argMinNull" +"argMinOrDefault" +"argMinOrNull" +"argMinResample" +"argMinSimpleState" +"argMinState" +"array" +"arrayAUC" +"arrayAll" +"arrayAvg" +"arrayCompact" +"arrayConcat" +"arrayCount" +"arrayCumSum" +"arrayCumSumNonNegative" +"arrayDifference" +"arrayDistinct" +"arrayDotProduct" +"arrayElement" +"arrayEnumerate" +"arrayEnumerateDense" +"arrayEnumerateDenseRanked" +"arrayEnumerateUniq" +"arrayEnumerateUniqRanked" +"arrayExists" +"arrayFill" +"arrayFilter" +"arrayFirst" +"arrayFirstIndex" +"arrayFirstOrNull" +"arrayFlatten" +"arrayFold" +"arrayIntersect" +"arrayJaccardIndex" +"arrayJoin" +"arrayLast" +"arrayLastIndex" +"arrayLastOrNull" +"arrayMap" +"arrayMax" +"arrayMin" +"arrayPartialReverseSort" +"arrayPartialShuffle" +"arrayPartialSort" +"arrayPopBack" +"arrayPopFront" +"arrayProduct" +"arrayPushBack" +"arrayPushFront" +"arrayRandomSample" +"arrayReduce" +"arrayReduceInRanges" +"arrayResize" +"arrayReverse" +"arrayReverseFill" +"arrayReverseSort" +"arrayReverseSplit" +"arrayRotateLeft" +"arrayRotateRight" +"arrayShiftLeft" +"arrayShiftRight" +"arrayShingles" +"arrayShuffle" +"arraySlice" +"arraySort" +"arraySplit" +"arrayStringConcat" +"arraySum" +"arrayUniq" +"arrayWithConstant" +"arrayZip" +"array_agg" +"array_aggArgMax" +"array_aggArgMin" +"array_aggArray" +"array_aggDistinct" +"array_aggForEach" +"array_aggIf" +"array_aggMap" +"array_aggMerge" +"array_aggNull" +"array_aggOrDefault" +"array_aggOrNull" +"array_aggResample" +"array_aggSimpleState" +"array_aggState" +"array_concat_agg" +"array_concat_aggArgMax" +"array_concat_aggArgMin" +"array_concat_aggArray" +"array_concat_aggDistinct" +"array_concat_aggForEach" +"array_concat_aggIf" +"array_concat_aggMap" +"array_concat_aggMerge" +"array_concat_aggNull" +"array_concat_aggOrDefault" +"array_concat_aggOrNull" +"array_concat_aggResample" +"array_concat_aggSimpleState" +"array_concat_aggState" +"ascii" +"asin" +"asinh" +"assumeNotNull" +"atan" +"atan2" +"atanh" +"avg" +"avgArgMax" +"avgArgMin" +"avgArray" +"avgDistinct" +"avgForEach" +"avgIf" +"avgMap" +"avgMerge" +"avgNull" +"avgOrDefault" +"avgOrNull" +"avgResample" +"avgSimpleState" +"avgState" +"avgWeighted" +"avgWeightedArgMax" +"avgWeightedArgMin" +"avgWeightedArray" +"avgWeightedDistinct" +"avgWeightedForEach" +"avgWeightedIf" +"avgWeightedMap" +"avgWeightedMerge" +"avgWeightedNull" +"avgWeightedOrDefault" +"avgWeightedOrNull" +"avgWeightedResample" +"avgWeightedSimpleState" +"avgWeightedState" +"bar" +"base58Decode" +"base58Encode" +"base64Decode" +"base64Encode" +"base64URLDecode" +"base64URLEncode" +"basename" +"bin" +"bitAnd" +"bitCount" +"bitHammingDistance" +"bitNot" +"bitOr" +"bitPositionsToArray" +"bitRotateLeft" +"bitRotateRight" +"bitShiftLeft" +"bitShiftRight" +"bitSlice" +"bitTest" +"bitTestAll" +"bitTestAny" +"bitXor" +"bitmapAnd" +"bitmapAndCardinality" +"bitmapAndnot" +"bitmapAndnotCardinality" +"bitmapBuild" +"bitmapCardinality" +"bitmapContains" +"bitmapHasAll" +"bitmapHasAny" +"bitmapMax" +"bitmapMin" +"bitmapOr" +"bitmapOrCardinality" +"bitmapSubsetInRange" +"bitmapSubsetLimit" +"bitmapToArray" +"bitmapTransform" +"bitmapXor" +"bitmapXorCardinality" +"bitmaskToArray" +"bitmaskToList" +"blockNumber" +"blockSerializedSize" +"blockSize" +"boundingRatio" +"boundingRatioArgMax" +"boundingRatioArgMin" +"boundingRatioArray" +"boundingRatioDistinct" +"boundingRatioForEach" +"boundingRatioIf" +"boundingRatioMap" +"boundingRatioMerge" +"boundingRatioNull" +"boundingRatioOrDefault" +"boundingRatioOrNull" +"boundingRatioResample" +"boundingRatioSimpleState" +"boundingRatioState" +"buildId" +"byteHammingDistance" +"byteSize" +"byteSlice" +"byteSwap" +"caseWithExpr" +"caseWithExpression" +"caseWithoutExpr" +"caseWithoutExpression" +"catboostEvaluate" +"categoricalInformationValue" +"categoricalInformationValueArgMax" +"categoricalInformationValueArgMin" +"categoricalInformationValueArray" +"categoricalInformationValueDistinct" +"categoricalInformationValueForEach" +"categoricalInformationValueIf" +"categoricalInformationValueMap" +"categoricalInformationValueMerge" +"categoricalInformationValueNull" +"categoricalInformationValueOrDefault" +"categoricalInformationValueOrNull" +"categoricalInformationValueResample" +"categoricalInformationValueSimpleState" +"categoricalInformationValueState" +"cbrt" +"ceil" +"ceiling" +"changeDay" +"changeHour" +"changeMinute" +"changeMonth" +"changeSecond" +"changeYear" +"char" +"cityHash64" +"clamp" +"coalesce" +"concat" +"concatAssumeInjective" +"concatWithSeparator" +"concatWithSeparatorAssumeInjective" +"concat_ws" +"connectionId" +"connection_id" +"contingency" +"contingencyArgMax" +"contingencyArgMin" +"contingencyArray" +"contingencyDistinct" +"contingencyForEach" +"contingencyIf" +"contingencyMap" +"contingencyMerge" +"contingencyNull" +"contingencyOrDefault" +"contingencyOrNull" +"contingencyResample" +"contingencySimpleState" +"contingencyState" +"convertCharset" +"corr" +"corrArgMax" +"corrArgMin" +"corrArray" +"corrDistinct" +"corrForEach" +"corrIf" +"corrMap" +"corrMatrix" +"corrMatrixArgMax" +"corrMatrixArgMin" +"corrMatrixArray" +"corrMatrixDistinct" +"corrMatrixForEach" +"corrMatrixIf" +"corrMatrixMap" +"corrMatrixMerge" +"corrMatrixNull" +"corrMatrixOrDefault" +"corrMatrixOrNull" +"corrMatrixResample" +"corrMatrixSimpleState" +"corrMatrixState" +"corrMerge" +"corrNull" +"corrOrDefault" +"corrOrNull" +"corrResample" +"corrSimpleState" +"corrStable" +"corrStableArgMax" +"corrStableArgMin" +"corrStableArray" +"corrStableDistinct" +"corrStableForEach" +"corrStableIf" +"corrStableMap" +"corrStableMerge" +"corrStableNull" +"corrStableOrDefault" +"corrStableOrNull" +"corrStableResample" +"corrStableSimpleState" +"corrStableState" +"corrState" +"cos" +"cosh" +"cosineDistance" +"count" +"countArgMax" +"countArgMin" +"countArray" +"countDigits" +"countDistinct" +"countEqual" +"countForEach" +"countIf" +"countMap" +"countMatches" +"countMatchesCaseInsensitive" +"countMerge" +"countNull" +"countOrDefault" +"countOrNull" +"countResample" +"countSimpleState" +"countState" +"countSubstrings" +"countSubstringsCaseInsensitive" +"countSubstringsCaseInsensitiveUTF8" +"covarPop" +"covarPopArgMax" +"covarPopArgMin" +"covarPopArray" +"covarPopDistinct" +"covarPopForEach" +"covarPopIf" +"covarPopMap" +"covarPopMatrix" +"covarPopMatrixArgMax" +"covarPopMatrixArgMin" +"covarPopMatrixArray" +"covarPopMatrixDistinct" +"covarPopMatrixForEach" +"covarPopMatrixIf" +"covarPopMatrixMap" +"covarPopMatrixMerge" +"covarPopMatrixNull" +"covarPopMatrixOrDefault" +"covarPopMatrixOrNull" +"covarPopMatrixResample" +"covarPopMatrixSimpleState" +"covarPopMatrixState" +"covarPopMerge" +"covarPopNull" +"covarPopOrDefault" +"covarPopOrNull" +"covarPopResample" +"covarPopSimpleState" +"covarPopStable" +"covarPopStableArgMax" +"covarPopStableArgMin" +"covarPopStableArray" +"covarPopStableDistinct" +"covarPopStableForEach" +"covarPopStableIf" +"covarPopStableMap" +"covarPopStableMerge" +"covarPopStableNull" +"covarPopStableOrDefault" +"covarPopStableOrNull" +"covarPopStableResample" +"covarPopStableSimpleState" +"covarPopStableState" +"covarPopState" +"covarSamp" +"covarSampArgMax" +"covarSampArgMin" +"covarSampArray" +"covarSampDistinct" +"covarSampForEach" +"covarSampIf" +"covarSampMap" +"covarSampMatrix" +"covarSampMatrixArgMax" +"covarSampMatrixArgMin" +"covarSampMatrixArray" +"covarSampMatrixDistinct" +"covarSampMatrixForEach" +"covarSampMatrixIf" +"covarSampMatrixMap" +"covarSampMatrixMerge" +"covarSampMatrixNull" +"covarSampMatrixOrDefault" +"covarSampMatrixOrNull" +"covarSampMatrixResample" +"covarSampMatrixSimpleState" +"covarSampMatrixState" +"covarSampMerge" +"covarSampNull" +"covarSampOrDefault" +"covarSampOrNull" +"covarSampResample" +"covarSampSimpleState" +"covarSampStable" +"covarSampStableArgMax" +"covarSampStableArgMin" +"covarSampStableArray" +"covarSampStableDistinct" +"covarSampStableForEach" +"covarSampStableIf" +"covarSampStableMap" +"covarSampStableMerge" +"covarSampStableNull" +"covarSampStableOrDefault" +"covarSampStableOrNull" +"covarSampStableResample" +"covarSampStableSimpleState" +"covarSampStableState" +"covarSampState" +"cramersV" +"cramersVArgMax" +"cramersVArgMin" +"cramersVArray" +"cramersVBiasCorrected" +"cramersVBiasCorrectedArgMax" +"cramersVBiasCorrectedArgMin" +"cramersVBiasCorrectedArray" +"cramersVBiasCorrectedDistinct" +"cramersVBiasCorrectedForEach" +"cramersVBiasCorrectedIf" +"cramersVBiasCorrectedMap" +"cramersVBiasCorrectedMerge" +"cramersVBiasCorrectedNull" +"cramersVBiasCorrectedOrDefault" +"cramersVBiasCorrectedOrNull" +"cramersVBiasCorrectedResample" +"cramersVBiasCorrectedSimpleState" +"cramersVBiasCorrectedState" +"cramersVDistinct" +"cramersVForEach" +"cramersVIf" +"cramersVMap" +"cramersVMerge" +"cramersVNull" +"cramersVOrDefault" +"cramersVOrNull" +"cramersVResample" +"cramersVSimpleState" +"cramersVState" +"curdate" +"currentDatabase" +"currentProfiles" +"currentRoles" +"currentSchemas" +"currentUser" +"current_database" +"current_date" +"current_schemas" +"current_timestamp" +"current_user" +"cutFragment" +"cutIPv6" +"cutQueryString" +"cutQueryStringAndFragment" +"cutToFirstSignificantSubdomain" +"cutToFirstSignificantSubdomainCustom" +"cutToFirstSignificantSubdomainCustomRFC" +"cutToFirstSignificantSubdomainCustomWithWWW" +"cutToFirstSignificantSubdomainCustomWithWWWRFC" +"cutToFirstSignificantSubdomainRFC" +"cutToFirstSignificantSubdomainWithWWW" +"cutToFirstSignificantSubdomainWithWWWRFC" +"cutURLParameter" +"cutWWW" +"damerauLevenshteinDistance" +"dateDiff" +"dateName" +"dateTime64ToSnowflake" +"dateTime64ToSnowflakeID" +"dateTimeToSnowflake" +"dateTimeToSnowflakeID" +"dateTrunc" +"date_diff" +"decodeHTMLComponent" +"decodeURLComponent" +"decodeURLFormComponent" +"decodeXMLComponent" +"decrypt" +"defaultProfiles" +"defaultRoles" +"defaultValueOfArgumentType" +"defaultValueOfTypeName" +"degrees" +"deltaSum" +"deltaSumArgMax" +"deltaSumArgMin" +"deltaSumArray" +"deltaSumDistinct" +"deltaSumForEach" +"deltaSumIf" +"deltaSumMap" +"deltaSumMerge" +"deltaSumNull" +"deltaSumOrDefault" +"deltaSumOrNull" +"deltaSumResample" +"deltaSumSimpleState" +"deltaSumState" +"deltaSumTimestamp" +"deltaSumTimestampArgMax" +"deltaSumTimestampArgMin" +"deltaSumTimestampArray" +"deltaSumTimestampDistinct" +"deltaSumTimestampForEach" +"deltaSumTimestampIf" +"deltaSumTimestampMap" +"deltaSumTimestampMerge" +"deltaSumTimestampNull" +"deltaSumTimestampOrDefault" +"deltaSumTimestampOrNull" +"deltaSumTimestampResample" +"deltaSumTimestampSimpleState" +"deltaSumTimestampState" +"demangle" +"denseRank" +"denseRankArgMax" +"denseRankArgMin" +"denseRankArray" +"denseRankDistinct" +"denseRankForEach" +"denseRankIf" +"denseRankMap" +"denseRankMerge" +"denseRankNull" +"denseRankOrDefault" +"denseRankOrNull" +"denseRankResample" +"denseRankSimpleState" +"denseRankState" +"dense_rank" +"dense_rankArgMax" +"dense_rankArgMin" +"dense_rankArray" +"dense_rankDistinct" +"dense_rankForEach" +"dense_rankIf" +"dense_rankMap" +"dense_rankMerge" +"dense_rankNull" +"dense_rankOrDefault" +"dense_rankOrNull" +"dense_rankResample" +"dense_rankSimpleState" +"dense_rankState" +"detectCharset" +"detectLanguage" +"detectLanguageMixed" +"detectLanguageUnknown" +"detectProgrammingLanguage" +"detectTonality" +"dictGet" +"dictGetAll" +"dictGetChildren" +"dictGetDate" +"dictGetDateOrDefault" +"dictGetDateTime" +"dictGetDateTimeOrDefault" +"dictGetDescendants" +"dictGetFloat32" +"dictGetFloat32OrDefault" +"dictGetFloat64" +"dictGetFloat64OrDefault" +"dictGetHierarchy" +"dictGetIPv4" +"dictGetIPv4OrDefault" +"dictGetIPv6" +"dictGetIPv6OrDefault" +"dictGetInt16" +"dictGetInt16OrDefault" +"dictGetInt32" +"dictGetInt32OrDefault" +"dictGetInt64" +"dictGetInt64OrDefault" +"dictGetInt8" +"dictGetInt8OrDefault" +"dictGetOrDefault" +"dictGetOrNull" +"dictGetString" +"dictGetStringOrDefault" +"dictGetUInt16" +"dictGetUInt16OrDefault" +"dictGetUInt32" +"dictGetUInt32OrDefault" +"dictGetUInt64" +"dictGetUInt64OrDefault" +"dictGetUInt8" +"dictGetUInt8OrDefault" +"dictGetUUID" +"dictGetUUIDOrDefault" +"dictHas" +"dictIsIn" +"displayName" +"distanceL1" +"distanceL2" +"distanceL2Squared" +"distanceLinf" +"distanceLp" +"divide" +"divideDecimal" +"domain" +"domainRFC" +"domainWithoutWWW" +"domainWithoutWWWRFC" +"dotProduct" +"dumpColumnStructure" +"dynamicElement" +"dynamicType" +"e" +"editDistance" +"editDistanceUTF8" +"empty" +"emptyArrayDate" +"emptyArrayDateTime" +"emptyArrayFloat32" +"emptyArrayFloat64" +"emptyArrayInt16" +"emptyArrayInt32" +"emptyArrayInt64" +"emptyArrayInt8" +"emptyArrayString" +"emptyArrayToSingle" +"emptyArrayUInt16" +"emptyArrayUInt32" +"emptyArrayUInt64" +"emptyArrayUInt8" +"enabledProfiles" +"enabledRoles" +"encodeURLComponent" +"encodeURLFormComponent" +"encodeXMLComponent" +"encrypt" +"endsWith" +"endsWithUTF8" +"entropy" +"entropyArgMax" +"entropyArgMin" +"entropyArray" +"entropyDistinct" +"entropyForEach" +"entropyIf" +"entropyMap" +"entropyMerge" +"entropyNull" +"entropyOrDefault" +"entropyOrNull" +"entropyResample" +"entropySimpleState" +"entropyState" +"equals" +"erf" +"erfc" +"errorCodeToName" +"evalMLMethod" +"exp" +"exp10" +"exp2" +"exponentialMovingAverage" +"exponentialMovingAverageArgMax" +"exponentialMovingAverageArgMin" +"exponentialMovingAverageArray" +"exponentialMovingAverageDistinct" +"exponentialMovingAverageForEach" +"exponentialMovingAverageIf" +"exponentialMovingAverageMap" +"exponentialMovingAverageMerge" +"exponentialMovingAverageNull" +"exponentialMovingAverageOrDefault" +"exponentialMovingAverageOrNull" +"exponentialMovingAverageResample" +"exponentialMovingAverageSimpleState" +"exponentialMovingAverageState" +"exponentialTimeDecayedAvg" +"exponentialTimeDecayedAvgArgMax" +"exponentialTimeDecayedAvgArgMin" +"exponentialTimeDecayedAvgArray" +"exponentialTimeDecayedAvgDistinct" +"exponentialTimeDecayedAvgForEach" +"exponentialTimeDecayedAvgIf" +"exponentialTimeDecayedAvgMap" +"exponentialTimeDecayedAvgMerge" +"exponentialTimeDecayedAvgNull" +"exponentialTimeDecayedAvgOrDefault" +"exponentialTimeDecayedAvgOrNull" +"exponentialTimeDecayedAvgResample" +"exponentialTimeDecayedAvgSimpleState" +"exponentialTimeDecayedAvgState" +"exponentialTimeDecayedCount" +"exponentialTimeDecayedCountArgMax" +"exponentialTimeDecayedCountArgMin" +"exponentialTimeDecayedCountArray" +"exponentialTimeDecayedCountDistinct" +"exponentialTimeDecayedCountForEach" +"exponentialTimeDecayedCountIf" +"exponentialTimeDecayedCountMap" +"exponentialTimeDecayedCountMerge" +"exponentialTimeDecayedCountNull" +"exponentialTimeDecayedCountOrDefault" +"exponentialTimeDecayedCountOrNull" +"exponentialTimeDecayedCountResample" +"exponentialTimeDecayedCountSimpleState" +"exponentialTimeDecayedCountState" +"exponentialTimeDecayedMax" +"exponentialTimeDecayedMaxArgMax" +"exponentialTimeDecayedMaxArgMin" +"exponentialTimeDecayedMaxArray" +"exponentialTimeDecayedMaxDistinct" +"exponentialTimeDecayedMaxForEach" +"exponentialTimeDecayedMaxIf" +"exponentialTimeDecayedMaxMap" +"exponentialTimeDecayedMaxMerge" +"exponentialTimeDecayedMaxNull" +"exponentialTimeDecayedMaxOrDefault" +"exponentialTimeDecayedMaxOrNull" +"exponentialTimeDecayedMaxResample" +"exponentialTimeDecayedMaxSimpleState" +"exponentialTimeDecayedMaxState" +"exponentialTimeDecayedSum" +"exponentialTimeDecayedSumArgMax" +"exponentialTimeDecayedSumArgMin" +"exponentialTimeDecayedSumArray" +"exponentialTimeDecayedSumDistinct" +"exponentialTimeDecayedSumForEach" +"exponentialTimeDecayedSumIf" +"exponentialTimeDecayedSumMap" +"exponentialTimeDecayedSumMerge" +"exponentialTimeDecayedSumNull" +"exponentialTimeDecayedSumOrDefault" +"exponentialTimeDecayedSumOrNull" +"exponentialTimeDecayedSumResample" +"exponentialTimeDecayedSumSimpleState" +"exponentialTimeDecayedSumState" +"extract" +"extractAll" +"extractAllGroups" +"extractAllGroupsHorizontal" +"extractAllGroupsVertical" +"extractGroups" +"extractKeyValuePairs" +"extractKeyValuePairsWithEscaping" +"extractTextFromHTML" +"extractURLParameter" +"extractURLParameterNames" +"extractURLParameters" +"factorial" +"farmFingerprint64" +"farmHash64" +"file" +"filesystemAvailable" +"filesystemCapacity" +"filesystemUnreserved" +"finalizeAggregation" +"firstLine" +"firstSignificantSubdomain" +"firstSignificantSubdomainCustom" +"firstSignificantSubdomainCustomRFC" +"firstSignificantSubdomainRFC" +"first_value" +"first_valueArgMax" +"first_valueArgMin" +"first_valueArray" +"first_valueDistinct" +"first_valueForEach" +"first_valueIf" +"first_valueMap" +"first_valueMerge" +"first_valueNull" +"first_valueOrDefault" +"first_valueOrNull" +"first_valueResample" +"first_valueSimpleState" +"first_valueState" +"first_value_respect_nulls" +"first_value_respect_nullsArgMax" +"first_value_respect_nullsArgMin" +"first_value_respect_nullsArray" +"first_value_respect_nullsDistinct" +"first_value_respect_nullsForEach" +"first_value_respect_nullsIf" +"first_value_respect_nullsMap" +"first_value_respect_nullsMerge" +"first_value_respect_nullsNull" +"first_value_respect_nullsOrDefault" +"first_value_respect_nullsOrNull" +"first_value_respect_nullsResample" +"first_value_respect_nullsSimpleState" +"first_value_respect_nullsState" +"flameGraph" +"flameGraphArgMax" +"flameGraphArgMin" +"flameGraphArray" +"flameGraphDistinct" +"flameGraphForEach" +"flameGraphIf" +"flameGraphMap" +"flameGraphMerge" +"flameGraphNull" +"flameGraphOrDefault" +"flameGraphOrNull" +"flameGraphResample" +"flameGraphSimpleState" +"flameGraphState" +"flatten" +"flattenTuple" +"floor" +"format" +"formatDateTime" +"formatDateTimeInJodaSyntax" +"formatQuery" +"formatQueryOrNull" +"formatQuerySingleLine" +"formatQuerySingleLineOrNull" +"formatReadableDecimalSize" +"formatReadableQuantity" +"formatReadableSize" +"formatReadableTimeDelta" +"formatRow" +"formatRowNoNewline" +"fragment" +"fromDaysSinceYearZero" +"fromDaysSinceYearZero32" +"fromModifiedJulianDay" +"fromModifiedJulianDayOrNull" +"fromUTCTimestamp" +"fromUnixTimestamp" +"fromUnixTimestamp64Micro" +"fromUnixTimestamp64Milli" +"fromUnixTimestamp64Nano" +"fromUnixTimestampInJodaSyntax" +"from_utc_timestamp" +"fullHostName" +"fuzzBits" +"gccMurmurHash" +"gcd" +"generateRandomStructure" +"generateSnowflakeID" +"generateULID" +"generateUUIDv4" +"generateUUIDv7" +"geoDistance" +"geoToH3" +"geoToS2" +"geohashDecode" +"geohashEncode" +"geohashesInBox" +"getClientHTTPHeader" +"getMacro" +"getOSKernelVersion" +"getServerPort" +"getSetting" +"getSizeOfEnumType" +"getSubcolumn" +"getTypeSerializationStreams" +"globalIn" +"globalInIgnoreSet" +"globalNotIn" +"globalNotInIgnoreSet" +"globalNotNullIn" +"globalNotNullInIgnoreSet" +"globalNullIn" +"globalNullInIgnoreSet" +"globalVariable" +"greatCircleAngle" +"greatCircleDistance" +"greater" +"greaterOrEquals" +"greatest" +"groupArray" +"groupArrayArgMax" +"groupArrayArgMin" +"groupArrayArray" +"groupArrayDistinct" +"groupArrayForEach" +"groupArrayIf" +"groupArrayInsertAt" +"groupArrayInsertAtArgMax" +"groupArrayInsertAtArgMin" +"groupArrayInsertAtArray" +"groupArrayInsertAtDistinct" +"groupArrayInsertAtForEach" +"groupArrayInsertAtIf" +"groupArrayInsertAtMap" +"groupArrayInsertAtMerge" +"groupArrayInsertAtNull" +"groupArrayInsertAtOrDefault" +"groupArrayInsertAtOrNull" +"groupArrayInsertAtResample" +"groupArrayInsertAtSimpleState" +"groupArrayInsertAtState" +"groupArrayIntersect" +"groupArrayIntersectArgMax" +"groupArrayIntersectArgMin" +"groupArrayIntersectArray" +"groupArrayIntersectDistinct" +"groupArrayIntersectForEach" +"groupArrayIntersectIf" +"groupArrayIntersectMap" +"groupArrayIntersectMerge" +"groupArrayIntersectNull" +"groupArrayIntersectOrDefault" +"groupArrayIntersectOrNull" +"groupArrayIntersectResample" +"groupArrayIntersectSimpleState" +"groupArrayIntersectState" +"groupArrayLast" +"groupArrayLastArgMax" +"groupArrayLastArgMin" +"groupArrayLastArray" +"groupArrayLastDistinct" +"groupArrayLastForEach" +"groupArrayLastIf" +"groupArrayLastMap" +"groupArrayLastMerge" +"groupArrayLastNull" +"groupArrayLastOrDefault" +"groupArrayLastOrNull" +"groupArrayLastResample" +"groupArrayLastSimpleState" +"groupArrayLastState" +"groupArrayMap" +"groupArrayMerge" +"groupArrayMovingAvg" +"groupArrayMovingAvgArgMax" +"groupArrayMovingAvgArgMin" +"groupArrayMovingAvgArray" +"groupArrayMovingAvgDistinct" +"groupArrayMovingAvgForEach" +"groupArrayMovingAvgIf" +"groupArrayMovingAvgMap" +"groupArrayMovingAvgMerge" +"groupArrayMovingAvgNull" +"groupArrayMovingAvgOrDefault" +"groupArrayMovingAvgOrNull" +"groupArrayMovingAvgResample" +"groupArrayMovingAvgSimpleState" +"groupArrayMovingAvgState" +"groupArrayMovingSum" +"groupArrayMovingSumArgMax" +"groupArrayMovingSumArgMin" +"groupArrayMovingSumArray" +"groupArrayMovingSumDistinct" +"groupArrayMovingSumForEach" +"groupArrayMovingSumIf" +"groupArrayMovingSumMap" +"groupArrayMovingSumMerge" +"groupArrayMovingSumNull" +"groupArrayMovingSumOrDefault" +"groupArrayMovingSumOrNull" +"groupArrayMovingSumResample" +"groupArrayMovingSumSimpleState" +"groupArrayMovingSumState" +"groupArrayNull" +"groupArrayOrDefault" +"groupArrayOrNull" +"groupArrayResample" +"groupArraySample" +"groupArraySampleArgMax" +"groupArraySampleArgMin" +"groupArraySampleArray" +"groupArraySampleDistinct" +"groupArraySampleForEach" +"groupArraySampleIf" +"groupArraySampleMap" +"groupArraySampleMerge" +"groupArraySampleNull" +"groupArraySampleOrDefault" +"groupArraySampleOrNull" +"groupArraySampleResample" +"groupArraySampleSimpleState" +"groupArraySampleState" +"groupArraySimpleState" +"groupArraySorted" +"groupArraySortedArgMax" +"groupArraySortedArgMin" +"groupArraySortedArray" +"groupArraySortedDistinct" +"groupArraySortedForEach" +"groupArraySortedIf" +"groupArraySortedMap" +"groupArraySortedMerge" +"groupArraySortedNull" +"groupArraySortedOrDefault" +"groupArraySortedOrNull" +"groupArraySortedResample" +"groupArraySortedSimpleState" +"groupArraySortedState" +"groupArrayState" +"groupBitAnd" +"groupBitAndArgMax" +"groupBitAndArgMin" +"groupBitAndArray" +"groupBitAndDistinct" +"groupBitAndForEach" +"groupBitAndIf" +"groupBitAndMap" +"groupBitAndMerge" +"groupBitAndNull" +"groupBitAndOrDefault" +"groupBitAndOrNull" +"groupBitAndResample" +"groupBitAndSimpleState" +"groupBitAndState" +"groupBitOr" +"groupBitOrArgMax" +"groupBitOrArgMin" +"groupBitOrArray" +"groupBitOrDistinct" +"groupBitOrForEach" +"groupBitOrIf" +"groupBitOrMap" +"groupBitOrMerge" +"groupBitOrNull" +"groupBitOrOrDefault" +"groupBitOrOrNull" +"groupBitOrResample" +"groupBitOrSimpleState" +"groupBitOrState" +"groupBitXor" +"groupBitXorArgMax" +"groupBitXorArgMin" +"groupBitXorArray" +"groupBitXorDistinct" +"groupBitXorForEach" +"groupBitXorIf" +"groupBitXorMap" +"groupBitXorMerge" +"groupBitXorNull" +"groupBitXorOrDefault" +"groupBitXorOrNull" +"groupBitXorResample" +"groupBitXorSimpleState" +"groupBitXorState" +"groupBitmap" +"groupBitmapAnd" +"groupBitmapAndArgMax" +"groupBitmapAndArgMin" +"groupBitmapAndArray" +"groupBitmapAndDistinct" +"groupBitmapAndForEach" +"groupBitmapAndIf" +"groupBitmapAndMap" +"groupBitmapAndMerge" +"groupBitmapAndNull" +"groupBitmapAndOrDefault" +"groupBitmapAndOrNull" +"groupBitmapAndResample" +"groupBitmapAndSimpleState" +"groupBitmapAndState" +"groupBitmapArgMax" +"groupBitmapArgMin" +"groupBitmapArray" +"groupBitmapDistinct" +"groupBitmapForEach" +"groupBitmapIf" +"groupBitmapMap" +"groupBitmapMerge" +"groupBitmapNull" +"groupBitmapOr" +"groupBitmapOrArgMax" +"groupBitmapOrArgMin" +"groupBitmapOrArray" +"groupBitmapOrDefault" +"groupBitmapOrDistinct" +"groupBitmapOrForEach" +"groupBitmapOrIf" +"groupBitmapOrMap" +"groupBitmapOrMerge" +"groupBitmapOrNull" +"groupBitmapOrNull" +"groupBitmapOrOrDefault" +"groupBitmapOrOrNull" +"groupBitmapOrResample" +"groupBitmapOrSimpleState" +"groupBitmapOrState" +"groupBitmapResample" +"groupBitmapSimpleState" +"groupBitmapState" +"groupBitmapXor" +"groupBitmapXorArgMax" +"groupBitmapXorArgMin" +"groupBitmapXorArray" +"groupBitmapXorDistinct" +"groupBitmapXorForEach" +"groupBitmapXorIf" +"groupBitmapXorMap" +"groupBitmapXorMerge" +"groupBitmapXorNull" +"groupBitmapXorOrDefault" +"groupBitmapXorOrNull" +"groupBitmapXorResample" +"groupBitmapXorSimpleState" +"groupBitmapXorState" +"groupConcat" +"groupConcatArgMax" +"groupConcatArgMin" +"groupConcatArray" +"groupConcatDistinct" +"groupConcatForEach" +"groupConcatIf" +"groupConcatMap" +"groupConcatMerge" +"groupConcatNull" +"groupConcatOrDefault" +"groupConcatOrNull" +"groupConcatResample" +"groupConcatSimpleState" +"groupConcatState" +"groupUniqArray" +"groupUniqArrayArgMax" +"groupUniqArrayArgMin" +"groupUniqArrayArray" +"groupUniqArrayDistinct" +"groupUniqArrayForEach" +"groupUniqArrayIf" +"groupUniqArrayMap" +"groupUniqArrayMerge" +"groupUniqArrayNull" +"groupUniqArrayOrDefault" +"groupUniqArrayOrNull" +"groupUniqArrayResample" +"groupUniqArraySimpleState" +"groupUniqArrayState" +"group_concat" +"group_concatArgMax" +"group_concatArgMin" +"group_concatArray" +"group_concatDistinct" +"group_concatForEach" +"group_concatIf" +"group_concatMap" +"group_concatMerge" +"group_concatNull" +"group_concatOrDefault" +"group_concatOrNull" +"group_concatResample" +"group_concatSimpleState" +"group_concatState" +"h3CellAreaM2" +"h3CellAreaRads2" +"h3Distance" +"h3EdgeAngle" +"h3EdgeLengthKm" +"h3EdgeLengthM" +"h3ExactEdgeLengthKm" +"h3ExactEdgeLengthM" +"h3ExactEdgeLengthRads" +"h3GetBaseCell" +"h3GetDestinationIndexFromUnidirectionalEdge" +"h3GetFaces" +"h3GetIndexesFromUnidirectionalEdge" +"h3GetOriginIndexFromUnidirectionalEdge" +"h3GetPentagonIndexes" +"h3GetRes0Indexes" +"h3GetResolution" +"h3GetUnidirectionalEdge" +"h3GetUnidirectionalEdgeBoundary" +"h3GetUnidirectionalEdgesFromHexagon" +"h3HexAreaKm2" +"h3HexAreaM2" +"h3HexRing" +"h3IndexesAreNeighbors" +"h3IsPentagon" +"h3IsResClassIII" +"h3IsValid" +"h3Line" +"h3NumHexagons" +"h3PointDistKm" +"h3PointDistM" +"h3PointDistRads" +"h3ToCenterChild" +"h3ToChildren" +"h3ToGeo" +"h3ToGeoBoundary" +"h3ToParent" +"h3ToString" +"h3UnidirectionalEdgeIsValid" +"h3kRing" +"halfMD5" +"has" +"hasAll" +"hasAny" +"hasColumnInTable" +"hasSubsequence" +"hasSubsequenceCaseInsensitive" +"hasSubsequenceCaseInsensitiveUTF8" +"hasSubsequenceUTF8" +"hasSubstr" +"hasThreadFuzzer" +"hasToken" +"hasTokenCaseInsensitive" +"hasTokenCaseInsensitiveOrNull" +"hasTokenOrNull" +"hex" +"hilbertDecode" +"hilbertEncode" +"histogram" +"histogramArgMax" +"histogramArgMin" +"histogramArray" +"histogramDistinct" +"histogramForEach" +"histogramIf" +"histogramMap" +"histogramMerge" +"histogramNull" +"histogramOrDefault" +"histogramOrNull" +"histogramResample" +"histogramSimpleState" +"histogramState" +"hiveHash" +"hop" +"hopEnd" +"hopStart" +"hostName" +"hostname" +"hypot" +"identity" +"idnaDecode" +"idnaEncode" +"if" +"ifNotFinite" +"ifNull" +"ignore" +"ilike" +"in" +"inIgnoreSet" +"indexHint" +"indexOf" +"initcap" +"initcapUTF8" +"initialQueryID" +"initial_query_id" +"initializeAggregation" +"instr" +"intDiv" +"intDivOrZero" +"intExp10" +"intExp2" +"intHash32" +"intHash64" +"intervalLengthSum" +"intervalLengthSumArgMax" +"intervalLengthSumArgMin" +"intervalLengthSumArray" +"intervalLengthSumDistinct" +"intervalLengthSumForEach" +"intervalLengthSumIf" +"intervalLengthSumMap" +"intervalLengthSumMerge" +"intervalLengthSumNull" +"intervalLengthSumOrDefault" +"intervalLengthSumOrNull" +"intervalLengthSumResample" +"intervalLengthSumSimpleState" +"intervalLengthSumState" +"isConstant" +"isDecimalOverflow" +"isFinite" +"isIPAddressInRange" +"isIPv4String" +"isIPv6String" +"isInfinite" +"isNaN" +"isNotDistinctFrom" +"isNotNull" +"isNull" +"isNullable" +"isValidJSON" +"isValidUTF8" +"isZeroOrNull" +"jaroSimilarity" +"jaroWinklerSimilarity" +"javaHash" +"javaHashUTF16LE" +"joinGet" +"joinGetOrNull" +"jsonMergePatch" +"jumpConsistentHash" +"kafkaMurmurHash" +"kolmogorovSmirnovTest" +"kolmogorovSmirnovTestArgMax" +"kolmogorovSmirnovTestArgMin" +"kolmogorovSmirnovTestArray" +"kolmogorovSmirnovTestDistinct" +"kolmogorovSmirnovTestForEach" +"kolmogorovSmirnovTestIf" +"kolmogorovSmirnovTestMap" +"kolmogorovSmirnovTestMerge" +"kolmogorovSmirnovTestNull" +"kolmogorovSmirnovTestOrDefault" +"kolmogorovSmirnovTestOrNull" +"kolmogorovSmirnovTestResample" +"kolmogorovSmirnovTestSimpleState" +"kolmogorovSmirnovTestState" +"kostikConsistentHash" +"kql_array_sort_asc" +"kql_array_sort_desc" +"kurtPop" +"kurtPopArgMax" +"kurtPopArgMin" +"kurtPopArray" +"kurtPopDistinct" +"kurtPopForEach" +"kurtPopIf" +"kurtPopMap" +"kurtPopMerge" +"kurtPopNull" +"kurtPopOrDefault" +"kurtPopOrNull" +"kurtPopResample" +"kurtPopSimpleState" +"kurtPopState" +"kurtSamp" +"kurtSampArgMax" +"kurtSampArgMin" +"kurtSampArray" +"kurtSampDistinct" +"kurtSampForEach" +"kurtSampIf" +"kurtSampMap" +"kurtSampMerge" +"kurtSampNull" +"kurtSampOrDefault" +"kurtSampOrNull" +"kurtSampResample" +"kurtSampSimpleState" +"kurtSampState" +"lagInFrame" +"lagInFrameArgMax" +"lagInFrameArgMin" +"lagInFrameArray" +"lagInFrameDistinct" +"lagInFrameForEach" +"lagInFrameIf" +"lagInFrameMap" +"lagInFrameMerge" +"lagInFrameNull" +"lagInFrameOrDefault" +"lagInFrameOrNull" +"lagInFrameResample" +"lagInFrameSimpleState" +"lagInFrameState" +"largestTriangleThreeBuckets" +"largestTriangleThreeBucketsArgMax" +"largestTriangleThreeBucketsArgMin" +"largestTriangleThreeBucketsArray" +"largestTriangleThreeBucketsDistinct" +"largestTriangleThreeBucketsForEach" +"largestTriangleThreeBucketsIf" +"largestTriangleThreeBucketsMap" +"largestTriangleThreeBucketsMerge" +"largestTriangleThreeBucketsNull" +"largestTriangleThreeBucketsOrDefault" +"largestTriangleThreeBucketsOrNull" +"largestTriangleThreeBucketsResample" +"largestTriangleThreeBucketsSimpleState" +"largestTriangleThreeBucketsState" +"last_value" +"last_valueArgMax" +"last_valueArgMin" +"last_valueArray" +"last_valueDistinct" +"last_valueForEach" +"last_valueIf" +"last_valueMap" +"last_valueMerge" +"last_valueNull" +"last_valueOrDefault" +"last_valueOrNull" +"last_valueResample" +"last_valueSimpleState" +"last_valueState" +"last_value_respect_nulls" +"last_value_respect_nullsArgMax" +"last_value_respect_nullsArgMin" +"last_value_respect_nullsArray" +"last_value_respect_nullsDistinct" +"last_value_respect_nullsForEach" +"last_value_respect_nullsIf" +"last_value_respect_nullsMap" +"last_value_respect_nullsMerge" +"last_value_respect_nullsNull" +"last_value_respect_nullsOrDefault" +"last_value_respect_nullsOrNull" +"last_value_respect_nullsResample" +"last_value_respect_nullsSimpleState" +"last_value_respect_nullsState" +"lcase" +"lcm" +"leadInFrame" +"leadInFrameArgMax" +"leadInFrameArgMin" +"leadInFrameArray" +"leadInFrameDistinct" +"leadInFrameForEach" +"leadInFrameIf" +"leadInFrameMap" +"leadInFrameMerge" +"leadInFrameNull" +"leadInFrameOrDefault" +"leadInFrameOrNull" +"leadInFrameResample" +"leadInFrameSimpleState" +"leadInFrameState" +"least" +"left" +"leftPad" +"leftPadUTF8" +"leftUTF8" +"lemmatize" +"length" +"lengthUTF8" +"less" +"lessOrEquals" +"levenshteinDistance" +"levenshteinDistanceUTF8" +"lgamma" +"like" +"ln" +"locate" +"log" +"log10" +"log1p" +"log2" +"logTrace" +"lowCardinalityIndices" +"lowCardinalityKeys" +"lower" +"lowerUTF8" +"lpad" +"ltrim" +"lttb" +"lttbArgMax" +"lttbArgMin" +"lttbArray" +"lttbDistinct" +"lttbForEach" +"lttbIf" +"lttbMap" +"lttbMerge" +"lttbNull" +"lttbOrDefault" +"lttbOrNull" +"lttbResample" +"lttbSimpleState" +"lttbState" +"makeDate" +"makeDate32" +"makeDateTime" +"makeDateTime64" +"mannWhitneyUTest" +"mannWhitneyUTestArgMax" +"mannWhitneyUTestArgMin" +"mannWhitneyUTestArray" +"mannWhitneyUTestDistinct" +"mannWhitneyUTestForEach" +"mannWhitneyUTestIf" +"mannWhitneyUTestMap" +"mannWhitneyUTestMerge" +"mannWhitneyUTestNull" +"mannWhitneyUTestOrDefault" +"mannWhitneyUTestOrNull" +"mannWhitneyUTestResample" +"mannWhitneyUTestSimpleState" +"mannWhitneyUTestState" +"map" +"mapAdd" +"mapAll" +"mapApply" +"mapConcat" +"mapContains" +"mapContainsKeyLike" +"mapExists" +"mapExtractKeyLike" +"mapFilter" +"mapFromArrays" +"mapFromString" +"mapKeys" +"mapPartialReverseSort" +"mapPartialSort" +"mapPopulateSeries" +"mapReverseSort" +"mapSort" +"mapSubtract" +"mapUpdate" +"mapValues" +"match" +"materialize" +"max" +"max2" +"maxArgMax" +"maxArgMin" +"maxArray" +"maxDistinct" +"maxForEach" +"maxIf" +"maxIntersections" +"maxIntersectionsArgMax" +"maxIntersectionsArgMin" +"maxIntersectionsArray" +"maxIntersectionsDistinct" +"maxIntersectionsForEach" +"maxIntersectionsIf" +"maxIntersectionsMap" +"maxIntersectionsMerge" +"maxIntersectionsNull" +"maxIntersectionsOrDefault" +"maxIntersectionsOrNull" +"maxIntersectionsPosition" +"maxIntersectionsPositionArgMax" +"maxIntersectionsPositionArgMin" +"maxIntersectionsPositionArray" +"maxIntersectionsPositionDistinct" +"maxIntersectionsPositionForEach" +"maxIntersectionsPositionIf" +"maxIntersectionsPositionMap" +"maxIntersectionsPositionMerge" +"maxIntersectionsPositionNull" +"maxIntersectionsPositionOrDefault" +"maxIntersectionsPositionOrNull" +"maxIntersectionsPositionResample" +"maxIntersectionsPositionSimpleState" +"maxIntersectionsPositionState" +"maxIntersectionsResample" +"maxIntersectionsSimpleState" +"maxIntersectionsState" +"maxMap" +"maxMappedArrays" +"maxMappedArraysArgMax" +"maxMappedArraysArgMin" +"maxMappedArraysArray" +"maxMappedArraysDistinct" +"maxMappedArraysForEach" +"maxMappedArraysIf" +"maxMappedArraysMap" +"maxMappedArraysMerge" +"maxMappedArraysNull" +"maxMappedArraysOrDefault" +"maxMappedArraysOrNull" +"maxMappedArraysResample" +"maxMappedArraysSimpleState" +"maxMappedArraysState" +"maxMerge" +"maxNull" +"maxOrDefault" +"maxOrNull" +"maxResample" +"maxSimpleState" +"maxState" +"meanZTest" +"meanZTestArgMax" +"meanZTestArgMin" +"meanZTestArray" +"meanZTestDistinct" +"meanZTestForEach" +"meanZTestIf" +"meanZTestMap" +"meanZTestMerge" +"meanZTestNull" +"meanZTestOrDefault" +"meanZTestOrNull" +"meanZTestResample" +"meanZTestSimpleState" +"meanZTestState" +"median" +"medianArgMax" +"medianArgMin" +"medianArray" +"medianBFloat16" +"medianBFloat16ArgMax" +"medianBFloat16ArgMin" +"medianBFloat16Array" +"medianBFloat16Distinct" +"medianBFloat16ForEach" +"medianBFloat16If" +"medianBFloat16Map" +"medianBFloat16Merge" +"medianBFloat16Null" +"medianBFloat16OrDefault" +"medianBFloat16OrNull" +"medianBFloat16Resample" +"medianBFloat16SimpleState" +"medianBFloat16State" +"medianBFloat16Weighted" +"medianBFloat16WeightedArgMax" +"medianBFloat16WeightedArgMin" +"medianBFloat16WeightedArray" +"medianBFloat16WeightedDistinct" +"medianBFloat16WeightedForEach" +"medianBFloat16WeightedIf" +"medianBFloat16WeightedMap" +"medianBFloat16WeightedMerge" +"medianBFloat16WeightedNull" +"medianBFloat16WeightedOrDefault" +"medianBFloat16WeightedOrNull" +"medianBFloat16WeightedResample" +"medianBFloat16WeightedSimpleState" +"medianBFloat16WeightedState" +"medianDD" +"medianDDArgMax" +"medianDDArgMin" +"medianDDArray" +"medianDDDistinct" +"medianDDForEach" +"medianDDIf" +"medianDDMap" +"medianDDMerge" +"medianDDNull" +"medianDDOrDefault" +"medianDDOrNull" +"medianDDResample" +"medianDDSimpleState" +"medianDDState" +"medianDeterministic" +"medianDeterministicArgMax" +"medianDeterministicArgMin" +"medianDeterministicArray" +"medianDeterministicDistinct" +"medianDeterministicForEach" +"medianDeterministicIf" +"medianDeterministicMap" +"medianDeterministicMerge" +"medianDeterministicNull" +"medianDeterministicOrDefault" +"medianDeterministicOrNull" +"medianDeterministicResample" +"medianDeterministicSimpleState" +"medianDeterministicState" +"medianDistinct" +"medianExact" +"medianExactArgMax" +"medianExactArgMin" +"medianExactArray" +"medianExactDistinct" +"medianExactForEach" +"medianExactHigh" +"medianExactHighArgMax" +"medianExactHighArgMin" +"medianExactHighArray" +"medianExactHighDistinct" +"medianExactHighForEach" +"medianExactHighIf" +"medianExactHighMap" +"medianExactHighMerge" +"medianExactHighNull" +"medianExactHighOrDefault" +"medianExactHighOrNull" +"medianExactHighResample" +"medianExactHighSimpleState" +"medianExactHighState" +"medianExactIf" +"medianExactLow" +"medianExactLowArgMax" +"medianExactLowArgMin" +"medianExactLowArray" +"medianExactLowDistinct" +"medianExactLowForEach" +"medianExactLowIf" +"medianExactLowMap" +"medianExactLowMerge" +"medianExactLowNull" +"medianExactLowOrDefault" +"medianExactLowOrNull" +"medianExactLowResample" +"medianExactLowSimpleState" +"medianExactLowState" +"medianExactMap" +"medianExactMerge" +"medianExactNull" +"medianExactOrDefault" +"medianExactOrNull" +"medianExactResample" +"medianExactSimpleState" +"medianExactState" +"medianExactWeighted" +"medianExactWeightedArgMax" +"medianExactWeightedArgMin" +"medianExactWeightedArray" +"medianExactWeightedDistinct" +"medianExactWeightedForEach" +"medianExactWeightedIf" +"medianExactWeightedMap" +"medianExactWeightedMerge" +"medianExactWeightedNull" +"medianExactWeightedOrDefault" +"medianExactWeightedOrNull" +"medianExactWeightedResample" +"medianExactWeightedSimpleState" +"medianExactWeightedState" +"medianForEach" +"medianGK" +"medianGKArgMax" +"medianGKArgMin" +"medianGKArray" +"medianGKDistinct" +"medianGKForEach" +"medianGKIf" +"medianGKMap" +"medianGKMerge" +"medianGKNull" +"medianGKOrDefault" +"medianGKOrNull" +"medianGKResample" +"medianGKSimpleState" +"medianGKState" +"medianIf" +"medianInterpolatedWeighted" +"medianInterpolatedWeightedArgMax" +"medianInterpolatedWeightedArgMin" +"medianInterpolatedWeightedArray" +"medianInterpolatedWeightedDistinct" +"medianInterpolatedWeightedForEach" +"medianInterpolatedWeightedIf" +"medianInterpolatedWeightedMap" +"medianInterpolatedWeightedMerge" +"medianInterpolatedWeightedNull" +"medianInterpolatedWeightedOrDefault" +"medianInterpolatedWeightedOrNull" +"medianInterpolatedWeightedResample" +"medianInterpolatedWeightedSimpleState" +"medianInterpolatedWeightedState" +"medianMap" +"medianMerge" +"medianNull" +"medianOrDefault" +"medianOrNull" +"medianResample" +"medianSimpleState" +"medianState" +"medianTDigest" +"medianTDigestArgMax" +"medianTDigestArgMin" +"medianTDigestArray" +"medianTDigestDistinct" +"medianTDigestForEach" +"medianTDigestIf" +"medianTDigestMap" +"medianTDigestMerge" +"medianTDigestNull" +"medianTDigestOrDefault" +"medianTDigestOrNull" +"medianTDigestResample" +"medianTDigestSimpleState" +"medianTDigestState" +"medianTDigestWeighted" +"medianTDigestWeightedArgMax" +"medianTDigestWeightedArgMin" +"medianTDigestWeightedArray" +"medianTDigestWeightedDistinct" +"medianTDigestWeightedForEach" +"medianTDigestWeightedIf" +"medianTDigestWeightedMap" +"medianTDigestWeightedMerge" +"medianTDigestWeightedNull" +"medianTDigestWeightedOrDefault" +"medianTDigestWeightedOrNull" +"medianTDigestWeightedResample" +"medianTDigestWeightedSimpleState" +"medianTDigestWeightedState" +"medianTiming" +"medianTimingArgMax" +"medianTimingArgMin" +"medianTimingArray" +"medianTimingDistinct" +"medianTimingForEach" +"medianTimingIf" +"medianTimingMap" +"medianTimingMerge" +"medianTimingNull" +"medianTimingOrDefault" +"medianTimingOrNull" +"medianTimingResample" +"medianTimingSimpleState" +"medianTimingState" +"medianTimingWeighted" +"medianTimingWeightedArgMax" +"medianTimingWeightedArgMin" +"medianTimingWeightedArray" +"medianTimingWeightedDistinct" +"medianTimingWeightedForEach" +"medianTimingWeightedIf" +"medianTimingWeightedMap" +"medianTimingWeightedMerge" +"medianTimingWeightedNull" +"medianTimingWeightedOrDefault" +"medianTimingWeightedOrNull" +"medianTimingWeightedResample" +"medianTimingWeightedSimpleState" +"medianTimingWeightedState" +"metroHash64" +"mid" +"min" +"min2" +"minArgMax" +"minArgMin" +"minArray" +"minDistinct" +"minForEach" +"minIf" +"minMap" +"minMappedArrays" +"minMappedArraysArgMax" +"minMappedArraysArgMin" +"minMappedArraysArray" +"minMappedArraysDistinct" +"minMappedArraysForEach" +"minMappedArraysIf" +"minMappedArraysMap" +"minMappedArraysMerge" +"minMappedArraysNull" +"minMappedArraysOrDefault" +"minMappedArraysOrNull" +"minMappedArraysResample" +"minMappedArraysSimpleState" +"minMappedArraysState" +"minMerge" +"minNull" +"minOrDefault" +"minOrNull" +"minResample" +"minSampleSizeContinous" +"minSampleSizeContinuous" +"minSampleSizeConversion" +"minSimpleState" +"minState" +"minus" +"mismatches" +"mod" +"modulo" +"moduloLegacy" +"moduloOrZero" +"monthName" +"mortonDecode" +"mortonEncode" +"multiFuzzyMatchAllIndices" +"multiFuzzyMatchAny" +"multiFuzzyMatchAnyIndex" +"multiIf" +"multiMatchAllIndices" +"multiMatchAny" +"multiMatchAnyIndex" +"multiSearchAllPositions" +"multiSearchAllPositionsCaseInsensitive" +"multiSearchAllPositionsCaseInsensitiveUTF8" +"multiSearchAllPositionsUTF8" +"multiSearchAny" +"multiSearchAnyCaseInsensitive" +"multiSearchAnyCaseInsensitiveUTF8" +"multiSearchAnyUTF8" +"multiSearchFirstIndex" +"multiSearchFirstIndexCaseInsensitive" +"multiSearchFirstIndexCaseInsensitiveUTF8" +"multiSearchFirstIndexUTF8" +"multiSearchFirstPosition" +"multiSearchFirstPositionCaseInsensitive" +"multiSearchFirstPositionCaseInsensitiveUTF8" +"multiSearchFirstPositionUTF8" +"multiply" +"multiplyDecimal" +"murmurHash2_32" +"murmurHash2_64" +"murmurHash3_128" +"murmurHash3_32" +"murmurHash3_64" +"negate" +"neighbor" +"nested" +"netloc" +"ngramDistance" +"ngramDistanceCaseInsensitive" +"ngramDistanceCaseInsensitiveUTF8" +"ngramDistanceUTF8" +"ngramMinHash" +"ngramMinHashArg" +"ngramMinHashArgCaseInsensitive" +"ngramMinHashArgCaseInsensitiveUTF8" +"ngramMinHashArgUTF8" +"ngramMinHashCaseInsensitive" +"ngramMinHashCaseInsensitiveUTF8" +"ngramMinHashUTF8" +"ngramSearch" +"ngramSearchCaseInsensitive" +"ngramSearchCaseInsensitiveUTF8" +"ngramSearchUTF8" +"ngramSimHash" +"ngramSimHashCaseInsensitive" +"ngramSimHashCaseInsensitiveUTF8" +"ngramSimHashUTF8" +"ngrams" +"nonNegativeDerivative" +"nonNegativeDerivativeArgMax" +"nonNegativeDerivativeArgMin" +"nonNegativeDerivativeArray" +"nonNegativeDerivativeDistinct" +"nonNegativeDerivativeForEach" +"nonNegativeDerivativeIf" +"nonNegativeDerivativeMap" +"nonNegativeDerivativeMerge" +"nonNegativeDerivativeNull" +"nonNegativeDerivativeOrDefault" +"nonNegativeDerivativeOrNull" +"nonNegativeDerivativeResample" +"nonNegativeDerivativeSimpleState" +"nonNegativeDerivativeState" +"normL1" +"normL2" +"normL2Squared" +"normLinf" +"normLp" +"normalizeL1" +"normalizeL2" +"normalizeLinf" +"normalizeLp" +"normalizeQuery" +"normalizeQueryKeepNames" +"normalizeUTF8NFC" +"normalizeUTF8NFD" +"normalizeUTF8NFKC" +"normalizeUTF8NFKD" +"normalizedQueryHash" +"normalizedQueryHashKeepNames" +"not" +"notEmpty" +"notEquals" +"notILike" +"notIn" +"notInIgnoreSet" +"notLike" +"notNullIn" +"notNullInIgnoreSet" +"nothing" +"nothingArgMax" +"nothingArgMin" +"nothingArray" +"nothingDistinct" +"nothingForEach" +"nothingIf" +"nothingMap" +"nothingMerge" +"nothingNull" +"nothingNull" +"nothingNullArgMax" +"nothingNullArgMin" +"nothingNullArray" +"nothingNullDistinct" +"nothingNullForEach" +"nothingNullIf" +"nothingNullMap" +"nothingNullMerge" +"nothingNullNull" +"nothingNullOrDefault" +"nothingNullOrNull" +"nothingNullResample" +"nothingNullSimpleState" +"nothingNullState" +"nothingOrDefault" +"nothingOrNull" +"nothingResample" +"nothingSimpleState" +"nothingState" +"nothingUInt64" +"nothingUInt64ArgMax" +"nothingUInt64ArgMin" +"nothingUInt64Array" +"nothingUInt64Distinct" +"nothingUInt64ForEach" +"nothingUInt64If" +"nothingUInt64Map" +"nothingUInt64Merge" +"nothingUInt64Null" +"nothingUInt64OrDefault" +"nothingUInt64OrNull" +"nothingUInt64Resample" +"nothingUInt64SimpleState" +"nothingUInt64State" +"now" +"now64" +"nowInBlock" +"nth_value" +"nth_valueArgMax" +"nth_valueArgMin" +"nth_valueArray" +"nth_valueDistinct" +"nth_valueForEach" +"nth_valueIf" +"nth_valueMap" +"nth_valueMerge" +"nth_valueNull" +"nth_valueOrDefault" +"nth_valueOrNull" +"nth_valueResample" +"nth_valueSimpleState" +"nth_valueState" +"ntile" +"ntileArgMax" +"ntileArgMin" +"ntileArray" +"ntileDistinct" +"ntileForEach" +"ntileIf" +"ntileMap" +"ntileMerge" +"ntileNull" +"ntileOrDefault" +"ntileOrNull" +"ntileResample" +"ntileSimpleState" +"ntileState" +"nullIf" +"nullIn" +"nullInIgnoreSet" +"or" +"parseDateTime" +"parseDateTime32BestEffort" +"parseDateTime32BestEffortOrNull" +"parseDateTime32BestEffortOrZero" +"parseDateTime64BestEffort" +"parseDateTime64BestEffortOrNull" +"parseDateTime64BestEffortOrZero" +"parseDateTime64BestEffortUS" +"parseDateTime64BestEffortUSOrNull" +"parseDateTime64BestEffortUSOrZero" +"parseDateTimeBestEffort" +"parseDateTimeBestEffortOrNull" +"parseDateTimeBestEffortOrZero" +"parseDateTimeBestEffortUS" +"parseDateTimeBestEffortUSOrNull" +"parseDateTimeBestEffortUSOrZero" +"parseDateTimeInJodaSyntax" +"parseDateTimeInJodaSyntaxOrNull" +"parseDateTimeInJodaSyntaxOrZero" +"parseDateTimeOrNull" +"parseDateTimeOrZero" +"parseReadableSize" +"parseReadableSizeOrNull" +"parseReadableSizeOrZero" +"parseTimeDelta" +"partitionID" +"partitionId" +"path" +"pathFull" +"percentRank" +"percentRankArgMax" +"percentRankArgMin" +"percentRankArray" +"percentRankDistinct" +"percentRankForEach" +"percentRankIf" +"percentRankMap" +"percentRankMerge" +"percentRankNull" +"percentRankOrDefault" +"percentRankOrNull" +"percentRankResample" +"percentRankSimpleState" +"percentRankState" +"percent_rank" +"percent_rankArgMax" +"percent_rankArgMin" +"percent_rankArray" +"percent_rankDistinct" +"percent_rankForEach" +"percent_rankIf" +"percent_rankMap" +"percent_rankMerge" +"percent_rankNull" +"percent_rankOrDefault" +"percent_rankOrNull" +"percent_rankResample" +"percent_rankSimpleState" +"percent_rankState" +"pi" +"plus" +"pmod" +"pointInEllipses" +"pointInPolygon" +"polygonAreaCartesian" +"polygonAreaSpherical" +"polygonConvexHullCartesian" +"polygonPerimeterCartesian" +"polygonPerimeterSpherical" +"polygonsDistanceCartesian" +"polygonsDistanceSpherical" +"polygonsEqualsCartesian" +"polygonsIntersectionCartesian" +"polygonsIntersectionSpherical" +"polygonsSymDifferenceCartesian" +"polygonsSymDifferenceSpherical" +"polygonsUnionCartesian" +"polygonsUnionSpherical" +"polygonsWithinCartesian" +"polygonsWithinSpherical" +"port" +"portRFC" +"position" +"positionCaseInsensitive" +"positionCaseInsensitiveUTF8" +"positionUTF8" +"positiveModulo" +"positive_modulo" +"pow" +"power" +"printf" +"proportionsZTest" +"protocol" +"punycodeDecode" +"punycodeEncode" +"quantile" +"quantileArgMax" +"quantileArgMin" +"quantileArray" +"quantileBFloat16" +"quantileBFloat16ArgMax" +"quantileBFloat16ArgMin" +"quantileBFloat16Array" +"quantileBFloat16Distinct" +"quantileBFloat16ForEach" +"quantileBFloat16If" +"quantileBFloat16Map" +"quantileBFloat16Merge" +"quantileBFloat16Null" +"quantileBFloat16OrDefault" +"quantileBFloat16OrNull" +"quantileBFloat16Resample" +"quantileBFloat16SimpleState" +"quantileBFloat16State" +"quantileBFloat16Weighted" +"quantileBFloat16WeightedArgMax" +"quantileBFloat16WeightedArgMin" +"quantileBFloat16WeightedArray" +"quantileBFloat16WeightedDistinct" +"quantileBFloat16WeightedForEach" +"quantileBFloat16WeightedIf" +"quantileBFloat16WeightedMap" +"quantileBFloat16WeightedMerge" +"quantileBFloat16WeightedNull" +"quantileBFloat16WeightedOrDefault" +"quantileBFloat16WeightedOrNull" +"quantileBFloat16WeightedResample" +"quantileBFloat16WeightedSimpleState" +"quantileBFloat16WeightedState" +"quantileDD" +"quantileDDArgMax" +"quantileDDArgMin" +"quantileDDArray" +"quantileDDDistinct" +"quantileDDForEach" +"quantileDDIf" +"quantileDDMap" +"quantileDDMerge" +"quantileDDNull" +"quantileDDOrDefault" +"quantileDDOrNull" +"quantileDDResample" +"quantileDDSimpleState" +"quantileDDState" +"quantileDeterministic" +"quantileDeterministicArgMax" +"quantileDeterministicArgMin" +"quantileDeterministicArray" +"quantileDeterministicDistinct" +"quantileDeterministicForEach" +"quantileDeterministicIf" +"quantileDeterministicMap" +"quantileDeterministicMerge" +"quantileDeterministicNull" +"quantileDeterministicOrDefault" +"quantileDeterministicOrNull" +"quantileDeterministicResample" +"quantileDeterministicSimpleState" +"quantileDeterministicState" +"quantileDistinct" +"quantileExact" +"quantileExactArgMax" +"quantileExactArgMin" +"quantileExactArray" +"quantileExactDistinct" +"quantileExactExclusive" +"quantileExactExclusiveArgMax" +"quantileExactExclusiveArgMin" +"quantileExactExclusiveArray" +"quantileExactExclusiveDistinct" +"quantileExactExclusiveForEach" +"quantileExactExclusiveIf" +"quantileExactExclusiveMap" +"quantileExactExclusiveMerge" +"quantileExactExclusiveNull" +"quantileExactExclusiveOrDefault" +"quantileExactExclusiveOrNull" +"quantileExactExclusiveResample" +"quantileExactExclusiveSimpleState" +"quantileExactExclusiveState" +"quantileExactForEach" +"quantileExactHigh" +"quantileExactHighArgMax" +"quantileExactHighArgMin" +"quantileExactHighArray" +"quantileExactHighDistinct" +"quantileExactHighForEach" +"quantileExactHighIf" +"quantileExactHighMap" +"quantileExactHighMerge" +"quantileExactHighNull" +"quantileExactHighOrDefault" +"quantileExactHighOrNull" +"quantileExactHighResample" +"quantileExactHighSimpleState" +"quantileExactHighState" +"quantileExactIf" +"quantileExactInclusive" +"quantileExactInclusiveArgMax" +"quantileExactInclusiveArgMin" +"quantileExactInclusiveArray" +"quantileExactInclusiveDistinct" +"quantileExactInclusiveForEach" +"quantileExactInclusiveIf" +"quantileExactInclusiveMap" +"quantileExactInclusiveMerge" +"quantileExactInclusiveNull" +"quantileExactInclusiveOrDefault" +"quantileExactInclusiveOrNull" +"quantileExactInclusiveResample" +"quantileExactInclusiveSimpleState" +"quantileExactInclusiveState" +"quantileExactLow" +"quantileExactLowArgMax" +"quantileExactLowArgMin" +"quantileExactLowArray" +"quantileExactLowDistinct" +"quantileExactLowForEach" +"quantileExactLowIf" +"quantileExactLowMap" +"quantileExactLowMerge" +"quantileExactLowNull" +"quantileExactLowOrDefault" +"quantileExactLowOrNull" +"quantileExactLowResample" +"quantileExactLowSimpleState" +"quantileExactLowState" +"quantileExactMap" +"quantileExactMerge" +"quantileExactNull" +"quantileExactOrDefault" +"quantileExactOrNull" +"quantileExactResample" +"quantileExactSimpleState" +"quantileExactState" +"quantileExactWeighted" +"quantileExactWeightedArgMax" +"quantileExactWeightedArgMin" +"quantileExactWeightedArray" +"quantileExactWeightedDistinct" +"quantileExactWeightedForEach" +"quantileExactWeightedIf" +"quantileExactWeightedMap" +"quantileExactWeightedMerge" +"quantileExactWeightedNull" +"quantileExactWeightedOrDefault" +"quantileExactWeightedOrNull" +"quantileExactWeightedResample" +"quantileExactWeightedSimpleState" +"quantileExactWeightedState" +"quantileForEach" +"quantileGK" +"quantileGKArgMax" +"quantileGKArgMin" +"quantileGKArray" +"quantileGKDistinct" +"quantileGKForEach" +"quantileGKIf" +"quantileGKMap" +"quantileGKMerge" +"quantileGKNull" +"quantileGKOrDefault" +"quantileGKOrNull" +"quantileGKResample" +"quantileGKSimpleState" +"quantileGKState" +"quantileIf" +"quantileInterpolatedWeighted" +"quantileInterpolatedWeightedArgMax" +"quantileInterpolatedWeightedArgMin" +"quantileInterpolatedWeightedArray" +"quantileInterpolatedWeightedDistinct" +"quantileInterpolatedWeightedForEach" +"quantileInterpolatedWeightedIf" +"quantileInterpolatedWeightedMap" +"quantileInterpolatedWeightedMerge" +"quantileInterpolatedWeightedNull" +"quantileInterpolatedWeightedOrDefault" +"quantileInterpolatedWeightedOrNull" +"quantileInterpolatedWeightedResample" +"quantileInterpolatedWeightedSimpleState" +"quantileInterpolatedWeightedState" +"quantileMap" +"quantileMerge" +"quantileNull" +"quantileOrDefault" +"quantileOrNull" +"quantileResample" +"quantileSimpleState" +"quantileState" +"quantileTDigest" +"quantileTDigestArgMax" +"quantileTDigestArgMin" +"quantileTDigestArray" +"quantileTDigestDistinct" +"quantileTDigestForEach" +"quantileTDigestIf" +"quantileTDigestMap" +"quantileTDigestMerge" +"quantileTDigestNull" +"quantileTDigestOrDefault" +"quantileTDigestOrNull" +"quantileTDigestResample" +"quantileTDigestSimpleState" +"quantileTDigestState" +"quantileTDigestWeighted" +"quantileTDigestWeightedArgMax" +"quantileTDigestWeightedArgMin" +"quantileTDigestWeightedArray" +"quantileTDigestWeightedDistinct" +"quantileTDigestWeightedForEach" +"quantileTDigestWeightedIf" +"quantileTDigestWeightedMap" +"quantileTDigestWeightedMerge" +"quantileTDigestWeightedNull" +"quantileTDigestWeightedOrDefault" +"quantileTDigestWeightedOrNull" +"quantileTDigestWeightedResample" +"quantileTDigestWeightedSimpleState" +"quantileTDigestWeightedState" +"quantileTiming" +"quantileTimingArgMax" +"quantileTimingArgMin" +"quantileTimingArray" +"quantileTimingDistinct" +"quantileTimingForEach" +"quantileTimingIf" +"quantileTimingMap" +"quantileTimingMerge" +"quantileTimingNull" +"quantileTimingOrDefault" +"quantileTimingOrNull" +"quantileTimingResample" +"quantileTimingSimpleState" +"quantileTimingState" +"quantileTimingWeighted" +"quantileTimingWeightedArgMax" +"quantileTimingWeightedArgMin" +"quantileTimingWeightedArray" +"quantileTimingWeightedDistinct" +"quantileTimingWeightedForEach" +"quantileTimingWeightedIf" +"quantileTimingWeightedMap" +"quantileTimingWeightedMerge" +"quantileTimingWeightedNull" +"quantileTimingWeightedOrDefault" +"quantileTimingWeightedOrNull" +"quantileTimingWeightedResample" +"quantileTimingWeightedSimpleState" +"quantileTimingWeightedState" +"quantiles" +"quantilesArgMax" +"quantilesArgMin" +"quantilesArray" +"quantilesBFloat16" +"quantilesBFloat16ArgMax" +"quantilesBFloat16ArgMin" +"quantilesBFloat16Array" +"quantilesBFloat16Distinct" +"quantilesBFloat16ForEach" +"quantilesBFloat16If" +"quantilesBFloat16Map" +"quantilesBFloat16Merge" +"quantilesBFloat16Null" +"quantilesBFloat16OrDefault" +"quantilesBFloat16OrNull" +"quantilesBFloat16Resample" +"quantilesBFloat16SimpleState" +"quantilesBFloat16State" +"quantilesBFloat16Weighted" +"quantilesBFloat16WeightedArgMax" +"quantilesBFloat16WeightedArgMin" +"quantilesBFloat16WeightedArray" +"quantilesBFloat16WeightedDistinct" +"quantilesBFloat16WeightedForEach" +"quantilesBFloat16WeightedIf" +"quantilesBFloat16WeightedMap" +"quantilesBFloat16WeightedMerge" +"quantilesBFloat16WeightedNull" +"quantilesBFloat16WeightedOrDefault" +"quantilesBFloat16WeightedOrNull" +"quantilesBFloat16WeightedResample" +"quantilesBFloat16WeightedSimpleState" +"quantilesBFloat16WeightedState" +"quantilesDD" +"quantilesDDArgMax" +"quantilesDDArgMin" +"quantilesDDArray" +"quantilesDDDistinct" +"quantilesDDForEach" +"quantilesDDIf" +"quantilesDDMap" +"quantilesDDMerge" +"quantilesDDNull" +"quantilesDDOrDefault" +"quantilesDDOrNull" +"quantilesDDResample" +"quantilesDDSimpleState" +"quantilesDDState" +"quantilesDeterministic" +"quantilesDeterministicArgMax" +"quantilesDeterministicArgMin" +"quantilesDeterministicArray" +"quantilesDeterministicDistinct" +"quantilesDeterministicForEach" +"quantilesDeterministicIf" +"quantilesDeterministicMap" +"quantilesDeterministicMerge" +"quantilesDeterministicNull" +"quantilesDeterministicOrDefault" +"quantilesDeterministicOrNull" +"quantilesDeterministicResample" +"quantilesDeterministicSimpleState" +"quantilesDeterministicState" +"quantilesDistinct" +"quantilesExact" +"quantilesExactArgMax" +"quantilesExactArgMin" +"quantilesExactArray" +"quantilesExactDistinct" +"quantilesExactExclusive" +"quantilesExactExclusiveArgMax" +"quantilesExactExclusiveArgMin" +"quantilesExactExclusiveArray" +"quantilesExactExclusiveDistinct" +"quantilesExactExclusiveForEach" +"quantilesExactExclusiveIf" +"quantilesExactExclusiveMap" +"quantilesExactExclusiveMerge" +"quantilesExactExclusiveNull" +"quantilesExactExclusiveOrDefault" +"quantilesExactExclusiveOrNull" +"quantilesExactExclusiveResample" +"quantilesExactExclusiveSimpleState" +"quantilesExactExclusiveState" +"quantilesExactForEach" +"quantilesExactHigh" +"quantilesExactHighArgMax" +"quantilesExactHighArgMin" +"quantilesExactHighArray" +"quantilesExactHighDistinct" +"quantilesExactHighForEach" +"quantilesExactHighIf" +"quantilesExactHighMap" +"quantilesExactHighMerge" +"quantilesExactHighNull" +"quantilesExactHighOrDefault" +"quantilesExactHighOrNull" +"quantilesExactHighResample" +"quantilesExactHighSimpleState" +"quantilesExactHighState" +"quantilesExactIf" +"quantilesExactInclusive" +"quantilesExactInclusiveArgMax" +"quantilesExactInclusiveArgMin" +"quantilesExactInclusiveArray" +"quantilesExactInclusiveDistinct" +"quantilesExactInclusiveForEach" +"quantilesExactInclusiveIf" +"quantilesExactInclusiveMap" +"quantilesExactInclusiveMerge" +"quantilesExactInclusiveNull" +"quantilesExactInclusiveOrDefault" +"quantilesExactInclusiveOrNull" +"quantilesExactInclusiveResample" +"quantilesExactInclusiveSimpleState" +"quantilesExactInclusiveState" +"quantilesExactLow" +"quantilesExactLowArgMax" +"quantilesExactLowArgMin" +"quantilesExactLowArray" +"quantilesExactLowDistinct" +"quantilesExactLowForEach" +"quantilesExactLowIf" +"quantilesExactLowMap" +"quantilesExactLowMerge" +"quantilesExactLowNull" +"quantilesExactLowOrDefault" +"quantilesExactLowOrNull" +"quantilesExactLowResample" +"quantilesExactLowSimpleState" +"quantilesExactLowState" +"quantilesExactMap" +"quantilesExactMerge" +"quantilesExactNull" +"quantilesExactOrDefault" +"quantilesExactOrNull" +"quantilesExactResample" +"quantilesExactSimpleState" +"quantilesExactState" +"quantilesExactWeighted" +"quantilesExactWeightedArgMax" +"quantilesExactWeightedArgMin" +"quantilesExactWeightedArray" +"quantilesExactWeightedDistinct" +"quantilesExactWeightedForEach" +"quantilesExactWeightedIf" +"quantilesExactWeightedMap" +"quantilesExactWeightedMerge" +"quantilesExactWeightedNull" +"quantilesExactWeightedOrDefault" +"quantilesExactWeightedOrNull" +"quantilesExactWeightedResample" +"quantilesExactWeightedSimpleState" +"quantilesExactWeightedState" +"quantilesForEach" +"quantilesGK" +"quantilesGKArgMax" +"quantilesGKArgMin" +"quantilesGKArray" +"quantilesGKDistinct" +"quantilesGKForEach" +"quantilesGKIf" +"quantilesGKMap" +"quantilesGKMerge" +"quantilesGKNull" +"quantilesGKOrDefault" +"quantilesGKOrNull" +"quantilesGKResample" +"quantilesGKSimpleState" +"quantilesGKState" +"quantilesIf" +"quantilesInterpolatedWeighted" +"quantilesInterpolatedWeightedArgMax" +"quantilesInterpolatedWeightedArgMin" +"quantilesInterpolatedWeightedArray" +"quantilesInterpolatedWeightedDistinct" +"quantilesInterpolatedWeightedForEach" +"quantilesInterpolatedWeightedIf" +"quantilesInterpolatedWeightedMap" +"quantilesInterpolatedWeightedMerge" +"quantilesInterpolatedWeightedNull" +"quantilesInterpolatedWeightedOrDefault" +"quantilesInterpolatedWeightedOrNull" +"quantilesInterpolatedWeightedResample" +"quantilesInterpolatedWeightedSimpleState" +"quantilesInterpolatedWeightedState" +"quantilesMap" +"quantilesMerge" +"quantilesNull" +"quantilesOrDefault" +"quantilesOrNull" +"quantilesResample" +"quantilesSimpleState" +"quantilesState" +"quantilesTDigest" +"quantilesTDigestArgMax" +"quantilesTDigestArgMin" +"quantilesTDigestArray" +"quantilesTDigestDistinct" +"quantilesTDigestForEach" +"quantilesTDigestIf" +"quantilesTDigestMap" +"quantilesTDigestMerge" +"quantilesTDigestNull" +"quantilesTDigestOrDefault" +"quantilesTDigestOrNull" +"quantilesTDigestResample" +"quantilesTDigestSimpleState" +"quantilesTDigestState" +"quantilesTDigestWeighted" +"quantilesTDigestWeightedArgMax" +"quantilesTDigestWeightedArgMin" +"quantilesTDigestWeightedArray" +"quantilesTDigestWeightedDistinct" +"quantilesTDigestWeightedForEach" +"quantilesTDigestWeightedIf" +"quantilesTDigestWeightedMap" +"quantilesTDigestWeightedMerge" +"quantilesTDigestWeightedNull" +"quantilesTDigestWeightedOrDefault" +"quantilesTDigestWeightedOrNull" +"quantilesTDigestWeightedResample" +"quantilesTDigestWeightedSimpleState" +"quantilesTDigestWeightedState" +"quantilesTiming" +"quantilesTimingArgMax" +"quantilesTimingArgMin" +"quantilesTimingArray" +"quantilesTimingDistinct" +"quantilesTimingForEach" +"quantilesTimingIf" +"quantilesTimingMap" +"quantilesTimingMerge" +"quantilesTimingNull" +"quantilesTimingOrDefault" +"quantilesTimingOrNull" +"quantilesTimingResample" +"quantilesTimingSimpleState" +"quantilesTimingState" +"quantilesTimingWeighted" +"quantilesTimingWeightedArgMax" +"quantilesTimingWeightedArgMin" +"quantilesTimingWeightedArray" +"quantilesTimingWeightedDistinct" +"quantilesTimingWeightedForEach" +"quantilesTimingWeightedIf" +"quantilesTimingWeightedMap" +"quantilesTimingWeightedMerge" +"quantilesTimingWeightedNull" +"quantilesTimingWeightedOrDefault" +"quantilesTimingWeightedOrNull" +"quantilesTimingWeightedResample" +"quantilesTimingWeightedSimpleState" +"quantilesTimingWeightedState" +"queryID" +"queryString" +"queryStringAndFragment" +"query_id" +"radians" +"rand" +"rand32" +"rand64" +"randBernoulli" +"randBinomial" +"randCanonical" +"randChiSquared" +"randConstant" +"randExponential" +"randFisherF" +"randLogNormal" +"randNegativeBinomial" +"randNormal" +"randPoisson" +"randStudentT" +"randUniform" +"randomFixedString" +"randomPrintableASCII" +"randomString" +"randomStringUTF8" +"range" +"rank" +"rankArgMax" +"rankArgMin" +"rankArray" +"rankCorr" +"rankCorrArgMax" +"rankCorrArgMin" +"rankCorrArray" +"rankCorrDistinct" +"rankCorrForEach" +"rankCorrIf" +"rankCorrMap" +"rankCorrMerge" +"rankCorrNull" +"rankCorrOrDefault" +"rankCorrOrNull" +"rankCorrResample" +"rankCorrSimpleState" +"rankCorrState" +"rankDistinct" +"rankForEach" +"rankIf" +"rankMap" +"rankMerge" +"rankNull" +"rankOrDefault" +"rankOrNull" +"rankResample" +"rankSimpleState" +"rankState" +"readWKTLineString" +"readWKTMultiLineString" +"readWKTMultiPolygon" +"readWKTPoint" +"readWKTPolygon" +"readWKTRing" +"regexpExtract" +"regexpQuoteMeta" +"regionHierarchy" +"regionIn" +"regionToArea" +"regionToCity" +"regionToContinent" +"regionToCountry" +"regionToDistrict" +"regionToName" +"regionToPopulation" +"regionToTopContinent" +"reinterpret" +"reinterpretAsDate" +"reinterpretAsDateTime" +"reinterpretAsFixedString" +"reinterpretAsFloat32" +"reinterpretAsFloat64" +"reinterpretAsInt128" +"reinterpretAsInt16" +"reinterpretAsInt256" +"reinterpretAsInt32" +"reinterpretAsInt64" +"reinterpretAsInt8" +"reinterpretAsString" +"reinterpretAsUInt128" +"reinterpretAsUInt16" +"reinterpretAsUInt256" +"reinterpretAsUInt32" +"reinterpretAsUInt64" +"reinterpretAsUInt8" +"reinterpretAsUUID" +"repeat" +"replace" +"replaceAll" +"replaceOne" +"replaceRegexpAll" +"replaceRegexpOne" +"replicate" +"retention" +"retentionArgMax" +"retentionArgMin" +"retentionArray" +"retentionDistinct" +"retentionForEach" +"retentionIf" +"retentionMap" +"retentionMerge" +"retentionNull" +"retentionOrDefault" +"retentionOrNull" +"retentionResample" +"retentionSimpleState" +"retentionState" +"reverse" +"reverseUTF8" +"revision" +"right" +"rightPad" +"rightPadUTF8" +"rightUTF8" +"round" +"roundAge" +"roundBankers" +"roundDown" +"roundDuration" +"roundToExp2" +"rowNumberInAllBlocks" +"rowNumberInBlock" +"row_number" +"row_numberArgMax" +"row_numberArgMin" +"row_numberArray" +"row_numberDistinct" +"row_numberForEach" +"row_numberIf" +"row_numberMap" +"row_numberMerge" +"row_numberNull" +"row_numberOrDefault" +"row_numberOrNull" +"row_numberResample" +"row_numberSimpleState" +"row_numberState" +"rpad" +"rtrim" +"runningAccumulate" +"runningConcurrency" +"runningDifference" +"runningDifferenceStartingWithFirstValue" +"s2CapContains" +"s2CapUnion" +"s2CellsIntersect" +"s2GetNeighbors" +"s2RectAdd" +"s2RectContains" +"s2RectIntersection" +"s2RectUnion" +"s2ToGeo" +"scalarProduct" +"sequenceCount" +"sequenceCountArgMax" +"sequenceCountArgMin" +"sequenceCountArray" +"sequenceCountDistinct" +"sequenceCountForEach" +"sequenceCountIf" +"sequenceCountMap" +"sequenceCountMerge" +"sequenceCountNull" +"sequenceCountOrDefault" +"sequenceCountOrNull" +"sequenceCountResample" +"sequenceCountSimpleState" +"sequenceCountState" +"sequenceMatch" +"sequenceMatchArgMax" +"sequenceMatchArgMin" +"sequenceMatchArray" +"sequenceMatchDistinct" +"sequenceMatchForEach" +"sequenceMatchIf" +"sequenceMatchMap" +"sequenceMatchMerge" +"sequenceMatchNull" +"sequenceMatchOrDefault" +"sequenceMatchOrNull" +"sequenceMatchResample" +"sequenceMatchSimpleState" +"sequenceMatchState" +"sequenceNextNode" +"sequenceNextNodeArgMax" +"sequenceNextNodeArgMin" +"sequenceNextNodeArray" +"sequenceNextNodeDistinct" +"sequenceNextNodeForEach" +"sequenceNextNodeIf" +"sequenceNextNodeMap" +"sequenceNextNodeMerge" +"sequenceNextNodeNull" +"sequenceNextNodeOrDefault" +"sequenceNextNodeOrNull" +"sequenceNextNodeResample" +"sequenceNextNodeSimpleState" +"sequenceNextNodeState" +"seriesDecomposeSTL" +"seriesOutliersDetectTukey" +"seriesPeriodDetectFFT" +"serverTimeZone" +"serverTimezone" +"serverUUID" +"shardCount" +"shardNum" +"showCertificate" +"sigmoid" +"sign" +"simpleJSONExtractBool" +"simpleJSONExtractFloat" +"simpleJSONExtractInt" +"simpleJSONExtractRaw" +"simpleJSONExtractString" +"simpleJSONExtractUInt" +"simpleJSONHas" +"simpleLinearRegression" +"simpleLinearRegressionArgMax" +"simpleLinearRegressionArgMin" +"simpleLinearRegressionArray" +"simpleLinearRegressionDistinct" +"simpleLinearRegressionForEach" +"simpleLinearRegressionIf" +"simpleLinearRegressionMap" +"simpleLinearRegressionMerge" +"simpleLinearRegressionNull" +"simpleLinearRegressionOrDefault" +"simpleLinearRegressionOrNull" +"simpleLinearRegressionResample" +"simpleLinearRegressionSimpleState" +"simpleLinearRegressionState" +"sin" +"singleValueOrNull" +"singleValueOrNullArgMax" +"singleValueOrNullArgMin" +"singleValueOrNullArray" +"singleValueOrNullDistinct" +"singleValueOrNullForEach" +"singleValueOrNullIf" +"singleValueOrNullMap" +"singleValueOrNullMerge" +"singleValueOrNullNull" +"singleValueOrNullOrDefault" +"singleValueOrNullOrNull" +"singleValueOrNullResample" +"singleValueOrNullSimpleState" +"singleValueOrNullState" +"sinh" +"sipHash128" +"sipHash128Keyed" +"sipHash128Reference" +"sipHash128ReferenceKeyed" +"sipHash64" +"sipHash64Keyed" +"skewPop" +"skewPopArgMax" +"skewPopArgMin" +"skewPopArray" +"skewPopDistinct" +"skewPopForEach" +"skewPopIf" +"skewPopMap" +"skewPopMerge" +"skewPopNull" +"skewPopOrDefault" +"skewPopOrNull" +"skewPopResample" +"skewPopSimpleState" +"skewPopState" +"skewSamp" +"skewSampArgMax" +"skewSampArgMin" +"skewSampArray" +"skewSampDistinct" +"skewSampForEach" +"skewSampIf" +"skewSampMap" +"skewSampMerge" +"skewSampNull" +"skewSampOrDefault" +"skewSampOrNull" +"skewSampResample" +"skewSampSimpleState" +"skewSampState" +"sleep" +"sleepEachRow" +"snowflakeIDToDateTime" +"snowflakeIDToDateTime64" +"snowflakeToDateTime" +"snowflakeToDateTime64" +"soundex" +"space" +"sparkBar" +"sparkBarArgMax" +"sparkBarArgMin" +"sparkBarArray" +"sparkBarDistinct" +"sparkBarForEach" +"sparkBarIf" +"sparkBarMap" +"sparkBarMerge" +"sparkBarNull" +"sparkBarOrDefault" +"sparkBarOrNull" +"sparkBarResample" +"sparkBarSimpleState" +"sparkBarState" +"sparkbar" +"sparkbarArgMax" +"sparkbarArgMin" +"sparkbarArray" +"sparkbarDistinct" +"sparkbarForEach" +"sparkbarIf" +"sparkbarMap" +"sparkbarMerge" +"sparkbarNull" +"sparkbarOrDefault" +"sparkbarOrNull" +"sparkbarResample" +"sparkbarSimpleState" +"sparkbarState" +"splitByAlpha" +"splitByChar" +"splitByNonAlpha" +"splitByRegexp" +"splitByString" +"splitByWhitespace" +"sqid" +"sqidDecode" +"sqidEncode" +"sqrt" +"startsWith" +"startsWithUTF8" +"stddevPop" +"stddevPopArgMax" +"stddevPopArgMin" +"stddevPopArray" +"stddevPopDistinct" +"stddevPopForEach" +"stddevPopIf" +"stddevPopMap" +"stddevPopMerge" +"stddevPopNull" +"stddevPopOrDefault" +"stddevPopOrNull" +"stddevPopResample" +"stddevPopSimpleState" +"stddevPopStable" +"stddevPopStableArgMax" +"stddevPopStableArgMin" +"stddevPopStableArray" +"stddevPopStableDistinct" +"stddevPopStableForEach" +"stddevPopStableIf" +"stddevPopStableMap" +"stddevPopStableMerge" +"stddevPopStableNull" +"stddevPopStableOrDefault" +"stddevPopStableOrNull" +"stddevPopStableResample" +"stddevPopStableSimpleState" +"stddevPopStableState" +"stddevPopState" +"stddevSamp" +"stddevSampArgMax" +"stddevSampArgMin" +"stddevSampArray" +"stddevSampDistinct" +"stddevSampForEach" +"stddevSampIf" +"stddevSampMap" +"stddevSampMerge" +"stddevSampNull" +"stddevSampOrDefault" +"stddevSampOrNull" +"stddevSampResample" +"stddevSampSimpleState" +"stddevSampStable" +"stddevSampStableArgMax" +"stddevSampStableArgMin" +"stddevSampStableArray" +"stddevSampStableDistinct" +"stddevSampStableForEach" +"stddevSampStableIf" +"stddevSampStableMap" +"stddevSampStableMerge" +"stddevSampStableNull" +"stddevSampStableOrDefault" +"stddevSampStableOrNull" +"stddevSampStableResample" +"stddevSampStableSimpleState" +"stddevSampStableState" +"stddevSampState" +"stem" +"stochasticLinearRegression" +"stochasticLinearRegressionArgMax" +"stochasticLinearRegressionArgMin" +"stochasticLinearRegressionArray" +"stochasticLinearRegressionDistinct" +"stochasticLinearRegressionForEach" +"stochasticLinearRegressionIf" +"stochasticLinearRegressionMap" +"stochasticLinearRegressionMerge" +"stochasticLinearRegressionNull" +"stochasticLinearRegressionOrDefault" +"stochasticLinearRegressionOrNull" +"stochasticLinearRegressionResample" +"stochasticLinearRegressionSimpleState" +"stochasticLinearRegressionState" +"stochasticLogisticRegression" +"stochasticLogisticRegressionArgMax" +"stochasticLogisticRegressionArgMin" +"stochasticLogisticRegressionArray" +"stochasticLogisticRegressionDistinct" +"stochasticLogisticRegressionForEach" +"stochasticLogisticRegressionIf" +"stochasticLogisticRegressionMap" +"stochasticLogisticRegressionMerge" +"stochasticLogisticRegressionNull" +"stochasticLogisticRegressionOrDefault" +"stochasticLogisticRegressionOrNull" +"stochasticLogisticRegressionResample" +"stochasticLogisticRegressionSimpleState" +"stochasticLogisticRegressionState" +"str_to_date" +"str_to_map" +"stringJaccardIndex" +"stringJaccardIndexUTF8" +"stringToH3" +"structureToCapnProtoSchema" +"structureToProtobufSchema" +"studentTTest" +"studentTTestArgMax" +"studentTTestArgMin" +"studentTTestArray" +"studentTTestDistinct" +"studentTTestForEach" +"studentTTestIf" +"studentTTestMap" +"studentTTestMerge" +"studentTTestNull" +"studentTTestOrDefault" +"studentTTestOrNull" +"studentTTestResample" +"studentTTestSimpleState" +"studentTTestState" +"subBitmap" +"subDate" +"substr" +"substring" +"substringIndex" +"substringIndexUTF8" +"substringUTF8" +"subtractDays" +"subtractHours" +"subtractInterval" +"subtractMicroseconds" +"subtractMilliseconds" +"subtractMinutes" +"subtractMonths" +"subtractNanoseconds" +"subtractQuarters" +"subtractSeconds" +"subtractTupleOfIntervals" +"subtractWeeks" +"subtractYears" +"sum" +"sumArgMax" +"sumArgMin" +"sumArray" +"sumCount" +"sumCountArgMax" +"sumCountArgMin" +"sumCountArray" +"sumCountDistinct" +"sumCountForEach" +"sumCountIf" +"sumCountMap" +"sumCountMerge" +"sumCountNull" +"sumCountOrDefault" +"sumCountOrNull" +"sumCountResample" +"sumCountSimpleState" +"sumCountState" +"sumDistinct" +"sumForEach" +"sumIf" +"sumKahan" +"sumKahanArgMax" +"sumKahanArgMin" +"sumKahanArray" +"sumKahanDistinct" +"sumKahanForEach" +"sumKahanIf" +"sumKahanMap" +"sumKahanMerge" +"sumKahanNull" +"sumKahanOrDefault" +"sumKahanOrNull" +"sumKahanResample" +"sumKahanSimpleState" +"sumKahanState" +"sumMap" +"sumMapFiltered" +"sumMapFilteredArgMax" +"sumMapFilteredArgMin" +"sumMapFilteredArray" +"sumMapFilteredDistinct" +"sumMapFilteredForEach" +"sumMapFilteredIf" +"sumMapFilteredMap" +"sumMapFilteredMerge" +"sumMapFilteredNull" +"sumMapFilteredOrDefault" +"sumMapFilteredOrNull" +"sumMapFilteredResample" +"sumMapFilteredSimpleState" +"sumMapFilteredState" +"sumMapFilteredWithOverflow" +"sumMapFilteredWithOverflowArgMax" +"sumMapFilteredWithOverflowArgMin" +"sumMapFilteredWithOverflowArray" +"sumMapFilteredWithOverflowDistinct" +"sumMapFilteredWithOverflowForEach" +"sumMapFilteredWithOverflowIf" +"sumMapFilteredWithOverflowMap" +"sumMapFilteredWithOverflowMerge" +"sumMapFilteredWithOverflowNull" +"sumMapFilteredWithOverflowOrDefault" +"sumMapFilteredWithOverflowOrNull" +"sumMapFilteredWithOverflowResample" +"sumMapFilteredWithOverflowSimpleState" +"sumMapFilteredWithOverflowState" +"sumMapWithOverflow" +"sumMapWithOverflowArgMax" +"sumMapWithOverflowArgMin" +"sumMapWithOverflowArray" +"sumMapWithOverflowDistinct" +"sumMapWithOverflowForEach" +"sumMapWithOverflowIf" +"sumMapWithOverflowMap" +"sumMapWithOverflowMerge" +"sumMapWithOverflowNull" +"sumMapWithOverflowOrDefault" +"sumMapWithOverflowOrNull" +"sumMapWithOverflowResample" +"sumMapWithOverflowSimpleState" +"sumMapWithOverflowState" +"sumMappedArrays" +"sumMappedArraysArgMax" +"sumMappedArraysArgMin" +"sumMappedArraysArray" +"sumMappedArraysDistinct" +"sumMappedArraysForEach" +"sumMappedArraysIf" +"sumMappedArraysMap" +"sumMappedArraysMerge" +"sumMappedArraysNull" +"sumMappedArraysOrDefault" +"sumMappedArraysOrNull" +"sumMappedArraysResample" +"sumMappedArraysSimpleState" +"sumMappedArraysState" +"sumMerge" +"sumNull" +"sumOrDefault" +"sumOrNull" +"sumResample" +"sumSimpleState" +"sumState" +"sumWithOverflow" +"sumWithOverflowArgMax" +"sumWithOverflowArgMin" +"sumWithOverflowArray" +"sumWithOverflowDistinct" +"sumWithOverflowForEach" +"sumWithOverflowIf" +"sumWithOverflowMap" +"sumWithOverflowMerge" +"sumWithOverflowNull" +"sumWithOverflowOrDefault" +"sumWithOverflowOrNull" +"sumWithOverflowResample" +"sumWithOverflowSimpleState" +"sumWithOverflowState" +"svg" +"synonyms" +"tan" +"tanh" +"tcpPort" +"tgamma" +"theilsU" +"theilsUArgMax" +"theilsUArgMin" +"theilsUArray" +"theilsUDistinct" +"theilsUForEach" +"theilsUIf" +"theilsUMap" +"theilsUMerge" +"theilsUNull" +"theilsUOrDefault" +"theilsUOrNull" +"theilsUResample" +"theilsUSimpleState" +"theilsUState" +"throwIf" +"tid" +"timeDiff" +"timeSlot" +"timeSlots" +"timeZone" +"timeZoneOf" +"timeZoneOffset" +"timestamp" +"timestampDiff" +"timestamp_diff" +"timezone" +"timezoneOf" +"timezoneOffset" +"toBool" +"toColumnTypeName" +"toDate" +"toDate32" +"toDate32OrDefault" +"toDate32OrNull" +"toDate32OrZero" +"toDateOrDefault" +"toDateOrNull" +"toDateOrZero" +"toDateTime" +"toDateTime32" +"toDateTime64" +"toDateTime64OrDefault" +"toDateTime64OrNull" +"toDateTime64OrZero" +"toDateTimeOrDefault" +"toDateTimeOrNull" +"toDateTimeOrZero" +"toDayOfMonth" +"toDayOfWeek" +"toDayOfYear" +"toDaysSinceYearZero" +"toDecimal128" +"toDecimal128OrDefault" +"toDecimal128OrNull" +"toDecimal128OrZero" +"toDecimal256" +"toDecimal256OrDefault" +"toDecimal256OrNull" +"toDecimal256OrZero" +"toDecimal32" +"toDecimal32OrDefault" +"toDecimal32OrNull" +"toDecimal32OrZero" +"toDecimal64" +"toDecimal64OrDefault" +"toDecimal64OrNull" +"toDecimal64OrZero" +"toDecimalString" +"toFixedString" +"toFloat32" +"toFloat32OrDefault" +"toFloat32OrNull" +"toFloat32OrZero" +"toFloat64" +"toFloat64OrDefault" +"toFloat64OrNull" +"toFloat64OrZero" +"toHour" +"toIPv4" +"toIPv4OrDefault" +"toIPv4OrNull" +"toIPv4OrZero" +"toIPv6" +"toIPv6OrDefault" +"toIPv6OrNull" +"toIPv6OrZero" +"toISOWeek" +"toISOYear" +"toInt128" +"toInt128OrDefault" +"toInt128OrNull" +"toInt128OrZero" +"toInt16" +"toInt16OrDefault" +"toInt16OrNull" +"toInt16OrZero" +"toInt256" +"toInt256OrDefault" +"toInt256OrNull" +"toInt256OrZero" +"toInt32" +"toInt32OrDefault" +"toInt32OrNull" +"toInt32OrZero" +"toInt64" +"toInt64OrDefault" +"toInt64OrNull" +"toInt64OrZero" +"toInt8" +"toInt8OrDefault" +"toInt8OrNull" +"toInt8OrZero" +"toIntervalDay" +"toIntervalHour" +"toIntervalMicrosecond" +"toIntervalMillisecond" +"toIntervalMinute" +"toIntervalMonth" +"toIntervalNanosecond" +"toIntervalQuarter" +"toIntervalSecond" +"toIntervalWeek" +"toIntervalYear" +"toJSONString" +"toLastDayOfMonth" +"toLastDayOfWeek" +"toLowCardinality" +"toMillisecond" +"toMinute" +"toModifiedJulianDay" +"toModifiedJulianDayOrNull" +"toMonday" +"toMonth" +"toNullable" +"toQuarter" +"toRelativeDayNum" +"toRelativeHourNum" +"toRelativeMinuteNum" +"toRelativeMonthNum" +"toRelativeQuarterNum" +"toRelativeSecondNum" +"toRelativeWeekNum" +"toRelativeYearNum" +"toSecond" +"toStartOfDay" +"toStartOfFifteenMinutes" +"toStartOfFiveMinute" +"toStartOfFiveMinutes" +"toStartOfHour" +"toStartOfISOYear" +"toStartOfInterval" +"toStartOfMicrosecond" +"toStartOfMillisecond" +"toStartOfMinute" +"toStartOfMonth" +"toStartOfNanosecond" +"toStartOfQuarter" +"toStartOfSecond" +"toStartOfTenMinutes" +"toStartOfWeek" +"toStartOfYear" +"toString" +"toStringCutToZero" +"toTime" +"toTimeZone" +"toTimezone" +"toTypeName" +"toUInt128" +"toUInt128OrDefault" +"toUInt128OrNull" +"toUInt128OrZero" +"toUInt16" +"toUInt16OrDefault" +"toUInt16OrNull" +"toUInt16OrZero" +"toUInt256" +"toUInt256OrDefault" +"toUInt256OrNull" +"toUInt256OrZero" +"toUInt32" +"toUInt32OrDefault" +"toUInt32OrNull" +"toUInt32OrZero" +"toUInt64" +"toUInt64OrDefault" +"toUInt64OrNull" +"toUInt64OrZero" +"toUInt8" +"toUInt8OrDefault" +"toUInt8OrNull" +"toUInt8OrZero" +"toUTCTimestamp" +"toUUID" +"toUUIDOrDefault" +"toUUIDOrNull" +"toUUIDOrZero" +"toUnixTimestamp" +"toUnixTimestamp64Micro" +"toUnixTimestamp64Milli" +"toUnixTimestamp64Nano" +"toValidUTF8" +"toWeek" +"toYYYYMM" +"toYYYYMMDD" +"toYYYYMMDDhhmmss" +"toYear" +"toYearWeek" +"to_utc_timestamp" +"today" +"tokens" +"topK" +"topKArgMax" +"topKArgMin" +"topKArray" +"topKDistinct" +"topKForEach" +"topKIf" +"topKMap" +"topKMerge" +"topKNull" +"topKOrDefault" +"topKOrNull" +"topKResample" +"topKSimpleState" +"topKState" +"topKWeighted" +"topKWeightedArgMax" +"topKWeightedArgMin" +"topKWeightedArray" +"topKWeightedDistinct" +"topKWeightedForEach" +"topKWeightedIf" +"topKWeightedMap" +"topKWeightedMerge" +"topKWeightedNull" +"topKWeightedOrDefault" +"topKWeightedOrNull" +"topKWeightedResample" +"topKWeightedSimpleState" +"topKWeightedState" +"topLevelDomain" +"topLevelDomainRFC" +"transactionID" +"transactionLatestSnapshot" +"transactionOldestSnapshot" +"transform" +"translate" +"translateUTF8" +"trim" +"trimBoth" +"trimLeft" +"trimRight" +"trunc" +"truncate" +"tryBase58Decode" +"tryBase64Decode" +"tryBase64URLDecode" +"tryDecrypt" +"tryIdnaEncode" +"tryPunycodeDecode" +"tumble" +"tumbleEnd" +"tumbleStart" +"tuple" +"tupleConcat" +"tupleDivide" +"tupleDivideByNumber" +"tupleElement" +"tupleHammingDistance" +"tupleIntDiv" +"tupleIntDivByNumber" +"tupleIntDivOrZero" +"tupleIntDivOrZeroByNumber" +"tupleMinus" +"tupleModulo" +"tupleModuloByNumber" +"tupleMultiply" +"tupleMultiplyByNumber" +"tupleNames" +"tupleNegate" +"tuplePlus" +"tupleToNameValuePairs" +"ucase" +"unbin" +"unhex" +"uniq" +"uniqArgMax" +"uniqArgMin" +"uniqArray" +"uniqCombined" +"uniqCombined64" +"uniqCombined64ArgMax" +"uniqCombined64ArgMin" +"uniqCombined64Array" +"uniqCombined64Distinct" +"uniqCombined64ForEach" +"uniqCombined64If" +"uniqCombined64Map" +"uniqCombined64Merge" +"uniqCombined64Null" +"uniqCombined64OrDefault" +"uniqCombined64OrNull" +"uniqCombined64Resample" +"uniqCombined64SimpleState" +"uniqCombined64State" +"uniqCombinedArgMax" +"uniqCombinedArgMin" +"uniqCombinedArray" +"uniqCombinedDistinct" +"uniqCombinedForEach" +"uniqCombinedIf" +"uniqCombinedMap" +"uniqCombinedMerge" +"uniqCombinedNull" +"uniqCombinedOrDefault" +"uniqCombinedOrNull" +"uniqCombinedResample" +"uniqCombinedSimpleState" +"uniqCombinedState" +"uniqDistinct" +"uniqExact" +"uniqExactArgMax" +"uniqExactArgMin" +"uniqExactArray" +"uniqExactDistinct" +"uniqExactForEach" +"uniqExactIf" +"uniqExactMap" +"uniqExactMerge" +"uniqExactNull" +"uniqExactOrDefault" +"uniqExactOrNull" +"uniqExactResample" +"uniqExactSimpleState" +"uniqExactState" +"uniqForEach" +"uniqHLL12" +"uniqHLL12ArgMax" +"uniqHLL12ArgMin" +"uniqHLL12Array" +"uniqHLL12Distinct" +"uniqHLL12ForEach" +"uniqHLL12If" +"uniqHLL12Map" +"uniqHLL12Merge" +"uniqHLL12Null" +"uniqHLL12OrDefault" +"uniqHLL12OrNull" +"uniqHLL12Resample" +"uniqHLL12SimpleState" +"uniqHLL12State" +"uniqIf" +"uniqMap" +"uniqMerge" +"uniqNull" +"uniqOrDefault" +"uniqOrNull" +"uniqResample" +"uniqSimpleState" +"uniqState" +"uniqTheta" +"uniqThetaArgMax" +"uniqThetaArgMin" +"uniqThetaArray" +"uniqThetaDistinct" +"uniqThetaForEach" +"uniqThetaIf" +"uniqThetaIntersect" +"uniqThetaMap" +"uniqThetaMerge" +"uniqThetaNot" +"uniqThetaNull" +"uniqThetaOrDefault" +"uniqThetaOrNull" +"uniqThetaResample" +"uniqThetaSimpleState" +"uniqThetaState" +"uniqThetaUnion" +"uniqUpTo" +"uniqUpToArgMax" +"uniqUpToArgMin" +"uniqUpToArray" +"uniqUpToDistinct" +"uniqUpToForEach" +"uniqUpToIf" +"uniqUpToMap" +"uniqUpToMerge" +"uniqUpToNull" +"uniqUpToOrDefault" +"uniqUpToOrNull" +"uniqUpToResample" +"uniqUpToSimpleState" +"uniqUpToState" +"upper" +"upperUTF8" +"uptime" +"user" +"validateNestedArraySizes" +"varPop" +"varPopArgMax" +"varPopArgMin" +"varPopArray" +"varPopDistinct" +"varPopForEach" +"varPopIf" +"varPopMap" +"varPopMerge" +"varPopNull" +"varPopOrDefault" +"varPopOrNull" +"varPopResample" +"varPopSimpleState" +"varPopStable" +"varPopStableArgMax" +"varPopStableArgMin" +"varPopStableArray" +"varPopStableDistinct" +"varPopStableForEach" +"varPopStableIf" +"varPopStableMap" +"varPopStableMerge" +"varPopStableNull" +"varPopStableOrDefault" +"varPopStableOrNull" +"varPopStableResample" +"varPopStableSimpleState" +"varPopStableState" +"varPopState" +"varSamp" +"varSampArgMax" +"varSampArgMin" +"varSampArray" +"varSampDistinct" +"varSampForEach" +"varSampIf" +"varSampMap" +"varSampMerge" +"varSampNull" +"varSampOrDefault" +"varSampOrNull" +"varSampResample" +"varSampSimpleState" +"varSampStable" +"varSampStableArgMax" +"varSampStableArgMin" +"varSampStableArray" +"varSampStableDistinct" +"varSampStableForEach" +"varSampStableIf" +"varSampStableMap" +"varSampStableMerge" +"varSampStableNull" +"varSampStableOrDefault" +"varSampStableOrNull" +"varSampStableResample" +"varSampStableSimpleState" +"varSampStableState" +"varSampState" +"variantElement" +"variantType" +"vectorDifference" +"vectorSum" +"version" +"visibleWidth" +"visitParamExtractBool" +"visitParamExtractFloat" +"visitParamExtractInt" +"visitParamExtractRaw" +"visitParamExtractString" +"visitParamExtractUInt" +"visitParamHas" +"week" +"welchTTest" +"welchTTestArgMax" +"welchTTestArgMin" +"welchTTestArray" +"welchTTestDistinct" +"welchTTestForEach" +"welchTTestIf" +"welchTTestMap" +"welchTTestMerge" +"welchTTestNull" +"welchTTestOrDefault" +"welchTTestOrNull" +"welchTTestResample" +"welchTTestSimpleState" +"welchTTestState" +"widthBucket" +"width_bucket" +"windowFunnel" +"windowFunnelArgMax" +"windowFunnelArgMin" +"windowFunnelArray" +"windowFunnelDistinct" +"windowFunnelForEach" +"windowFunnelIf" +"windowFunnelMap" +"windowFunnelMerge" +"windowFunnelNull" +"windowFunnelOrDefault" +"windowFunnelOrNull" +"windowFunnelResample" +"windowFunnelSimpleState" +"windowFunnelState" +"windowID" +"wkt" +"wordShingleMinHash" +"wordShingleMinHashArg" +"wordShingleMinHashArgCaseInsensitive" +"wordShingleMinHashArgCaseInsensitiveUTF8" +"wordShingleMinHashArgUTF8" +"wordShingleMinHashCaseInsensitive" +"wordShingleMinHashCaseInsensitiveUTF8" +"wordShingleMinHashUTF8" +"wordShingleSimHash" +"wordShingleSimHashCaseInsensitive" +"wordShingleSimHashCaseInsensitiveUTF8" +"wordShingleSimHashUTF8" +"wyHash64" +"xor" +"xxHash32" +"xxHash64" +"xxh3" +"yandexConsistentHash" +"yearweek" +"yesterday" +"zookeeperSessionUptime" From 5a6090ad05117c76a4b37071a6362f30f395b235 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 13 Aug 2024 16:25:07 +0200 Subject: [PATCH 635/644] Fix --- src/Processors/Sources/PostgreSQLSource.cpp | 12 ++++++------ src/Processors/Sources/PostgreSQLSource.h | 14 +++++++++----- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/src/Processors/Sources/PostgreSQLSource.cpp b/src/Processors/Sources/PostgreSQLSource.cpp index a3d6fd691d8..b9bda46bd10 100644 --- a/src/Processors/Sources/PostgreSQLSource.cpp +++ b/src/Processors/Sources/PostgreSQLSource.cpp @@ -35,9 +35,9 @@ PostgreSQLSource::PostgreSQLSource( const Block & sample_block, UInt64 max_block_size_) : ISource(sample_block.cloneEmpty()) - , query_str(query_str_) , max_block_size(max_block_size_) , connection_holder(std::move(connection_holder_)) + , query_str(query_str_) { init(sample_block); } @@ -51,10 +51,10 @@ PostgreSQLSource::PostgreSQLSource( UInt64 max_block_size_, bool auto_commit_) : ISource(sample_block.cloneEmpty()) - , query_str(query_str_) - , tx(std::move(tx_)) , max_block_size(max_block_size_) , auto_commit(auto_commit_) + , query_str(query_str_) + , tx(std::move(tx_)) { init(sample_block); } @@ -204,15 +204,15 @@ PostgreSQLSource::~PostgreSQLSource() */ stream->close(); } - - stream.reset(); - tx.reset(); } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } + stream.reset(); + tx.reset(); + if (connection_holder) connection_holder->setBroken(); } diff --git a/src/Processors/Sources/PostgreSQLSource.h b/src/Processors/Sources/PostgreSQLSource.h index 8a648ae8bb5..319c5d8d7c2 100644 --- a/src/Processors/Sources/PostgreSQLSource.h +++ b/src/Processors/Sources/PostgreSQLSource.h @@ -38,14 +38,12 @@ protected: UInt64 max_block_size_, bool auto_commit_); - String query_str; - std::shared_ptr tx; - std::unique_ptr stream; - Status prepare() override; - void onStart(); Chunk generate() override; + + void onStart(); + void onFinish(); private: @@ -61,6 +59,12 @@ private: postgres::ConnectionHolderPtr connection_holder; std::unordered_map array_info; + +protected: + String query_str; + /// tx and stream must be destroyed before connection_holder. + std::shared_ptr tx; + std::unique_ptr stream; }; From 7ebb6efb2e413c44a82f2ac25d6dcb8e0da2f59a Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 13 Aug 2024 15:10:56 +0000 Subject: [PATCH 636/644] Style check --- tests/queries/0_stateless/00652_mergetree_mutations.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00652_mergetree_mutations.sh b/tests/queries/0_stateless/00652_mergetree_mutations.sh index 6be0ebf882f..edb306d3883 100755 --- a/tests/queries/0_stateless/00652_mergetree_mutations.sh +++ b/tests/queries/0_stateless/00652_mergetree_mutations.sh @@ -73,7 +73,7 @@ sleep 0.1 for i in {1..10} do - if [ $(${CLICKHOUSE_CLIENT} --query="SELECT count() FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner'") -eq 2 ]; then + if [ "$(${CLICKHOUSE_CLIENT} --query="SELECT count() FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner'")" -eq 2 ]; then break fi From ae614648a3397c4738b85ab8d138419387c562ed Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 13 Aug 2024 15:13:42 +0000 Subject: [PATCH 637/644] trigger sync From 7e209ebdf686e374fdf764cb6acb3e7de83e927f Mon Sep 17 00:00:00 2001 From: Austin Bruch Date: Tue, 13 Aug 2024 12:30:17 -0400 Subject: [PATCH 638/644] Remove trailing colon in header for consistency --- docs/en/sql-reference/table-functions/file.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 7908a3cb934..3243e6cf569 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -103,7 +103,7 @@ LIMIT 2; └─────────┴─────────┴─────────┘ ``` -### Inserting data from a file into a table: +### Inserting data from a file into a table ``` sql INSERT INTO FUNCTION From 6af5fedf420c667e2a7866c89dfe0bd1d2ff37dd Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 13 Aug 2024 19:26:35 +0000 Subject: [PATCH 639/644] Update autogenerated version to 24.9.1.1 and contributors --- cmake/autogenerated_versions.txt | 10 +++++----- .../StorageSystemContributors.generated.cpp | 17 +++++++++++++++++ 2 files changed, 22 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index d69646d3694..c82038804fe 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: VERSION_REVISION has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54489) +SET(VERSION_REVISION 54490) SET(VERSION_MAJOR 24) -SET(VERSION_MINOR 8) +SET(VERSION_MINOR 9) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH 3f8b27d7accd2b5ec4afe7d0dd459115323304af) -SET(VERSION_DESCRIBE v24.8.1.1-testing) -SET(VERSION_STRING 24.8.1.1) +SET(VERSION_GITHASH e02b434d2fc0c4fbee29ca675deab7474d274608) +SET(VERSION_DESCRIBE v24.9.1.1-testing) +SET(VERSION_STRING 24.9.1.1) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index 35b9c0008c6..eb6f0382d15 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -457,6 +457,7 @@ const char * auto_contributors[] { "Gleb-Tretyakov", "GoGoWen2021", "Gosha Letov", + "Graham Campbell", "Gregory", "Grigorii Sokolik", "Grigory", @@ -472,6 +473,7 @@ const char * auto_contributors[] { "Habibullah Oladepo", "HaiBo Li", "Hakob Saghatelyan", + "Halersson Paris", "Hamoon", "Han Fei", "Han Shukai", @@ -541,6 +543,7 @@ const char * auto_contributors[] { "JackyWoo", "Jacob Hayes", "Jacob Herrington", + "Jacob Reckhard", "Jai Jhala", "Jake Bamrah", "Jake Liu", @@ -661,6 +664,7 @@ const char * auto_contributors[] { "LaurieLY", "Lee sungju", "Lemore", + "Lennard Eijsackers", "Leonardo Cecchi", "Leonardo Maciel", "Leonid Krylov", @@ -804,6 +808,7 @@ const char * auto_contributors[] { "Mingliang Pan", "Misko Lee", "Misz606", + "Miсhael Stetsyuk", "MochiXu", "Mohamad Fadhil", "Mohammad Arab Anvari", @@ -922,6 +927,7 @@ const char * auto_contributors[] { "Pervakov Grigorii", "Pervakov Grigory", "Peter", + "Peter Nguyen", "Petr Vasilev", "Pham Anh Tuan", "Philip Hallstrom", @@ -981,6 +987,7 @@ const char * auto_contributors[] { "Ronald Bradford", "Rory Crispin", "Roy Bellingan", + "Ruihang Xia", "Ruslan", "Ruslan Mardugalliamov", "Ruslan Savchenko", @@ -1000,9 +1007,11 @@ const char * auto_contributors[] { "Sami Kerola", "Samuel Chou", "Samuel Colvin", + "Samuele Guerrini", "San", "Sanjam Panda", "Sariel", + "Sasha Sheikin", "Saulius Valatka", "Sean Haynes", "Sean Lafferty", @@ -1202,6 +1211,7 @@ const char * auto_contributors[] { "Vladimir Makarov", "Vladimir Mihailenco", "Vladimir Smirnov", + "Vladimir Varankin", "Vladislav Rassokhin", "Vladislav Smirnov", "Vladislav V", @@ -1275,6 +1285,7 @@ const char * auto_contributors[] { "Zhichun Wu", "Zhiguo Zhou", "Zhipeng", + "Zhukova, Maria", "Zhuo Qiu", "Zijie Lu", "Zimu Li", @@ -1502,6 +1513,7 @@ const char * auto_contributors[] { "hchen9", "hcz", "hdhoang", + "heguangnan", "heleihelei", "helifu", "hendrik-m", @@ -1572,6 +1584,7 @@ const char * auto_contributors[] { "kevinyhzou", "kgurjev", "khamadiev", + "khodyrevyurii", "kigerzhang", "kirillikoff", "kmeaw", @@ -1787,6 +1800,7 @@ const char * auto_contributors[] { "ruslandoga", "ryzuo", "s-kat", + "sakulali", "sanjam", "santaux", "santrancisco", @@ -1804,6 +1818,7 @@ const char * auto_contributors[] { "shabroo", "shangshujie", "shedx", + "shiyer7474", "shuai-xu", "shuchaome", "shuyang", @@ -1901,6 +1916,7 @@ const char * auto_contributors[] { "wzl", "xPoSx", "xbthink", + "xc0derx", "xiao", "xiaolei565", "xiebin", @@ -1964,6 +1980,7 @@ const char * auto_contributors[] { "zkun", "zlx19950903", "zombee0", + "zoomxi", "zvonand", "zvrr", "zvvr", From a9226f49e7e052d2c392214afe32f4d6de1d6d62 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 13 Aug 2024 20:24:40 +0000 Subject: [PATCH 640/644] remove name with cyrillic letter --- src/Storages/System/StorageSystemContributors.generated.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index eb6f0382d15..67dfe3bfe86 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -808,7 +808,6 @@ const char * auto_contributors[] { "Mingliang Pan", "Misko Lee", "Misz606", - "Miсhael Stetsyuk", "MochiXu", "Mohamad Fadhil", "Mohammad Arab Anvari", From 0c9e1a061f825e5b9c5d623d90d4d898cd05e44c Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 13 Aug 2024 18:49:18 +0200 Subject: [PATCH 641/644] CI: Create new release branch workflow updates --- .github/workflows/create_release.yml | 2 +- tests/ci/ci_utils.py | 5 + tests/ci/create_release.py | 240 ++++++++++++++++----------- tests/ci/docker_server.py | 2 +- tests/ci/version_helper.py | 17 +- 5 files changed, 161 insertions(+), 105 deletions(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index d4993b373df..73613c65266 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -129,9 +129,9 @@ jobs: if: ${{ inputs.type == 'patch' && ! inputs.only-repo }} shell: bash run: | - python3 ./tests/ci/create_release.py --set-progress-completed git reset --hard HEAD git checkout "$GITHUB_REF_NAME" + python3 ./tests/ci/create_release.py --set-progress-completed - name: Create GH Release if: ${{ inputs.type == 'patch' && ! inputs.only-repo }} shell: bash diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index d807f5be09f..b8778e0cc50 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -167,6 +167,11 @@ class GH: latest_branch = Shell.get_output( 'gh pr list --label release --repo ClickHouse/ClickHouse --search "sort:created" -L1 --json headRefName' ) + if latest_branch: + latest_branch = json.loads(latest_branch)[0]["headRefName"] + print( + f"Latest branch [{latest_branch}], release branch [{branch}], release latest [{latest_branch == branch}]" + ) return latest_branch == branch diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index 27eba273ce0..b5ea61e1952 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -61,6 +61,7 @@ class ReleaseContextManager: # create initial release info self.release_info = ReleaseInfo( release_branch="NA", + release_type="NA", commit_sha=args.ref, release_tag="NA", version="NA", @@ -93,6 +94,7 @@ class ReleaseContextManager: @dataclasses.dataclass class ReleaseInfo: version: str + release_type: str release_tag: str release_branch: str commit_sha: str @@ -131,7 +133,7 @@ class ReleaseInfo: return self def prepare( - self, commit_ref: str, release_type: str, skip_tag_check: bool + self, commit_ref: str, release_type: str, _skip_tag_check: bool ) -> "ReleaseInfo": version = None release_branch = None @@ -143,17 +145,18 @@ class ReleaseInfo: assert release_type in ("patch", "new") if release_type == "new": # check commit_ref is right and on a right branch - Shell.check( - f"git merge-base --is-ancestor {commit_ref} origin/master", - strict=True, - verbose=True, - ) + if commit_ref != "master": + Shell.check( + f"git merge-base --is-ancestor {commit_ref} origin/master", + strict=True, + verbose=True, + ) with checkout(commit_ref): commit_sha = Shell.get_output_or_raise(f"git rev-list -n1 {commit_ref}") # Git() must be inside "with checkout" contextmanager git = Git() version = get_version_from_repo(git=git) - release_branch = "master" + release_branch = f"{version.major}.{version.minor}" expected_prev_tag = f"v{version.major}.{version.minor}.1.1-new" version.bump().with_description(VersionType.NEW) assert ( @@ -204,10 +207,11 @@ class ReleaseInfo: expected_tag_prefix ) and git.latest_tag.endswith(expected_tag_suffix): pass - elif not skip_tag_check: - assert ( - False - ), f"BUG: Unexpected latest tag [{git.latest_tag}] expected [{expected_tag_prefix}*{expected_tag_suffix}]. Already Released?" + # TODO: uncomment and check with dry-run + # elif not skip_tag_check: + # assert ( + # False + # ), f"BUG: Unexpected latest tag [{git.latest_tag}] expected [{expected_tag_prefix}*{expected_tag_suffix}]. Already Released?" previous_release_sha = Shell.get_output_or_raise( f"git rev-list -n1 {previous_release_tag}" @@ -238,6 +242,7 @@ class ReleaseInfo: self.release_progress = ReleaseProgress.STARTED self.progress_status = ReleaseProgressDescription.OK self.latest = latest_release + self.release_type = release_type return self def push_release_tag(self, dry_run: bool) -> None: @@ -262,16 +267,15 @@ class ReleaseInfo: @staticmethod def _create_gh_label(label: str, color_hex: str, dry_run: bool) -> None: cmd = f"gh api repos/{CI.Envs.GITHUB_REPOSITORY}/labels -f name={label} -f color={color_hex}" - Shell.check(cmd, dry_run=dry_run, strict=True) + res = Shell.check(cmd, dry_run=dry_run, verbose=True) + if not res: + # not a critical error - do not fail. branch might be created already (recovery case) + print("WARNING: failed to create backport labels for the new branch") def push_new_release_branch(self, dry_run: bool) -> None: - assert ( - self.release_branch == "master" - ), "New release branch can be created only for release type [new]" git = Git() version = get_version_from_repo(git=git) - new_release_branch = f"{version.major}.{version.minor}" - stable_release_type = version.get_stable_release_type() + new_release_branch = self.release_branch version_after_release = copy(version) version_after_release.bump() assert ( @@ -285,11 +289,8 @@ class ReleaseInfo: print( f"Create and push new release branch [{new_release_branch}], commit [{self.commit_sha}]" ) - with checkout(self.release_branch): + with checkout("master"): with checkout_new(new_release_branch): - pr_labels = f"--label {CI.Labels.RELEASE}" - if stable_release_type == VersionType.LTS: - pr_labels += f" --label {CI.Labels.RELEASE_LTS}" cmd_push_branch = ( f"{GIT_PREFIX} push --set-upstream origin {new_release_branch}" ) @@ -302,67 +303,108 @@ class ReleaseInfo: ReleaseInfo._create_gh_label( f"v{new_release_branch}-affected", "c2bfff", dry_run=dry_run ) - Shell.check( - f"""gh pr create --repo {CI.Envs.GITHUB_REPOSITORY} --title 'Release pull request for branch {new_release_branch}' - --head {new_release_branch} {pr_labels} - --body 'This PullRequest is a part of ClickHouse release cycle. It is used by CI system only. Do not perform any changes with it.' - """, - dry_run=dry_run, - strict=True, - verbose=True, - ) def get_version_bump_branch(self): return f"bump_version_{self.version}" def update_version_and_contributors_list(self, dry_run: bool) -> None: - # Bump version, update contributors list, create PR - branch_upd_version_contributors = self.get_version_bump_branch() + # Bump version, update contributors list, create on release branch with checkout(self.commit_sha): git = Git() version = get_version_from_repo(git=git) - if self.release_branch == "master": + if self.release_type == "patch": + assert ( + version.string == self.version + ), f"BUG: version in release info does not match version in git commit, expected [{self.version}], got [{version.string}]" + version.bump_patch() + else: + version.reset_tweak() + version.with_description(version.get_stable_release_type()) + + with checkout(self.release_branch): + update_cmake_version(version) + update_contributors(raise_error=True) + cmd_commit_version_upd = f"{GIT_PREFIX} commit '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}' -m 'Update autogenerated version to {self.version} and contributors'" + cmd_push_branch = f"{GIT_PREFIX} push" + Shell.check( + cmd_commit_version_upd, strict=True, dry_run=dry_run, verbose=True + ) + Shell.check(cmd_push_branch, strict=True, dry_run=dry_run, verbose=True) + if dry_run: + Shell.check( + f"{GIT_PREFIX} diff '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", + verbose=True, + ) + Shell.check( + f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", + verbose=True, + ) + + # TODO: move to new GH step? + if self.release_type == "new": + print("Update version on master branch") + branch_upd_version_contributors = self.get_version_bump_branch() + with checkout(self.commit_sha): + git = Git() + version = get_version_from_repo(git=git) version.bump() version.with_description(VersionType.TESTING) - else: - version.with_description(version.get_stable_release_type()) - assert ( - version.string == self.version - ), f"BUG: version in release info does not match version in git commit, expected [{self.version}], got [{version.string}]" - with checkout(self.release_branch): - with checkout_new(branch_upd_version_contributors): - update_cmake_version(version) - update_contributors(raise_error=True) - cmd_commit_version_upd = f"{GIT_PREFIX} commit '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}' -m 'Update autogenerated version to {self.version} and contributors'" - cmd_push_branch = f"{GIT_PREFIX} push --set-upstream origin {branch_upd_version_contributors}" - actor = os.getenv("GITHUB_ACTOR", "") or "me" - body = f"Automatic version bump after release {self.release_tag}\n### Changelog category (leave one):\n- Not for changelog (changelog entry is not required)\n" - cmd_create_pr = f"gh pr create --repo {CI.Envs.GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base {self.release_branch} --body \"{body}\" --assignee {actor}" + with checkout("master"): + with checkout_new(branch_upd_version_contributors): + update_cmake_version(version) + update_contributors(raise_error=True) + cmd_commit_version_upd = f"{GIT_PREFIX} commit '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}' -m 'Update autogenerated version to {self.version} and contributors'" + cmd_push_branch = f"{GIT_PREFIX} push --set-upstream origin {branch_upd_version_contributors}" + actor = os.getenv("GITHUB_ACTOR", "") or "me" + body = f"Automatic version bump after release {self.release_tag}\n### Changelog category (leave one):\n- Not for changelog (changelog entry is not required)\n" + cmd_create_pr = f"gh pr create --repo {CI.Envs.GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base master --body \"{body}\" --assignee {actor}" + Shell.check( + cmd_commit_version_upd, + strict=True, + dry_run=dry_run, + verbose=True, + ) + Shell.check( + cmd_push_branch, strict=True, dry_run=dry_run, verbose=True + ) + Shell.check( + cmd_create_pr, strict=True, dry_run=dry_run, verbose=True + ) + if dry_run: + Shell.check( + f"{GIT_PREFIX} diff '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", + verbose=True, + ) + Shell.check( + f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", + verbose=True, + ) + self.version_bump_pr = "dry-run" + else: + self.version_bump_pr = GH.get_pr_url_by_branch( + branch=branch_upd_version_contributors + ) + + # TODO: move to new GH step? + print("Create Release PR") + with checkout(self.release_branch): + pr_labels = f"--label {CI.Labels.RELEASE}" + if version.get_stable_release_type() == VersionType.LTS: + pr_labels += f" --label {CI.Labels.RELEASE_LTS}" Shell.check( - cmd_commit_version_upd, strict=True, dry_run=dry_run, verbose=True + f"""gh pr create --repo {CI.Envs.GITHUB_REPOSITORY} --title 'Release pull request for branch {self.release_branch}' \ + --head {self.release_branch} {pr_labels} \ + --body 'This PullRequest is a part of ClickHouse release cycle. It is used by CI system only. Do not perform any changes with it.'""", + dry_run=dry_run, + strict=True, + verbose=True, ) - Shell.check(cmd_push_branch, strict=True, dry_run=dry_run, verbose=True) - Shell.check(cmd_create_pr, strict=True, dry_run=dry_run, verbose=True) - if dry_run: - Shell.check( - f"{GIT_PREFIX} diff '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", - verbose=True, - ) - Shell.check( - f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", - verbose=True, - ) - self.version_bump_pr = "dry-run" - else: - self.version_bump_pr = GH.get_pr_url_by_branch( - branch=branch_upd_version_contributors - ) def get_change_log_branch(self): return f"auto/{self.release_tag}" def update_release_info(self, dry_run: bool) -> "ReleaseInfo": - if self.release_branch != "master": + if self.release_type == "patch": if not self.changelog_pr: branch = self.get_change_log_branch() if not dry_run: @@ -371,21 +413,22 @@ class ReleaseInfo: url = "dry-run" print(f"ChangeLog PR url [{url}]") self.changelog_pr = url - - if not self.version_bump_pr: - branch = self.get_version_bump_branch() - if not dry_run: - url = GH.get_pr_url_by_branch(branch=branch) - else: - url = "dry-run" - print(f"Version bump PR url [{url}]") - self.version_bump_pr = url - - self.release_url = f"https://github.com/{CI.Envs.GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" - print(f"Release url [{self.release_url}]") - self.docker = f"docker run --rm clickhouse/clickhouse:{self.version} clickhouse --version" + else: + # new release branch - find version bump pr on a master branch + branch = self.get_version_bump_branch() + if not dry_run: + url = GH.get_pr_url_by_branch(branch=branch) + else: + url = "dry-run" + print(f"Version bump PR url [{url}]") + self.version_bump_pr = url + + self.release_url = f"https://github.com/{CI.Envs.GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" + print(f"Release url [{self.release_url}]") + self.dump() + return self def create_gh_release(self, packages_files: List[str], dry_run: bool) -> None: @@ -410,35 +453,40 @@ class ReleaseInfo: def merge_prs(self, dry_run: bool) -> None: repo = CI.Envs.GITHUB_REPOSITORY - assert self.version_bump_pr - if dry_run: - version_bump_pr_num = 12345 - else: - version_bump_pr_num = int(self.version_bump_pr.split("/")[-1]) - print("Merging Version bump PR") - res_1 = Shell.check( - f"gh pr merge {version_bump_pr_num} --repo {repo} --merge --auto", - verbose=True, - dry_run=dry_run, - ) - - res_2 = True - if not self.release_tag.endswith("-new"): + if self.release_type == "patch": assert self.changelog_pr print("Merging ChangeLog PR") if dry_run: changelog_pr_num = 23456 else: changelog_pr_num = int(self.changelog_pr.split("/")[-1]) - res_2 = Shell.check( + res = Shell.check( f"gh pr merge {changelog_pr_num} --repo {repo} --merge --auto", verbose=True, dry_run=dry_run, ) else: - assert not self.changelog_pr + if not dry_run: + assert not self.changelog_pr + res = True - self.prs_merged = res_1 and res_2 + if self.release_type == "new": + assert self.version_bump_pr + print("Merging Version Bump PR") + if dry_run: + version_bump_pr = 23456 + else: + version_bump_pr = int(self.version_bump_pr.split("/")[-1]) + res = res and Shell.check( + f"gh pr merge {version_bump_pr} --repo {repo} --merge --auto", + verbose=True, + dry_run=dry_run, + ) + else: + if not dry_run: + assert not self.changelog_pr + + self.prs_merged = res class RepoTypes: @@ -759,7 +807,7 @@ if __name__ == "__main__": release_info.prepare( commit_ref=args.ref, release_type=args.release_type, - skip_tag_check=args.skip_tag_check, + _skip_tag_check=args.skip_tag_check, ) if args.download_packages: diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index 3251ec5644e..34439c19f0a 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -70,7 +70,7 @@ def parse_args() -> argparse.Namespace: parser.add_argument( "--tag-type", type=str, - choices=("head", "release", "latest-release"), + choices=("head", "release", "release-latest"), default="head", help="defines required tags for resulting docker image. " "head - for master image (tag: head) " diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index 07a7a9601c0..b20b2bb25cf 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -85,6 +85,16 @@ class ClickHouseVersion: self._tweak = 1 return self + def bump_patch(self) -> "ClickHouseVersion": + self._revision += 1 + self._patch += 1 + self._tweak = 1 + return self + + def reset_tweak(self) -> "ClickHouseVersion": + self._tweak = 1 + return self + def major_update(self) -> "ClickHouseVersion": if self._git is not None: self._git.update() @@ -104,13 +114,6 @@ class ClickHouseVersion: self.major, self.minor, self.patch + 1, self.revision, self._git ) - def reset_tweak(self) -> "ClickHouseVersion": - if self._git is not None: - self._git.update() - return ClickHouseVersion( - self.major, self.minor, self.patch, self.revision, self._git, 1 - ) - @property def major(self) -> int: return self._major From b5134fd4903b91250bb6db16a8d52ff0b2469686 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 13 Aug 2024 16:13:25 +0100 Subject: [PATCH 642/644] fix build --- base/base/cgroupsv2.cpp | 6 +++++- base/base/cgroupsv2.h | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/base/base/cgroupsv2.cpp b/base/base/cgroupsv2.cpp index b4ca8271d64..e0e37c8729b 100644 --- a/base/base/cgroupsv2.cpp +++ b/base/base/cgroupsv2.cpp @@ -51,8 +51,9 @@ fs::path cgroupV2PathOfProcess() #endif } -std::optional getCgroupsV2PathContainingFile(std::string_view file_name) +std::optional getCgroupsV2PathContainingFile([[maybe_unused]] std::string_view file_name) { +#if defined(OS_LINUX) if (!cgroupsV2Enabled()) return {}; @@ -70,4 +71,7 @@ std::optional getCgroupsV2PathContainingFile(std::string_view file_ current_cgroup = current_cgroup.parent_path(); } return {}; +#else + return {}; +#endif } diff --git a/base/base/cgroupsv2.h b/base/base/cgroupsv2.h index 925a399471e..a6276474254 100644 --- a/base/base/cgroupsv2.h +++ b/base/base/cgroupsv2.h @@ -19,4 +19,4 @@ std::filesystem::path cgroupV2PathOfProcess(); /// Returns the most nested cgroup dir containing the specified file. /// If cgroups v2 is not enabled - returns an empty optional. -std::optional getCgroupsV2PathContainingFile(std::string_view file_name); +std::optional getCgroupsV2PathContainingFile([[maybe_unused]] std::string_view file_name); From f6f79e188d6c5a16bb327f9e62dce506ea8e8a19 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 14 Aug 2024 00:23:18 +0000 Subject: [PATCH 643/644] Apply libunwind changes needed for musl --- contrib/libunwind | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libunwind b/contrib/libunwind index a89d904befe..601db0b0e03 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit a89d904befea07814628c6ce0b44083c4e149c62 +Subproject commit 601db0b0e03018c01710470a37703b618f9cf08b From 962bf1d821a498aaeb6f16e5d4205272cfd00001 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 14 Aug 2024 13:37:14 +0200 Subject: [PATCH 644/644] CI: Fix for critical bug fix regex --- tests/ci/changelog.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index 39e426945d3..b7f73f22016 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -115,7 +115,6 @@ def get_descriptions(prs: PullRequests) -> Dict[str, List[Description]]: # pylint: enable=protected-access if repo_name not in repos: repos[repo_name] = pr.base.repo - in_changelog = False merge_commit = pr.merge_commit_sha if merge_commit is None: logging.warning("PR %s does not have merge-commit, skipping", pr.number) @@ -291,7 +290,7 @@ def generate_description(item: PullRequest, repo: Repository) -> Optional[Descri # Normalize bug fixes if ( re.match( - r"(?i)bug\Wfix", + r".*(?i)bug\Wfix", category, ) # Map "Critical Bug Fix" to "Bug fix" category for changelog