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/371] 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/371] 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/371] 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/371] 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/371] 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/371] 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/371] 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/371] 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/371] 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/371] 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/371] 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/371] 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/371] 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/371] 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/371] 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/371] 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 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 017/371] 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 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 018/371] 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 019/371] 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 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 020/371] 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 021/371] 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 022/371] 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 023/371] 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 024/371] 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 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 025/371] 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 026/371] 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 027/371] 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 028/371] 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 029/371] 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 030/371] 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 031/371] 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 032/371] 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 033/371] 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 034/371] 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 035/371] 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 036/371] 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 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 037/371] 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 038/371] 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 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 039/371] 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 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 040/371] 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 041/371] 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 766130bc98c116d198343f8fee6e0e5527fad712 Mon Sep 17 00:00:00 2001 From: skyoct Date: Tue, 18 Jun 2024 19:16:32 +0800 Subject: [PATCH 042/371] feat: add etag for object storage --- .../AzureBlobStorage/AzureObjectStorage.cpp | 2 ++ src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp | 2 +- src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp | 1 + src/Disks/ObjectStorages/IObjectStorage.h | 1 + src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 4 +++- src/IO/S3/getObjectInfo.cpp | 2 ++ src/IO/S3/getObjectInfo.h | 1 + .../ObjectStorage/StorageObjectStorageSource.cpp | 3 ++- src/Storages/VirtualColumnUtils.cpp | 10 +++++++++- src/Storages/VirtualColumnUtils.h | 2 +- 10 files changed, 23 insertions(+), 5 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index e7ecf7cd515..e4b85b79ab4 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -86,6 +86,7 @@ private: Poco::Timestamp::fromEpochTime( std::chrono::duration_cast( static_cast(blob.Details.LastModified).time_since_epoch()).count()), + blob.Details.ETag.ToString(), {}})); } @@ -186,6 +187,7 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith Poco::Timestamp::fromEpochTime( std::chrono::duration_cast( static_cast(blob.Details.LastModified).time_since_epoch()).count()), + blob.Details.ETag.ToString(), {}})); } diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp index 44854633d65..e9114c75077 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp @@ -205,7 +205,7 @@ void DiskObjectStorageMetadata::addObject(ObjectStorageKey key, size_t size) } total_size += size; - keys_with_meta.emplace_back(std::move(key), ObjectMetadata{size, {}, {}}); + keys_with_meta.emplace_back(std::move(key), ObjectMetadata{size, {}, {}, {}}); } ObjectKeyWithMetadata DiskObjectStorageMetadata::popLastObject() diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index dcb2af9d4d3..a28f1888020 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -221,6 +221,7 @@ void HDFSObjectStorage::listObjects(const std::string & path, RelativePathsWithM ObjectMetadata{ static_cast(ls.file_info[i].mSize), Poco::Timestamp::fromEpochTime(ls.file_info[i].mLastMod), + "", {}})); } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 7bc9e4073db..c1402522c5f 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -54,6 +54,7 @@ struct ObjectMetadata { uint64_t size_bytes = 0; Poco::Timestamp last_modified; + std::string etag; ObjectAttributes attributes; }; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 63e11dcd8c8..7b2f71a828e 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -145,7 +145,7 @@ private: auto objects = outcome.GetResult().GetContents(); for (const auto & object : objects) { - ObjectMetadata metadata{static_cast(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), {}}; + ObjectMetadata metadata{static_cast(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), object.GetETag(), {}}; batch.emplace_back(std::make_shared(object.GetKey(), std::move(metadata))); } @@ -329,6 +329,7 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet ObjectMetadata{ static_cast(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), + object.GetETag(), {}})); if (max_keys) @@ -476,6 +477,7 @@ ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) cons ObjectMetadata result; result.size_bytes = object_info.size; result.last_modified = Poco::Timestamp::fromEpochTime(object_info.last_modification_time); + result.etag = object_info.etag; result.attributes = object_info.metadata; return result; diff --git a/src/IO/S3/getObjectInfo.cpp b/src/IO/S3/getObjectInfo.cpp index 9271ad820e4..afa4079c261 100644 --- a/src/IO/S3/getObjectInfo.cpp +++ b/src/IO/S3/getObjectInfo.cpp @@ -54,6 +54,8 @@ namespace ObjectInfo object_info; object_info.size = static_cast(result.GetContentLength()); object_info.last_modification_time = result.GetLastModified().Seconds(); + String etag(result.GetETag.c_str(), result.GetETag().size()); + object_info.etag = etag; if (with_metadata) object_info.metadata = result.GetMetadata(); diff --git a/src/IO/S3/getObjectInfo.h b/src/IO/S3/getObjectInfo.h index 32f34f74069..2fec407f70e 100644 --- a/src/IO/S3/getObjectInfo.h +++ b/src/IO/S3/getObjectInfo.h @@ -15,6 +15,7 @@ struct ObjectInfo { size_t size = 0; time_t last_modification_time = 0; + String etag = ""; std::map metadata = {}; /// Set only if getObjectInfo() is called with `with_metadata = true`. }; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 2fc6993369d..8554fd9235d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -201,7 +201,8 @@ Chunk StorageObjectStorageSource::generate() .path = getUniqueStoragePathIdentifier(*configuration, reader.getObjectInfo(), false), .size = object_info.metadata->size_bytes, .filename = &filename, - .last_modified = object_info.metadata->last_modified + .last_modified = object_info.metadata->last_modified, + .etag = &(object_info.metadata->etag) }); return chunk; } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 778c9e13adb..960fff371a7 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -112,7 +112,7 @@ void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context) NameSet getVirtualNamesForFileLikeStorage() { - return {"_path", "_file", "_size", "_time"}; + return {"_path", "_file", "_size", "_time", "_etag", "_last_modified"}; } VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns) @@ -131,6 +131,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription add_virtual("_file", std::make_shared(std::make_shared())); add_virtual("_size", makeNullable(std::make_shared())); add_virtual("_time", makeNullable(std::make_shared())); + add_virtual("_etag", std::make_shared(std::make_shared())); return desc; } @@ -226,6 +227,13 @@ void addRequestedFileLikeStorageVirtualsToChunk( else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } + else if (virtual_column.name == "_etag") + { + if (virtual_values.etag) + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), (*virtual_values.etag))->convertToFullColumnIfConst()); + else + chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); + } } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index fbfbdd6c6cc..dc178277556 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -74,7 +74,7 @@ struct VirtualsForFileLikeStorage std::optional size { std::nullopt }; const String * filename { nullptr }; std::optional last_modified { std::nullopt }; - + const String * etag { nullptr }; }; void addRequestedFileLikeStorageVirtualsToChunk( 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 043/371] 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 044/371] 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 045/371] 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 046/371] 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 047/371] 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 048/371] 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 049/371] 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 8ca47905ef000d3eb72a89d9dfcd9b989fce4203 Mon Sep 17 00:00:00 2001 From: skyoct Date: Wed, 19 Jun 2024 14:06:36 +0800 Subject: [PATCH 050/371] fix get tag --- src/IO/S3/getObjectInfo.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/IO/S3/getObjectInfo.cpp b/src/IO/S3/getObjectInfo.cpp index afa4079c261..a21fb9fce54 100644 --- a/src/IO/S3/getObjectInfo.cpp +++ b/src/IO/S3/getObjectInfo.cpp @@ -54,8 +54,7 @@ namespace ObjectInfo object_info; object_info.size = static_cast(result.GetContentLength()); object_info.last_modification_time = result.GetLastModified().Seconds(); - String etag(result.GetETag.c_str(), result.GetETag().size()); - object_info.etag = etag; + object_info.etag = result.GetETag(); if (with_metadata) object_info.metadata = result.GetMetadata(); 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 051/371] 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 052/371] 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 7523d8b1aacf8b4a9b2fa6d7bc5e54f3ee61ffec Mon Sep 17 00:00:00 2001 From: skyoct Date: Wed, 19 Jun 2024 21:24:26 +0800 Subject: [PATCH 053/371] Feat add docs --- docs/en/engines/table-engines/integrations/s3.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 93f4a187656..d664c37bd0f 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -146,6 +146,7 @@ Code: 48. DB::Exception: Received from localhost:9000. DB::Exception: Reading fr - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_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`. +- `_etag` — ETag of the file. Type: `LowCardinalty(String)`. If the etag is unknown, the value is `NULL`. For more information about virtual columns see [here](../../../engines/table-engines/index.md#table_engines-virtual_columns). From fdfa6adbfa9b44f0943a39f5188285aff4329640 Mon Sep 17 00:00:00 2001 From: skyoct Date: Thu, 20 Jun 2024 22:54:33 +0800 Subject: [PATCH 054/371] ignore docs dict etag --- 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 64ff3e8e2cb..1be52597133 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -248,6 +248,7 @@ DoubleDelta Doxygen Durre ECMA +ETag Ecto EdgeAngle EdgeLengthKm From ff72bbb18d1e78f1edd11e2d04afdbfef00b1b9d Mon Sep 17 00:00:00 2001 From: skyoct Date: Thu, 20 Jun 2024 23:07:16 +0800 Subject: [PATCH 055/371] ignore etag dict --- 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 1be52597133..dda0cf51455 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1569,6 +1569,7 @@ enum's enums erfc errorCodeToName +etag evalMLMethod exFAT expiryMsec 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 056/371] 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 057/371] 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 531a7e3592aa165418b03b1603f30b4935990b81 Mon Sep 17 00:00:00 2001 From: skyoct Date: Mon, 24 Jun 2024 18:26:40 +0800 Subject: [PATCH 058/371] fix etag init --- src/IO/S3/getObjectInfo.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/S3/getObjectInfo.h b/src/IO/S3/getObjectInfo.h index 2fec407f70e..30d4c627d37 100644 --- a/src/IO/S3/getObjectInfo.h +++ b/src/IO/S3/getObjectInfo.h @@ -15,7 +15,7 @@ struct ObjectInfo { size_t size = 0; time_t last_modification_time = 0; - String etag = ""; + String etag; std::map metadata = {}; /// Set only if getObjectInfo() is called with `with_metadata = true`. }; From 15b36c1ece293157b636a3efdb4727b73c58637f Mon Sep 17 00:00:00 2001 From: skyoct Date: Wed, 26 Jun 2024 22:28:31 +0800 Subject: [PATCH 059/371] add s3 tests --- src/Storages/ObjectStorage/StorageObjectStorageSource.cpp | 2 +- tests/queries/0_stateless/02245_s3_virtual_columns.reference | 4 ++-- tests/queries/0_stateless/02245_s3_virtual_columns.sql | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 31d46a93e58..46b4b34cb24 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -202,7 +202,7 @@ Chunk StorageObjectStorageSource::generate() .size = object_info->metadata->size_bytes, .filename = &filename, .last_modified = object_info->metadata->last_modified, - .etag = &(object_info.metadata->etag) + .etag = &(object_info->metadata->etag) }); return chunk; } diff --git a/tests/queries/0_stateless/02245_s3_virtual_columns.reference b/tests/queries/0_stateless/02245_s3_virtual_columns.reference index 09383c51888..b0af2e54dfd 100644 --- a/tests/queries/0_stateless/02245_s3_virtual_columns.reference +++ b/tests/queries/0_stateless/02245_s3_virtual_columns.reference @@ -11,5 +11,5 @@ create table test_02245_2 (a UInt64, _path Int32) engine = S3(s3_conn, filename= insert into test_02245_2 select 1, 2 settings s3_truncate_on_insert=1; select * from test_02245_2; 1 2 -select _path from test_02245_2; -2 +select _path, isNotNull(_etag) from test_02245_2; +2 1 diff --git a/tests/queries/0_stateless/02245_s3_virtual_columns.sql b/tests/queries/0_stateless/02245_s3_virtual_columns.sql index e86344d2094..a66b212e5c7 100644 --- a/tests/queries/0_stateless/02245_s3_virtual_columns.sql +++ b/tests/queries/0_stateless/02245_s3_virtual_columns.sql @@ -12,4 +12,4 @@ drop table if exists test_02245_2; create table test_02245_2 (a UInt64, _path Int32) engine = S3(s3_conn, filename='test_02245_2', format=Parquet); insert into test_02245_2 select 1, 2 settings s3_truncate_on_insert=1; select * from test_02245_2; -select _path from test_02245_2; +select _path, isNotNull(_etag) from test_02245_2; From 7cfbd1427011a34b07ac65a39d5ae4a0bfc34141 Mon Sep 17 00:00:00 2001 From: skyoct Date: Thu, 27 Jun 2024 18:05:19 +0800 Subject: [PATCH 060/371] fix tests --- tests/queries/0_stateless/02245_s3_virtual_columns.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02245_s3_virtual_columns.reference b/tests/queries/0_stateless/02245_s3_virtual_columns.reference index b0af2e54dfd..3822f6ffa0f 100644 --- a/tests/queries/0_stateless/02245_s3_virtual_columns.reference +++ b/tests/queries/0_stateless/02245_s3_virtual_columns.reference @@ -12,4 +12,4 @@ insert into test_02245_2 select 1, 2 settings s3_truncate_on_insert=1; select * from test_02245_2; 1 2 select _path, isNotNull(_etag) from test_02245_2; -2 1 +2 1 From 71d71bd5fe8884b5f2b11e06302dce8e511f5b7c Mon Sep 17 00:00:00 2001 From: morning-color Date: Thu, 27 Jun 2024 20:06:14 +0800 Subject: [PATCH 061/371] 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 f58f6cfa737b1f18bab76ac8869f769b1908ea09 Mon Sep 17 00:00:00 2001 From: skyoct Date: Mon, 1 Jul 2024 18:26:54 +0800 Subject: [PATCH 062/371] remove _last_modified field --- 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 960fff371a7..a557848698a 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -112,7 +112,7 @@ void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context) NameSet getVirtualNamesForFileLikeStorage() { - return {"_path", "_file", "_size", "_time", "_etag", "_last_modified"}; + return {"_path", "_file", "_size", "_time", "_etag"}; } VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns) From a99c803ddf137e91f43c8f26f549f74f71eab102 Mon Sep 17 00:00:00 2001 From: morning-color Date: Thu, 4 Jul 2024 17:26:06 +0800 Subject: [PATCH 063/371] 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 064/371] 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 065/371] 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 066/371] 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 1505cb2b788b1e7d1f5721117f33e061e7ee21e1 Mon Sep 17 00:00:00 2001 From: skyoct Date: Tue, 9 Jul 2024 13:33:00 +0800 Subject: [PATCH 067/371] fix etag init --- src/Storages/ObjectStorage/StorageObjectStorageSource.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index b3377e902ec..4e41e32fd32 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -201,8 +201,9 @@ Chunk StorageObjectStorageSource::generate() {.path = getUniqueStoragePathIdentifier(*configuration, *object_info, false), .size = object_info->isArchive() ? object_info->fileSizeInArchive() : object_info->metadata->size_bytes, .filename = &filename, - .etag = &(object_info->metadata->etag), - .last_modified = object_info->metadata->last_modified}); + .last_modified = object_info->metadata->last_modified, + .etag = &(object_info->metadata->etag) + }); const auto & partition_columns = configuration->getPartitionColumns(); if (!partition_columns.empty() && chunk_size && chunk.hasColumns()) From 6cd6319ba70945e7ae50447772c57d61e488e72e Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Jul 2024 15:32:28 +0000 Subject: [PATCH 068/371] 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 069/371] 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 070/371] 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 071/371] 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 243edcc8aa622d47a90cb4fba33f5079269e39b8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 12 Jul 2024 01:01:52 +0200 Subject: [PATCH 072/371] add logs and metrics about rejected connections in Poco --- .../Foundation/include/Poco/ErrorHandler.h | 8 + base/poco/Foundation/src/ErrorHandler.cpp | 92 ++++--- base/poco/Net/src/SocketImpl.cpp | 1 + base/poco/Net/src/TCPServer.cpp | 212 +++++++-------- base/poco/Net/src/TCPServerDispatcher.cpp | 247 ++++++++++-------- programs/keeper/Keeper.cpp | 2 +- programs/server/Server.cpp | 4 +- src/Common/AsynchronousMetrics.cpp | 31 ++- src/Common/AsynchronousMetrics.h | 1 + src/Common/ErrorHandlers.h | 25 ++ src/Server/ProtocolServerAdapter.cpp | 2 + src/Server/ProtocolServerAdapter.h | 3 + 12 files changed, 367 insertions(+), 261 deletions(-) diff --git a/base/poco/Foundation/include/Poco/ErrorHandler.h b/base/poco/Foundation/include/Poco/ErrorHandler.h index c0b5bf9621e..961fec2bc3b 100644 --- a/base/poco/Foundation/include/Poco/ErrorHandler.h +++ b/base/poco/Foundation/include/Poco/ErrorHandler.h @@ -21,6 +21,7 @@ #include "Poco/Exception.h" #include "Poco/Foundation.h" #include "Poco/Mutex.h" +#include "Poco/Message.h" namespace Poco @@ -78,6 +79,10 @@ public: /// /// The default implementation just breaks into the debugger. + virtual void logMessageImpl(Message::Priority priority, const std::string & msg) {} + /// Write a messages to the log + /// Useful for logging from Poco + static void handle(const Exception & exc); /// Invokes the currently registered ErrorHandler. @@ -87,6 +92,9 @@ public: static void handle(); /// Invokes the currently registered ErrorHandler. + static void logMessage(Message::Priority priority, const std::string & msg); + /// Invokes the currently registered ErrorHandler. + static ErrorHandler * set(ErrorHandler * pHandler); /// Registers the given handler as the current error handler. /// diff --git a/base/poco/Foundation/src/ErrorHandler.cpp b/base/poco/Foundation/src/ErrorHandler.cpp index d0af8ea8a12..1d0a41b77ec 100644 --- a/base/poco/Foundation/src/ErrorHandler.cpp +++ b/base/poco/Foundation/src/ErrorHandler.cpp @@ -8,7 +8,7 @@ // Copyright (c) 2005-2006, Applied Informatics Software Engineering GmbH. // and Contributors. // -// SPDX-License-Identifier: BSL-1.0 +// SPDX-License-Identifier: BSL-1.0 // @@ -35,79 +35,91 @@ ErrorHandler::~ErrorHandler() void ErrorHandler::exception(const Exception& exc) { - poco_debugger_msg(exc.what()); + poco_debugger_msg(exc.what()); } - + void ErrorHandler::exception(const std::exception& exc) { - poco_debugger_msg(exc.what()); + poco_debugger_msg(exc.what()); } void ErrorHandler::exception() { - poco_debugger_msg("unknown exception"); + poco_debugger_msg("unknown exception"); } void ErrorHandler::handle(const Exception& exc) { - FastMutex::ScopedLock lock(_mutex); - try - { - _pHandler->exception(exc); - } - catch (...) - { - } + FastMutex::ScopedLock lock(_mutex); + try + { + _pHandler->exception(exc); + } + catch (...) + { + } } - + void ErrorHandler::handle(const std::exception& exc) { - FastMutex::ScopedLock lock(_mutex); - try - { - _pHandler->exception(exc); - } - catch (...) - { - } + FastMutex::ScopedLock lock(_mutex); + try + { + _pHandler->exception(exc); + } + catch (...) + { + } } void ErrorHandler::handle() { - FastMutex::ScopedLock lock(_mutex); - try - { - _pHandler->exception(); - } - catch (...) - { - } + FastMutex::ScopedLock lock(_mutex); + try + { + _pHandler->exception(); + } + catch (...) + { + } +} + +void ErrorHandler::logMessage(Message::Priority priority, const std::string & msg) +{ + FastMutex::ScopedLock lock(_mutex); + try + { + _pHandler->logMessageImpl(priority, msg); + } + catch (...) + { + } } ErrorHandler* ErrorHandler::set(ErrorHandler* pHandler) { - poco_check_ptr(pHandler); + poco_check_ptr(pHandler); - FastMutex::ScopedLock lock(_mutex); - ErrorHandler* pOld = _pHandler; - _pHandler = pHandler; - return pOld; + FastMutex::ScopedLock lock(_mutex); + ErrorHandler* pOld = _pHandler; + _pHandler = pHandler; + return pOld; } ErrorHandler* ErrorHandler::defaultHandler() { - // NOTE: Since this is called to initialize the static _pHandler - // variable, sh has to be a local static, otherwise we run - // into static initialization order issues. - static SingletonHolder sh; - return sh.get(); + // NOTE: Since this is called to initialize the static _pHandler + // variable, sh has to be a local static, otherwise we run + // into static initialization order issues. + static SingletonHolder sh; + return sh.get(); } diff --git a/base/poco/Net/src/SocketImpl.cpp b/base/poco/Net/src/SocketImpl.cpp index 484b8cfeec3..13a655d153d 100644 --- a/base/poco/Net/src/SocketImpl.cpp +++ b/base/poco/Net/src/SocketImpl.cpp @@ -17,6 +17,7 @@ #include "Poco/Net/StreamSocketImpl.h" #include "Poco/NumberFormatter.h" #include "Poco/Timestamp.h" +#include "Poco/ErrorHandler.h" #include // FD_SET needs memset on some platforms, so we can't use diff --git a/base/poco/Net/src/TCPServer.cpp b/base/poco/Net/src/TCPServer.cpp index 9bdae900bd6..b957829fb7d 100644 --- a/base/poco/Net/src/TCPServer.cpp +++ b/base/poco/Net/src/TCPServer.cpp @@ -8,7 +8,7 @@ // Copyright (c) 2005-2006, Applied Informatics Software Engineering GmbH. // and Contributors. // -// SPDX-License-Identifier: BSL-1.0 +// SPDX-License-Identifier: BSL-1.0 // @@ -44,190 +44,194 @@ TCPServerConnectionFilter::~TCPServerConnectionFilter() TCPServer::TCPServer(TCPServerConnectionFactory::Ptr pFactory, Poco::UInt16 portNumber, TCPServerParams::Ptr pParams): - _socket(ServerSocket(portNumber)), - _thread(threadName(_socket)), - _stopped(true) -{ - Poco::ThreadPool& pool = Poco::ThreadPool::defaultPool(); - if (pParams) - { - int toAdd = pParams->getMaxThreads() - pool.capacity(); - if (toAdd > 0) pool.addCapacity(toAdd); - } - _pDispatcher = new TCPServerDispatcher(pFactory, pool, pParams); - + _socket(ServerSocket(portNumber)), + _thread(threadName(_socket)), + _stopped(true) +{ + Poco::ThreadPool& pool = Poco::ThreadPool::defaultPool(); + if (pParams) + { + int toAdd = pParams->getMaxThreads() - pool.capacity(); + if (toAdd > 0) pool.addCapacity(toAdd); + } + _pDispatcher = new TCPServerDispatcher(pFactory, pool, pParams); + } TCPServer::TCPServer(TCPServerConnectionFactory::Ptr pFactory, const ServerSocket& socket, TCPServerParams::Ptr pParams): - _socket(socket), - _thread(threadName(socket)), - _stopped(true) + _socket(socket), + _thread(threadName(socket)), + _stopped(true) { - Poco::ThreadPool& pool = Poco::ThreadPool::defaultPool(); - if (pParams) - { - int toAdd = pParams->getMaxThreads() - pool.capacity(); - if (toAdd > 0) pool.addCapacity(toAdd); - } - _pDispatcher = new TCPServerDispatcher(pFactory, pool, pParams); + Poco::ThreadPool& pool = Poco::ThreadPool::defaultPool(); + if (pParams) + { + int toAdd = pParams->getMaxThreads() - pool.capacity(); + if (toAdd > 0) pool.addCapacity(toAdd); + } + _pDispatcher = new TCPServerDispatcher(pFactory, pool, pParams); } TCPServer::TCPServer(TCPServerConnectionFactory::Ptr pFactory, Poco::ThreadPool& threadPool, const ServerSocket& socket, TCPServerParams::Ptr pParams): - _socket(socket), - _pDispatcher(new TCPServerDispatcher(pFactory, threadPool, pParams)), - _thread(threadName(socket)), - _stopped(true) + _socket(socket), + _pDispatcher(new TCPServerDispatcher(pFactory, threadPool, pParams)), + _thread(threadName(socket)), + _stopped(true) { } TCPServer::~TCPServer() { - try - { - stop(); - _pDispatcher->release(); - } - catch (...) - { - poco_unexpected(); - } + try + { + stop(); + _pDispatcher->release(); + } + catch (...) + { + poco_unexpected(); + } } const TCPServerParams& TCPServer::params() const { - return _pDispatcher->params(); + return _pDispatcher->params(); } void TCPServer::start() { - poco_assert (_stopped); + poco_assert (_stopped); - _stopped = false; - _thread.start(*this); + _stopped = false; + _thread.start(*this); } - + void TCPServer::stop() { - if (!_stopped) - { - _stopped = true; - _thread.join(); - _pDispatcher->stop(); - } + if (!_stopped) + { + _stopped = true; + _thread.join(); + _pDispatcher->stop(); + } } void TCPServer::run() { - while (!_stopped) - { - Poco::Timespan timeout(250000); - try - { - if (_socket.poll(timeout, Socket::SELECT_READ)) - { - try - { - StreamSocket ss = _socket.acceptConnection(); - - if (!_pConnectionFilter || _pConnectionFilter->accept(ss)) - { - // enable nodelay per default: OSX really needs that + while (!_stopped) + { + Poco::Timespan timeout(250000); + try + { + if (_socket.poll(timeout, Socket::SELECT_READ)) + { + try + { + StreamSocket ss = _socket.acceptConnection(); + + if (!_pConnectionFilter || _pConnectionFilter->accept(ss)) + { + // enable nodelay per default: OSX really needs that #if defined(POCO_OS_FAMILY_UNIX) - if (ss.address().family() != AddressFamily::UNIX_LOCAL) + if (ss.address().family() != AddressFamily::UNIX_LOCAL) #endif - { - ss.setNoDelay(true); - } - _pDispatcher->enqueue(ss); - } - } - catch (Poco::Exception& exc) - { - ErrorHandler::handle(exc); - } - catch (std::exception& exc) - { - ErrorHandler::handle(exc); - } - catch (...) - { - ErrorHandler::handle(); - } - } - } - catch (Poco::Exception& exc) - { - ErrorHandler::handle(exc); - // possibly a resource issue since poll() failed; - // give some time to recover before trying again - Poco::Thread::sleep(50); - } - } + { + ss.setNoDelay(true); + } + _pDispatcher->enqueue(ss); + } + else + { + ErrorHandler::logMessage(Message::PRIO_WARNING, "Filtered out connection from " + ss.peerAddress().toString()); + } + } + catch (Poco::Exception& exc) + { + ErrorHandler::handle(exc); + } + catch (std::exception& exc) + { + ErrorHandler::handle(exc); + } + catch (...) + { + ErrorHandler::handle(); + } + } + } + catch (Poco::Exception& exc) + { + ErrorHandler::handle(exc); + // possibly a resource issue since poll() failed; + // give some time to recover before trying again + Poco::Thread::sleep(50); + } + } } int TCPServer::currentThreads() const { - return _pDispatcher->currentThreads(); + return _pDispatcher->currentThreads(); } int TCPServer::maxThreads() const { - return _pDispatcher->maxThreads(); + return _pDispatcher->maxThreads(); } - + int TCPServer::totalConnections() const { - return _pDispatcher->totalConnections(); + return _pDispatcher->totalConnections(); } int TCPServer::currentConnections() const { - return _pDispatcher->currentConnections(); + return _pDispatcher->currentConnections(); } int TCPServer::maxConcurrentConnections() const { - return _pDispatcher->maxConcurrentConnections(); + return _pDispatcher->maxConcurrentConnections(); } - + int TCPServer::queuedConnections() const { - return _pDispatcher->queuedConnections(); + return _pDispatcher->queuedConnections(); } int TCPServer::refusedConnections() const { - return _pDispatcher->refusedConnections(); + return _pDispatcher->refusedConnections(); } void TCPServer::setConnectionFilter(const TCPServerConnectionFilter::Ptr& pConnectionFilter) { - poco_assert (_stopped); + poco_assert (_stopped); - _pConnectionFilter = pConnectionFilter; + _pConnectionFilter = pConnectionFilter; } std::string TCPServer::threadName(const ServerSocket& socket) { - std::string name("TCPServer: "); - name.append(socket.address().toString()); - return name; + std::string name("TCPServer: "); + name.append(socket.address().toString()); + return name; } diff --git a/base/poco/Net/src/TCPServerDispatcher.cpp b/base/poco/Net/src/TCPServerDispatcher.cpp index 7f9f9a20ee7..9ca7c271e63 100644 --- a/base/poco/Net/src/TCPServerDispatcher.cpp +++ b/base/poco/Net/src/TCPServerDispatcher.cpp @@ -8,7 +8,7 @@ // Copyright (c) 2005-2007, Applied Informatics Software Engineering GmbH. // and Contributors. // -// SPDX-License-Identifier: BSL-1.0 +// SPDX-License-Identifier: BSL-1.0 // @@ -33,44 +33,44 @@ namespace Net { class TCPConnectionNotification: public Notification { public: - TCPConnectionNotification(const StreamSocket& socket): - _socket(socket) - { - } - - ~TCPConnectionNotification() - { - } - - const StreamSocket& socket() const - { - return _socket; - } + TCPConnectionNotification(const StreamSocket& socket): + _socket(socket) + { + } + + ~TCPConnectionNotification() + { + } + + const StreamSocket& socket() const + { + return _socket; + } private: - StreamSocket _socket; + StreamSocket _socket; }; TCPServerDispatcher::TCPServerDispatcher(TCPServerConnectionFactory::Ptr pFactory, Poco::ThreadPool& threadPool, TCPServerParams::Ptr pParams): - _rc(1), - _pParams(pParams), - _currentThreads(0), - _totalConnections(0), - _currentConnections(0), - _maxConcurrentConnections(0), - _refusedConnections(0), - _stopped(false), - _pConnectionFactory(pFactory), - _threadPool(threadPool) + _rc(1), + _pParams(pParams), + _currentThreads(0), + _totalConnections(0), + _currentConnections(0), + _maxConcurrentConnections(0), + _refusedConnections(0), + _stopped(false), + _pConnectionFactory(pFactory), + _threadPool(threadPool) { - poco_check_ptr (pFactory); + poco_check_ptr (pFactory); - if (!_pParams) - _pParams = new TCPServerParams; - - if (_pParams->getMaxThreads() == 0) - _pParams->setMaxThreads(threadPool.capacity()); + if (!_pParams) + _pParams = new TCPServerParams; + + if (_pParams->getMaxThreads() == 0) + _pParams->setMaxThreads(threadPool.capacity()); } @@ -81,161 +81,184 @@ TCPServerDispatcher::~TCPServerDispatcher() void TCPServerDispatcher::duplicate() { - ++_rc; + ++_rc; } void TCPServerDispatcher::release() { - if (--_rc == 0) delete this; + if (--_rc == 0) delete this; } void TCPServerDispatcher::run() { - AutoPtr guard(this); // ensure object stays alive + AutoPtr guard(this); // ensure object stays alive - int idleTime = (int) _pParams->getThreadIdleTime().totalMilliseconds(); + int idleTime = (int) _pParams->getThreadIdleTime().totalMilliseconds(); - for (;;) - { - try - { - AutoPtr pNf = _queue.waitDequeueNotification(idleTime); - if (pNf && !_stopped) - { - TCPConnectionNotification* pCNf = dynamic_cast(pNf.get()); - if (pCNf) - { - beginConnection(); - if (!_stopped) - { - std::unique_ptr pConnection(_pConnectionFactory->createConnection(pCNf->socket())); - poco_check_ptr(pConnection.get()); - pConnection->start(); - } - /// endConnection() should be called after destroying TCPServerConnection, - /// otherwise currentConnections() could become zero while some connections are yet still alive. - endConnection(); - } - } - } - catch (Poco::Exception &exc) { ErrorHandler::handle(exc); } - catch (std::exception &exc) { ErrorHandler::handle(exc); } - catch (...) { ErrorHandler::handle(); } - FastMutex::ScopedLock lock(_mutex); - if (_stopped || (_currentThreads > 1 && _queue.empty())) - { - --_currentThreads; - break; - } - } + for (;;) + { + try + { + AutoPtr pNf = _queue.waitDequeueNotification(idleTime); + if (pNf && !_stopped) + { + TCPConnectionNotification* pCNf = dynamic_cast(pNf.get()); + if (pCNf) + { + beginConnection(); + if (!_stopped) + { + std::unique_ptr pConnection(_pConnectionFactory->createConnection(pCNf->socket())); + poco_check_ptr(pConnection.get()); + pConnection->start(); + } + /// endConnection() should be called after destroying TCPServerConnection, + /// otherwise currentConnections() could become zero while some connections are yet still alive. + endConnection(); + } + } + } + catch (Poco::Exception &exc) { ErrorHandler::handle(exc); } + catch (std::exception &exc) { ErrorHandler::handle(exc); } + catch (...) { ErrorHandler::handle(); } + FastMutex::ScopedLock lock(_mutex); + if (_stopped || (_currentThreads > 1 && _queue.empty())) + { + --_currentThreads; + break; + } + } } namespace { - static const std::string threadName("TCPServerConnection"); + static const std::string threadName("TCPServerConnection"); } - + void TCPServerDispatcher::enqueue(const StreamSocket& socket) { - FastMutex::ScopedLock lock(_mutex); + FastMutex::ScopedLock lock(_mutex); - if (_queue.size() < _pParams->getMaxQueued()) - { - if (!_queue.hasIdleThreads() && _currentThreads < _pParams->getMaxThreads()) - { - try - { + ErrorHandler::logMessage(Message::PRIO_TEST, "Queue size: " + std::to_string(_queue.size()) + + ", current threads: " + std::to_string(_currentThreads) + + ", threads in pool: " + std::to_string(_threadPool.allocated()) + + ", current connections: " + std::to_string(_currentConnections)); + + + if (_queue.size() < _pParams->getMaxQueued()) + { + /// NOTE: the condition below is wrong. + /// Since the thread pool is shared between multiple servers/TCPServerDispatchers, + /// _currentThreads < _pParams->getMaxThreads() will be true when the pool is actually saturated. + /// As a result, queue is useless and connections never wait in queue. + /// Instead, we (mistakenly) think that we can create a thread for this connection, but we fail to create it + /// and the connection get rejected. + /// We could check _currentThreads < _threadPool.allocated() to make it work, + /// but it's not clear if we want to make it work + /// because it may be better to reject connection immediately if we don't have resources to handle it. + if (!_queue.hasIdleThreads() && _currentThreads < _pParams->getMaxThreads()) + { + try + { this->duplicate(); - _threadPool.startWithPriority(_pParams->getThreadPriority(), *this, threadName); - ++_currentThreads; - } - catch (Poco::Exception& exc) - { + _threadPool.startWithPriority(_pParams->getThreadPriority(), *this, threadName); + ++_currentThreads; + } + catch (Poco::Exception& exc) + { + ErrorHandler::logMessage(Message::PRIO_WARNING, "Got an exception while starting thread for connection from " + + socket.peerAddress().toString()); + ErrorHandler::handle(exc); this->release(); - ++_refusedConnections; - std::cerr << "Got exception while starting thread for connection. Error code: " - << exc.code() << ", message: '" << exc.displayText() << "'" << std::endl; - return; - } - } - _queue.enqueueNotification(new TCPConnectionNotification(socket)); - } - else - { - ++_refusedConnections; - } + ++_refusedConnections; + return; + } + } + else if (!_queue.hasIdleThreads()) + { + ErrorHandler::logMessage(Message::PRIO_TRACE, "Don't have idle threads, adding connection from " + + socket.peerAddress().toString() + " to the queue, size: " + std::to_string(_queue.size())); + } + _queue.enqueueNotification(new TCPConnectionNotification(socket)); + } + else + { + ErrorHandler::logMessage(Message::PRIO_WARNING, "Refusing connection from " + socket.peerAddress().toString() + + ", reached max queue size " + std::to_string(_pParams->getMaxQueued())); + ++_refusedConnections; + } } void TCPServerDispatcher::stop() { - _stopped = true; - _queue.clear(); - _queue.wakeUpAll(); + _stopped = true; + _queue.clear(); + _queue.wakeUpAll(); } int TCPServerDispatcher::currentThreads() const { - return _currentThreads; + return _currentThreads; } int TCPServerDispatcher::maxThreads() const { - FastMutex::ScopedLock lock(_mutex); - - return _threadPool.capacity(); + FastMutex::ScopedLock lock(_mutex); + + return _threadPool.capacity(); } int TCPServerDispatcher::totalConnections() const { - return _totalConnections; + return _totalConnections; } int TCPServerDispatcher::currentConnections() const { - return _currentConnections; + return _currentConnections; } int TCPServerDispatcher::maxConcurrentConnections() const { - return _maxConcurrentConnections; + return _maxConcurrentConnections; } int TCPServerDispatcher::queuedConnections() const { - return _queue.size(); + return _queue.size(); } int TCPServerDispatcher::refusedConnections() const { - return _refusedConnections; + return _refusedConnections; } void TCPServerDispatcher::beginConnection() { - FastMutex::ScopedLock lock(_mutex); + FastMutex::ScopedLock lock(_mutex); - ++_totalConnections; - ++_currentConnections; - if (_currentConnections > _maxConcurrentConnections) - _maxConcurrentConnections.store(_currentConnections); + ++_totalConnections; + ++_currentConnections; + if (_currentConnections > _maxConcurrentConnections) + _maxConcurrentConnections.store(_currentConnections); } void TCPServerDispatcher::endConnection() { - --_currentConnections; + --_currentConnections; } diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 44c2daa33ad..7ecfc513705 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -410,7 +410,7 @@ try std::lock_guard lock(servers_lock); metrics.reserve(servers->size()); for (const auto & server : *servers) - metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); + metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads(), server.refusedConnections()}); return metrics; } ); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 3b88bb36954..27d9e4f1394 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -909,10 +909,10 @@ try metrics.reserve(servers_to_start_before_tables.size() + servers.size()); for (const auto & server : servers_to_start_before_tables) - metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); + metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads(), server.refusedConnections()}); for (const auto & server : servers) - metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); + metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads(), server.refusedConnections()}); return metrics; } ); diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 6309f6079f6..56e7c4f3405 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -1613,7 +1613,7 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) #endif { - auto get_metric_name_doc = [](const String & name) -> std::pair + auto threads_get_metric_name_doc = [](const String & name) -> std::pair { static std::map> metric_map = { @@ -1637,11 +1637,38 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) return it->second; }; + auto rejected_connections_get_metric_name_doc = [](const String & name) -> std::pair + { + static std::map> metric_map = + { + {"tcp_port", {"TCPRejectedConnections", "Number of rejected connections for the TCP protocol (without TLS)."}}, + {"tcp_port_secure", {"TCPSecureRejectedConnections", "Number of rejected connections for the TCP protocol (with TLS)."}}, + {"http_port", {"HTTPRejectedConnections", "Number of rejected connections for the HTTP interface (without TLS)."}}, + {"https_port", {"HTTPSecureRejectedConnections", "Number of rejected connections for the HTTPS interface."}}, + {"interserver_http_port", {"InterserverRejectedConnections", "Number of rejected connections for the replicas communication protocol (without TLS)."}}, + {"interserver_https_port", {"InterserverSecureRejectedConnections", "Number of rejected connections for the replicas communication protocol (with TLS)."}}, + {"mysql_port", {"MySQLRejectedConnections", "Number of rejected connections for the MySQL compatibility protocol."}}, + {"postgresql_port", {"PostgreSQLRejectedConnections", "Number of rejected connections for the PostgreSQL compatibility protocol."}}, + {"grpc_port", {"GRPCRejectedConnections", "Number of rejected connections for the GRPC protocol."}}, + {"prometheus.port", {"PrometheusRejectedConnections", "Number of rejected connections for the Prometheus endpoint. Note: prometheus endpoints can be also used via the usual HTTP/HTTPs ports."}}, + {"keeper_server.tcp_port", {"KeeperTCPRejectedConnections", "Number of rejected connections for the Keeper TCP protocol (without TLS)."}}, + {"keeper_server.tcp_port_secure", {"KeeperTCPSecureRejectedConnections", "Number of rejected connections for the Keeper TCP protocol (with TLS)."}} + }; + auto it = metric_map.find(name); + if (it == metric_map.end()) + return { nullptr, nullptr }; + else + return it->second; + }; + const auto server_metrics = protocol_server_metrics_func(); for (const auto & server_metric : server_metrics) { - if (auto name_doc = get_metric_name_doc(server_metric.port_name); name_doc.first != nullptr) + if (auto name_doc = threads_get_metric_name_doc(server_metric.port_name); name_doc.first != nullptr) new_values[name_doc.first] = { server_metric.current_threads, name_doc.second }; + + if (auto name_doc = rejected_connections_get_metric_name_doc(server_metric.port_name); name_doc.first != nullptr) + new_values[name_doc.first] = { server_metric.rejected_connections, name_doc.second }; } } diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index 10a972d2458..04d0319e35b 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -42,6 +42,7 @@ struct ProtocolServerMetrics { String port_name; size_t current_threads; + size_t rejected_connections; }; /** Periodically (by default, each second) diff --git a/src/Common/ErrorHandlers.h b/src/Common/ErrorHandlers.h index a4a7c4683aa..4e7d391e66f 100644 --- a/src/Common/ErrorHandlers.h +++ b/src/Common/ErrorHandlers.h @@ -2,6 +2,7 @@ #include #include +#include /** ErrorHandler for Poco::Thread, @@ -26,8 +27,32 @@ public: void exception(const std::exception &) override { logException(); } void exception() override { logException(); } + void logMessageImpl(Poco::Message::Priority priority, const std::string & msg) override + { + switch (priority) + { + case Poco::Message::PRIO_FATAL: [[fallthrough]]; + case Poco::Message::PRIO_CRITICAL: + LOG_FATAL(trace_log, fmt::runtime(msg)); break; + case Poco::Message::PRIO_ERROR: + LOG_ERROR(trace_log, fmt::runtime(msg)); break; + case Poco::Message::PRIO_WARNING: + LOG_WARNING(trace_log, fmt::runtime(msg)); break; + case Poco::Message::PRIO_NOTICE: [[fallthrough]]; + case Poco::Message::PRIO_INFORMATION: + LOG_INFO(trace_log, fmt::runtime(msg)); break; + case Poco::Message::PRIO_DEBUG: + LOG_DEBUG(trace_log, fmt::runtime(msg)); break; + case Poco::Message::PRIO_TRACE: + LOG_TRACE(trace_log, fmt::runtime(msg)); break; + case Poco::Message::PRIO_TEST: + LOG_TEST(trace_log, fmt::runtime(msg)); break; + } + } + private: LoggerPtr log = getLogger("ServerErrorHandler"); + LoggerPtr trace_log = getLogger("Poco"); void logException() { diff --git a/src/Server/ProtocolServerAdapter.cpp b/src/Server/ProtocolServerAdapter.cpp index b41ad2376f1..6b723bc8d87 100644 --- a/src/Server/ProtocolServerAdapter.cpp +++ b/src/Server/ProtocolServerAdapter.cpp @@ -20,6 +20,7 @@ public: UInt16 portNumber() const override { return tcp_server->portNumber(); } size_t currentConnections() const override { return tcp_server->currentConnections(); } size_t currentThreads() const override { return tcp_server->currentThreads(); } + size_t refusedConnections() const override { return tcp_server->refusedConnections(); } private: std::unique_ptr tcp_server; @@ -54,6 +55,7 @@ public: UInt16 portNumber() const override { return grpc_server->portNumber(); } size_t currentConnections() const override { return grpc_server->currentConnections(); } size_t currentThreads() const override { return grpc_server->currentThreads(); } + size_t refusedConnections() const override { return 0; } private: std::unique_ptr grpc_server; diff --git a/src/Server/ProtocolServerAdapter.h b/src/Server/ProtocolServerAdapter.h index 76a6776ed9c..4a0b0cae8e7 100644 --- a/src/Server/ProtocolServerAdapter.h +++ b/src/Server/ProtocolServerAdapter.h @@ -38,6 +38,8 @@ public: /// Returns the number of currently handled connections. size_t currentConnections() const { return impl->currentConnections(); } + size_t refusedConnections() const { return impl->refusedConnections(); } + /// Returns the number of current threads. size_t currentThreads() const { return impl->currentThreads(); } @@ -61,6 +63,7 @@ private: virtual UInt16 portNumber() const = 0; virtual size_t currentConnections() const = 0; virtual size_t currentThreads() const = 0; + virtual size_t refusedConnections() const = 0; }; class TCPServerAdapterImpl; class GRPCServerAdapterImpl; 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 073/371] 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 074/371] 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 075/371] 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 076/371] 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 077/371] 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 078/371] 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 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 079/371] 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 080/371] 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 081/371] 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 082/371] 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 8aaf9c1d9824c136bbfc0532b040b2dca7564253 Mon Sep 17 00:00:00 2001 From: morning-color Date: Tue, 16 Jul 2024 11:00:55 +0800 Subject: [PATCH 083/371] 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 084/371] 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 085/371] 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 086/371] 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 087/371] 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 088/371] 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 0df6448e0f4b4b6d2acbb38466fbd34d979b4d90 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 16 Jul 2024 14:38:02 +0200 Subject: [PATCH 089/371] Update base/poco/Foundation/include/Poco/ErrorHandler.h Co-authored-by: Sergei Trifonov --- base/poco/Foundation/include/Poco/ErrorHandler.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/poco/Foundation/include/Poco/ErrorHandler.h b/base/poco/Foundation/include/Poco/ErrorHandler.h index 961fec2bc3b..f774f2ccf5e 100644 --- a/base/poco/Foundation/include/Poco/ErrorHandler.h +++ b/base/poco/Foundation/include/Poco/ErrorHandler.h @@ -93,7 +93,7 @@ public: /// Invokes the currently registered ErrorHandler. static void logMessage(Message::Priority priority, const std::string & msg); - /// Invokes the currently registered ErrorHandler. + /// Invokes the currently registered ErrorHandler to log a message. static ErrorHandler * set(ErrorHandler * pHandler); /// Registers the given handler as the current error handler. From f29700e04d3ca4d455908ed354472945597da4f5 Mon Sep 17 00:00:00 2001 From: morning-color Date: Wed, 17 Jul 2024 20:01:01 +0800 Subject: [PATCH 090/371] 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 e3a0b6ab5ff21518a494ebede1aea47edda22b6c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 11:45:52 +0200 Subject: [PATCH 091/371] 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 340214a246cd1c35d96cfb21be0576d87e05fea0 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 22 Jul 2024 12:55:50 +0200 Subject: [PATCH 092/371] Unpin docker-ce in integration-tests-runner --- docker/test/integration/runner/Dockerfile | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index d250b746e7d..ceb8a1b2b58 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -43,13 +43,11 @@ ENV TZ=Etc/UTC RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone ENV DOCKER_CHANNEL stable -# Unpin the docker version after the release 24.0.3 is released -# https://github.com/moby/moby/issues/45770#issuecomment-1618255130 RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - \ && add-apt-repository "deb https://download.docker.com/linux/ubuntu $(lsb_release -c -s) ${DOCKER_CHANNEL}" \ && apt-get update \ && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ - docker-ce='5:23.*' \ + docker-ce \ && rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ From d296e62bf363d7dfab9a5bf6925b67b5e4188151 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 22 Jul 2024 14:31:50 +0200 Subject: [PATCH 093/371] Update docker/test/integration/runner/Dockerfile --- docker/test/integration/runner/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index ceb8a1b2b58..71cf3a16967 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -47,7 +47,7 @@ RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - \ && add-apt-repository "deb https://download.docker.com/linux/ubuntu $(lsb_release -c -s) ${DOCKER_CHANNEL}" \ && apt-get update \ && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ - docker-ce \ + docker-ce="5:27.0.3*" \ && rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ 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 094/371] 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 095/371] 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 096/371] 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 9c28c64adf04cd03711a846a17babd683dc2c002 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 24 Jul 2024 19:55:03 +0000 Subject: [PATCH 097/371] Fix deprecated HDFS image and harden test_storage_hdfs. --- .../compose/docker_compose_hdfs.yml | 2 +- tests/integration/test_storage_hdfs/test.py | 134 +++++++++++------- 2 files changed, 87 insertions(+), 49 deletions(-) diff --git a/tests/integration/compose/docker_compose_hdfs.yml b/tests/integration/compose/docker_compose_hdfs.yml index 1cae54ad9e1..40a10df01f7 100644 --- a/tests/integration/compose/docker_compose_hdfs.yml +++ b/tests/integration/compose/docker_compose_hdfs.yml @@ -1,7 +1,7 @@ version: '2.3' services: hdfs1: - image: sequenceiq/hadoop-docker:2.7.0 + image: prasanthj/docker-hadoop:2.6.0 hostname: hdfs1 restart: always expose: diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 47d8f44c0b7..aaeb472dd52 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1,6 +1,7 @@ import os import pytest +import uuid import time from helpers.cluster import ClickHouseCluster, is_arm from helpers.test_tools import TSV @@ -31,13 +32,15 @@ def started_cluster(): def test_read_write_storage(started_cluster): + id = uuid.uuid4() hdfs_api = started_cluster.hdfs_api + filename = f"simple_storage_{id}" node1.query("drop table if exists SimpleHDFSStorage SYNC") node1.query( - "create table SimpleHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/simple_storage', 'TSV')" + f"create table SimpleHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/{filename}', 'TSV')" ) node1.query("insert into SimpleHDFSStorage values (1, 'Mark', 72.53)") - assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" + assert hdfs_api.read_data(f"/{filename}") == "1\tMark\t72.53\n" assert node1.query("select * from SimpleHDFSStorage") == "1\tMark\t72.53\n" @@ -92,6 +95,10 @@ 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 @@ -137,7 +144,6 @@ def test_read_write_table(started_cluster): def test_write_table(started_cluster): hdfs_api = started_cluster.hdfs_api - node1.query( "create table OtherHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/other_storage', 'TSV')" ) @@ -148,7 +154,8 @@ def test_write_table(started_cluster): result = "10\ttomas\t55.55\n11\tjack\t32.54\n" assert hdfs_api.read_data("/other_storage") == result assert node1.query("select * from OtherHDFSStorage order by id") == result - + node1.query("truncate table OtherHDFSStorage") + node1.query("drop table OtherHDFSStorage") def test_bad_hdfs_uri(started_cluster): try: @@ -166,6 +173,7 @@ def test_bad_hdfs_uri(started_cluster): print(ex) assert "Unable to connect to HDFS" in str(ex) + node1.query("drop table BadStorage2") try: node1.query( "create table BadStorage3 (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/<>', 'TSV')" @@ -173,6 +181,7 @@ def test_bad_hdfs_uri(started_cluster): except Exception as ex: print(ex) assert "Unable to open HDFS file" in str(ex) + node1.query("drop table BadStorage3") @pytest.mark.timeout(800) @@ -304,7 +313,8 @@ def test_write_gz_storage(started_cluster): node1.query("insert into GZHDFSStorage values (1, 'Mark', 72.53)") assert hdfs_api.read_gzip_data("/storage.gz") == "1\tMark\t72.53\n" assert node1.query("select * from GZHDFSStorage") == "1\tMark\t72.53\n" - + node1.query("truncate table GZHDFSStorage") + node1.query("drop table GZHDFSStorage") def test_write_gzip_storage(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -315,6 +325,8 @@ def test_write_gzip_storage(started_cluster): node1.query("insert into GZIPHDFSStorage values (1, 'Mark', 72.53)") assert hdfs_api.read_gzip_data("/gzip_storage") == "1\tMark\t72.53\n" assert node1.query("select * from GZIPHDFSStorage") == "1\tMark\t72.53\n" + node1.query("truncate table GZIPHDFSStorage") + node1.query("drop table GZIPHDFSStorage") def test_virtual_columns(started_cluster): @@ -333,7 +345,7 @@ def test_virtual_columns(started_cluster): ) == expected ) - + node1.query("drop table virtual_cols") def test_read_files_with_spaces(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -354,6 +366,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(f"drop table test") def test_truncate_table(started_cluster): @@ -375,47 +388,52 @@ def test_truncate_table(started_cluster): def test_partition_by(started_cluster): - hdfs_api = started_cluster.hdfs_api - + fs = HdfsClient(hosts=started_cluster.hdfs_ip) + id = uuid.uuid4() table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + dir = f"partition_{id}" + fs.mkdirs(f"/{dir}/", permission=777) + file_name = "test_{_partition_id}" partition_by = "column3" values = "(1, 2, 3), (3, 2, 1), (1, 3, 2)" - table_function = f"hdfs('hdfs://hdfs1:9000/{file_name}', 'TSV', '{table_format}')" + table_function = f"hdfs('hdfs://hdfs1:9000/{dir}/{file_name}', 'TSV', '{table_format}')" node1.query( f"insert into table function {table_function} PARTITION BY {partition_by} values {values}" ) result = node1.query( - f"select * from hdfs('hdfs://hdfs1:9000/test_1', 'TSV', '{table_format}')" + f"select * from hdfs('hdfs://hdfs1:9000/{dir}/test_1', 'TSV', '{table_format}')" ) assert result.strip() == "3\t2\t1" result = node1.query( - f"select * from hdfs('hdfs://hdfs1:9000/test_2', 'TSV', '{table_format}')" + f"select * from hdfs('hdfs://hdfs1:9000/{dir}/test_2', 'TSV', '{table_format}')" ) assert result.strip() == "1\t3\t2" result = node1.query( - f"select * from hdfs('hdfs://hdfs1:9000/test_3', 'TSV', '{table_format}')" + f"select * from hdfs('hdfs://hdfs1:9000/{dir}/test_3', 'TSV', '{table_format}')" ) assert result.strip() == "1\t2\t3" file_name = "test2_{_partition_id}" node1.query( - f"create table p(column1 UInt32, column2 UInt32, column3 UInt32) engine = HDFS('hdfs://hdfs1:9000/{file_name}', 'TSV') partition by column3" + f"create table p(column1 UInt32, column2 UInt32, column3 UInt32) engine = HDFS('hdfs://hdfs1:9000/{dir}/{file_name}', 'TSV') partition by column3" ) node1.query(f"insert into p values {values}") result = node1.query( - f"select * from hdfs('hdfs://hdfs1:9000/test2_1', 'TSV', '{table_format}')" + f"select * from hdfs('hdfs://hdfs1:9000/{dir}/test2_1', 'TSV', '{table_format}')" ) assert result.strip() == "3\t2\t1" result = node1.query( - f"select * from hdfs('hdfs://hdfs1:9000/test2_2', 'TSV', '{table_format}')" + f"select * from hdfs('hdfs://hdfs1:9000/{dir}/test2_2', 'TSV', '{table_format}')" ) assert result.strip() == "1\t3\t2" result = node1.query( - f"select * from hdfs('hdfs://hdfs1:9000/test2_3', 'TSV', '{table_format}')" + f"select * from hdfs('hdfs://hdfs1:9000/{dir}/test2_3', 'TSV', '{table_format}')" ) assert result.strip() == "1\t2\t3" + node1.query(f"drop table p") + fs.delete("/{dir}", recursive=True) def test_seekable_formats(started_cluster): @@ -425,7 +443,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}") @@ -433,7 +451,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 @@ -457,7 +475,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')") @@ -476,7 +494,7 @@ def test_schema_inference(started_cluster): result = node1.query(f"select count(*) from schema_inference") assert int(result) == 5000000 - + node1.query(f"drop table schema_inference") def test_hdfsCluster(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -510,6 +528,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): @@ -529,12 +548,16 @@ def test_overwrite(started_cluster): result = node1.query(f"select count() from test_overwrite") assert int(result) == 10 + node1.query(f"truncate table test_overwrite") + node1.query(f"drop table test_overwrite") def test_multiple_inserts(started_cluster): - hdfs_api = started_cluster.hdfs_api + fs = HdfsClient(hosts=started_cluster.hdfs_ip) + id = uuid.uuid4() + fs.mkdirs(f"/{id}/", permission=777) - table_function = f"hdfs('hdfs://hdfs1:9000/data_multiple_inserts', 'Parquet', 'a Int32, b String')" + table_function = f"hdfs('hdfs://hdfs1:9000/{id}/data_multiple_inserts', 'Parquet', 'a Int32, b String')" node1.query(f"create table test_multiple_inserts as {table_function}") node1.query( f"insert into test_multiple_inserts select number, randomString(100) from numbers(10)" @@ -551,7 +574,7 @@ def test_multiple_inserts(started_cluster): result = node1.query(f"drop table test_multiple_inserts") - table_function = f"hdfs('hdfs://hdfs1:9000/data_multiple_inserts.gz', 'Parquet', 'a Int32, b String')" + table_function = f"hdfs('hdfs://hdfs1:9000/{id}/data_multiple_inserts.gz', 'Parquet', 'a Int32, b String')" node1.query(f"create table test_multiple_inserts as {table_function}") node1.query( f"insert into test_multiple_inserts select number, randomString(100) FROM numbers(10)" @@ -565,7 +588,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): node1.query( @@ -574,6 +597,8 @@ def test_format_detection(started_cluster): node1.query(f"insert into arrow_table select 1") result = node1.query(f"select * from hdfs('hdfs://hdfs1:9000/data.arrow')") assert int(result) == 1 + node1.query(f"truncate table arrow_table") + node1.query(f"drop table arrow_table") def test_schema_inference_with_globs(started_cluster): @@ -618,6 +643,8 @@ def test_schema_inference_with_globs(started_cluster): def test_insert_select_schema_inference(started_cluster): + fs = HdfsClient(hosts=started_cluster.hdfs_ip) + node1.query( f"insert into table function hdfs('hdfs://hdfs1:9000/test.native.zst') select toUInt64(1) as x" ) @@ -627,6 +654,7 @@ def test_insert_select_schema_inference(started_cluster): result = node1.query(f"select * from hdfs('hdfs://hdfs1:9000/test.native.zst')") assert int(result) == 1 + fs.delete('/test.native.zst') def test_cluster_join(started_cluster): @@ -967,11 +995,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( @@ -1003,7 +1031,7 @@ def test_read_subcolumn_time(started_cluster): node = started_cluster.instances["node1"] node.query( - f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumn_time.tsv', auto, 'a UInt32') select (42)" + f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumn_time.tsv', auto, 'a UInt32') select (42) settings hdfs_truncate_on_insert=1" ) res = node.query( @@ -1014,91 +1042,101 @@ def test_read_subcolumn_time(started_cluster): def test_union_schema_inference_mode(started_cluster): + id = uuid.uuid4() + fs = HdfsClient(hosts=started_cluster.hdfs_ip) + + dir = f"union_{id}" + fs.mkdirs(f"/{dir}/", permission=777) + node = started_cluster.instances["node1"] node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference1.jsonl') select 1 as a" + f"insert into function hdfs('hdfs://hdfs1:9000/{dir}/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" + f"insert into function hdfs('hdfs://hdfs1:9000/{dir}/test_union_schema_inference2.jsonl') select 2 as b" ) node.query("system drop schema cache for hdfs") result = node.query( - "desc hdfs('hdfs://hdfs1:9000/test_union_schema_inference*.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + f"desc hdfs('hdfs://hdfs1:9000/{dir}/test_union_schema_inference*.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" ) assert result == "a\tNullable(Int64)\nb\tNullable(Int64)\n" result = node.query( - "select schema_inference_mode, splitByChar('/', source)[-1] as file, schema from system.schema_inference_cache where source like '%test_union_schema_inference%' order by file format TSV" + f"select schema_inference_mode, splitByChar('/', source)[-1] as file, schema from system.schema_inference_cache where source like '%test_union_schema_inference%' order by file format TSV" ) assert ( result == "UNION\ttest_union_schema_inference1.jsonl\ta Nullable(Int64)\n" "UNION\ttest_union_schema_inference2.jsonl\tb Nullable(Int64)\n" ) result = node.query( - "select * from hdfs('hdfs://hdfs1:9000/test_union_schema_inference*.jsonl') order by tuple(*) settings schema_inference_mode='union', describe_compact_output=1 format TSV" + f"select * from hdfs('hdfs://hdfs1:9000/{dir}/test_union_schema_inference*.jsonl') order by tuple(*) settings schema_inference_mode='union', describe_compact_output=1 format TSV" ) assert result == "1\t\\N\n" "\\N\t2\n" node.query(f"system drop schema cache for hdfs") result = node.query( - "desc hdfs('hdfs://hdfs1:9000/test_union_schema_inference2.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + f"desc hdfs('hdfs://hdfs1:9000/{dir}/test_union_schema_inference2.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" ) assert result == "b\tNullable(Int64)\n" result = node.query( - "desc hdfs('hdfs://hdfs1:9000/test_union_schema_inference*.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + f"desc hdfs('hdfs://hdfs1:9000/{dir}/test_union_schema_inference*.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" ) 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/{dir}/test_union_schema_inference3.jsonl', TSV) select 'Error'" ) error = node.query_and_get_error( - "desc hdfs('hdfs://hdfs1:9000/test_union_schema_inference*.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + f"desc hdfs('hdfs://hdfs1:9000/{dir}/test_union_schema_inference*.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" ) assert "CANNOT_EXTRACT_TABLE_STRUCTURE" in error def test_format_detection(started_cluster): node = started_cluster.instances["node1"] + fs = HdfsClient(hosts=started_cluster.hdfs_ip) + id = uuid.uuid4() + dir = f"{id}" + fs.mkdirs(f"/{dir}/", permission=777) 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)" + f"insert into function hdfs('hdfs://hdfs1:9000/{dir}/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)" + f"insert into function hdfs('hdfs://hdfs1:9000/{dir}/test_format_detection1', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(10)" ) expected_desc_result = node.query( - "desc hdfs('hdfs://hdfs1:9000/test_format_detection1', JSONEachRow)" + f"desc hdfs('hdfs://hdfs1:9000/{dir}/test_format_detection1', JSONEachRow)" ) - desc_result = node.query("desc hdfs('hdfs://hdfs1:9000/test_format_detection1')") + desc_result = node.query(f"desc hdfs('hdfs://hdfs1:9000/{dir}/test_format_detection1')") assert expected_desc_result == desc_result expected_result = node.query( - "select * from hdfs('hdfs://hdfs1:9000/test_format_detection1', JSONEachRow, 'x UInt64, y String') order by x, y" + f"select * from hdfs('hdfs://hdfs1:9000/{dir}/test_format_detection1', JSONEachRow, 'x UInt64, y String') order by x, y" ) result = node.query( - "select * from hdfs('hdfs://hdfs1:9000/test_format_detection1') order by x, y" + f"select * from hdfs('hdfs://hdfs1:9000/{dir}/test_format_detection1') order by x, y" ) assert expected_result == result result = node.query( - "select * from hdfs('hdfs://hdfs1:9000/test_format_detection1', auto, 'x UInt64, y String') order by x, y" + f"select * from hdfs('hdfs://hdfs1:9000/{dir}/test_format_detection1', auto, 'x UInt64, y String') order by x, y" ) assert expected_result == result result = node.query( - "select * from hdfs('hdfs://hdfs1:9000/test_format_detection{0,1}') order by x, y" + f"select * from hdfs('hdfs://hdfs1:9000/{dir}/test_format_detection{{0,1}}') order by x, y" ) assert expected_result == result @@ -1106,25 +1144,25 @@ def test_format_detection(started_cluster): node.query("system drop schema cache for hdfs") result = node.query( - "select * from hdfs('hdfs://hdfs1:9000/test_format_detection{0,1}') order by x, y" + f"select * from hdfs('hdfs://hdfs1:9000/{dir}/test_format_detection{{0,1}}') order by x, y" ) assert expected_result == result result = node.query( - "select * from hdfsCluster(test_cluster_two_shards, 'hdfs://hdfs1:9000/test_format_detection{0,1}') order by x, y" + f"select * from hdfsCluster(test_cluster_two_shards, 'hdfs://hdfs1:9000/{dir}/test_format_detection{{0,1}}') order by x, y" ) assert expected_result == result result = node.query( - "select * from hdfsCluster(test_cluster_two_shards, 'hdfs://hdfs1:9000/test_format_detection{0,1}', auto, auto) order by x, y" + f"select * from hdfsCluster(test_cluster_two_shards, 'hdfs://hdfs1:9000/{dir}/test_format_detection{{0,1}}', auto, auto) order by x, y" ) assert expected_result == result result = node.query( - "select * from hdfsCluster(test_cluster_two_shards, 'hdfs://hdfs1:9000/test_format_detection{0,1}', auto, 'x UInt64, y String') order by x, y" + f"select * from hdfsCluster(test_cluster_two_shards, 'hdfs://hdfs1:9000/{dir}/test_format_detection{{0,1}}', auto, 'x UInt64, y String') order by x, y" ) assert expected_result == result From 34c8351bce754740a18388dd623309dfb6030bb0 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 24 Jul 2024 19:55:21 +0000 Subject: [PATCH 098/371] black --- tests/integration/test_storage_hdfs/test.py | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index aaeb472dd52..4aac0142026 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -100,6 +100,7 @@ def test_read_write_storage_with_globs(started_cluster): node1.query("drop table HDFSStorageWithQuestionMark") node1.query("drop table HDFSStorageWithAsterisk") + def test_storage_with_multidirectory_glob(started_cluster): hdfs_api = started_cluster.hdfs_api for i in ["1", "2"]: @@ -157,6 +158,7 @@ def test_write_table(started_cluster): node1.query("truncate table OtherHDFSStorage") node1.query("drop table OtherHDFSStorage") + def test_bad_hdfs_uri(started_cluster): try: node1.query( @@ -316,6 +318,7 @@ def test_write_gz_storage(started_cluster): node1.query("truncate table GZHDFSStorage") node1.query("drop table GZHDFSStorage") + def test_write_gzip_storage(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -347,6 +350,7 @@ def test_virtual_columns(started_cluster): ) node1.query("drop table virtual_cols") + def test_read_files_with_spaces(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -397,7 +401,9 @@ def test_partition_by(started_cluster): file_name = "test_{_partition_id}" partition_by = "column3" values = "(1, 2, 3), (3, 2, 1), (1, 3, 2)" - table_function = f"hdfs('hdfs://hdfs1:9000/{dir}/{file_name}', 'TSV', '{table_format}')" + table_function = ( + f"hdfs('hdfs://hdfs1:9000/{dir}/{file_name}', 'TSV', '{table_format}')" + ) node1.query( f"insert into table function {table_function} PARTITION BY {partition_by} values {values}" @@ -496,6 +502,7 @@ def test_schema_inference(started_cluster): assert int(result) == 5000000 node1.query(f"drop table schema_inference") + def test_hdfsCluster(started_cluster): hdfs_api = started_cluster.hdfs_api fs = HdfsClient(hosts=started_cluster.hdfs_ip) @@ -590,6 +597,7 @@ def test_multiple_inserts(started_cluster): assert int(result) == 60 node1.query(f"drop table test_multiple_inserts") + def test_format_detection(started_cluster): node1.query( f"create table arrow_table (x UInt64) engine=HDFS('hdfs://hdfs1:9000/data.arrow')" @@ -654,7 +662,7 @@ def test_insert_select_schema_inference(started_cluster): result = node1.query(f"select * from hdfs('hdfs://hdfs1:9000/test.native.zst')") assert int(result) == 1 - fs.delete('/test.native.zst') + fs.delete("/test.native.zst") def test_cluster_join(started_cluster): @@ -1115,7 +1123,9 @@ def test_format_detection(started_cluster): f"desc hdfs('hdfs://hdfs1:9000/{dir}/test_format_detection1', JSONEachRow)" ) - desc_result = node.query(f"desc hdfs('hdfs://hdfs1:9000/{dir}/test_format_detection1')") + desc_result = node.query( + f"desc hdfs('hdfs://hdfs1:9000/{dir}/test_format_detection1')" + ) assert expected_desc_result == desc_result From 3cbb3dc55f6582bc8abc7d5683080702080adcd8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 26 Jul 2024 08:41:44 +0000 Subject: [PATCH 099/371] 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 100/371] 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 d06b9ca99669ed3cfdf1beb68c9fbc1b90180536 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 26 Jul 2024 14:47:38 +0200 Subject: [PATCH 101/371] Ping CI From 0d8a3f13e39c9e0be87c8459db9582c7c336617f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 26 Jul 2024 21:49:41 +0100 Subject: [PATCH 102/371] 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 103/371] 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 104/371] 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 6317979825794882905bc02b3a18dd82cfd8ec1c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 Jul 2024 20:53:11 +0100 Subject: [PATCH 105/371] 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 106/371] 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 bd83ba88b03b7de15fbc7530d6c827156674f3b5 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 30 Jul 2024 18:48:34 +0000 Subject: [PATCH 107/371] 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 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 108/371] 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 109/371] 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 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 110/371] 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 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 111/371] 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 112/371] 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 113/371] 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 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 114/371] 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 115/371] 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 116/371] 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 117/371] 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 a80c7c080cdbb3bec4662501686133298b7a4a2d Mon Sep 17 00:00:00 2001 From: Jacob Reckhard Date: Thu, 1 Aug 2024 18:23:11 -0600 Subject: [PATCH 118/371] 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 119/371] 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 120/371] 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 0ec292a65f190f69c24420d5ca85d5658bffba0a Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 2 Aug 2024 00:32:01 -0400 Subject: [PATCH 121/371] 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 1c533f714529dd3065cfcdb0d69e5bbd28f51c29 Mon Sep 17 00:00:00 2001 From: skyoct Date: Fri, 2 Aug 2024 18:32:43 +0800 Subject: [PATCH 122/371] CI From 51b39a6c745d61cb2e6feb39659ddb3cac57ad03 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 1 Aug 2024 18:24:45 +0100 Subject: [PATCH 123/371] 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 08bde9cb44dce6fdf069527852faa8ec29a71b10 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 2 Aug 2024 18:28:33 -0400 Subject: [PATCH 124/371] 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 125/371] 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 8b7a294bc7db64f675faaf9b91a40e3e9f613935 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 30 May 2024 17:15:29 +0200 Subject: [PATCH 126/371] 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 127/371] 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 128/371] 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 129/371] 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 130/371] 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 131/371] 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 132/371] 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 133/371] 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 134/371] 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 135/371] 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 136/371] 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 137/371] 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 138/371] 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 139/371] 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 140/371] 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 141/371] 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 142/371] 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 143/371] 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 b2ec479cee2a07f7e1fe0a8384a36b5ffc999fda Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Sun, 4 Aug 2024 22:23:08 +0000 Subject: [PATCH 144/371] Fixed race condition in session log tests --- tests/integration/test_session_log/test.py | 19 +++++++++++++++---- .../0_stateless/02834_remote_session_log.sh | 10 ++++++++++ 2 files changed, 25 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py index 922e2557c50..cf2d0e62e1f 100644 --- a/tests/integration/test_session_log/test.py +++ b/tests/integration/test_session_log/test.py @@ -5,6 +5,7 @@ import pytest import random import sys import threading +import time from helpers.cluster import ClickHouseCluster, run_and_check @@ -117,6 +118,19 @@ def mysql_query(query, user_, pass_, raise_exception): assert raise_exception +def wait_for_corresponding_login_succecss_and_logout(user, expected_login_count): + # The client can exit sooner than the server records its disconnection and closes the session. + # When the client disconnects, two processes happen at the same time and are in the race condition: + # - the client application exits and returns control to the shell; + # - the server closes the session and records the logout event to the session log. + # We cannot expect that after the control is returned to the shell, the server records the logout event. + sql = f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '{user}' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '{user}' AND type = 'Logout')" + logins_and_logouts = instance.query(sql) + while int(logins_and_logouts) != expected_login_count: + time.sleep(0.1) + logins_and_logouts = instance.query(sql) + + @pytest.fixture(scope="module") def started_cluster(): try: @@ -276,10 +290,7 @@ def test_parallel_sessions(started_cluster): ) assert postgres_sessions == "30\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "30\n" + wait_for_corresponding_login_succecss_and_logout("parallel_user", 30) logout_failure_sessions = instance.query( f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'" diff --git a/tests/queries/0_stateless/02834_remote_session_log.sh b/tests/queries/0_stateless/02834_remote_session_log.sh index 3bedfb6c9ee..0581cb36136 100755 --- a/tests/queries/0_stateless/02834_remote_session_log.sh +++ b/tests/queries/0_stateless/02834_remote_session_log.sh @@ -44,6 +44,16 @@ for interface in 'TCP' 'HTTP' 'MySQL' do LOGIN_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}'"` CORRESPONDING_LOGOUT_RECORDS_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}')"` + # The client can exit sooner than the server records its disconnection and closes the session. + # When the client disconnects, two processes happen at the same time and are in the race condition: + # - the client application exits and returns control to the shell; + # - the server closes the session and records the logout event to the session log. + # We cannot expect that after the control is returned to the shell, the server records the logout event. + while [ "$LOGIN_COUNT" != "$CORRESPONDING_LOGOUT_RECORDS_COUNT" ] + do + sleep 0.1 + CORRESPONDING_LOGOUT_RECORDS_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}')"` + done if [ "$LOGIN_COUNT" == "$CORRESPONDING_LOGOUT_RECORDS_COUNT" ]; then echo "${interface} Login and logout count is equal" From 6897178c41286fd3d4f3064ad8e2c716d12e002f Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Sun, 4 Aug 2024 22:29:03 +0000 Subject: [PATCH 145/371] fixed typo --- tests/integration/test_session_log/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py index cf2d0e62e1f..5e424610ba2 100644 --- a/tests/integration/test_session_log/test.py +++ b/tests/integration/test_session_log/test.py @@ -118,7 +118,7 @@ def mysql_query(query, user_, pass_, raise_exception): assert raise_exception -def wait_for_corresponding_login_succecss_and_logout(user, expected_login_count): +def wait_for_corresponding_login_success_and_logout(user, expected_login_count): # The client can exit sooner than the server records its disconnection and closes the session. # When the client disconnects, two processes happen at the same time and are in the race condition: # - the client application exits and returns control to the shell; @@ -290,7 +290,7 @@ def test_parallel_sessions(started_cluster): ) assert postgres_sessions == "30\n" - wait_for_corresponding_login_succecss_and_logout("parallel_user", 30) + wait_for_corresponding_login_success_and_logout("parallel_user", 30) logout_failure_sessions = instance.query( f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'" From e5134e14ea68ff4d02bde892a7c66d00d4c1e800 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Mon, 5 Aug 2024 07:59:51 +0000 Subject: [PATCH 146/371] Disabled parallel run --- tests/ci/integration_tests_runner.py | 4 ++-- tests/integration/parallel_skip.json | 5 +++++ 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 2b348be8b51..22c52521c19 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -784,8 +784,8 @@ class ClickhouseIntegrationTestsRunner: logging.info("Starting check with retries") final_retry = 0 logs = [] - tires_num = 1 if should_fail else FLAKY_TRIES_COUNT - for i in range(tires_num): + tries_num = 1 if should_fail else FLAKY_TRIES_COUNT + for i in range(tries_num): final_retry += 1 logging.info("Running tests for the %s time", i) counters, tests_times, log_paths = self.try_run_test_group( diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 99fa626bd1e..9b8109f3f17 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -94,6 +94,11 @@ "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_setting_in_query", "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_client_suggestions_load", + "test_session_log/test.py::test_grpc_session", + "test_session_log/test.py::test_mysql_session", + "test_session_log/test.py::test_postgres_session", + "test_session_log/test.py::test_parallel_sessions", + "test_ttl_move/test.py::TestCancelBackgroundMoving::test_cancel_background_moving_on_stop_moves_query", "test_ttl_move/test.py::TestCancelBackgroundMoving::test_cancel_background_moving_on_table_detach", "test_ttl_move/test.py::TestCancelBackgroundMoving::test_cancel_background_moving_on_zookeeper_disconnect", From 29f1d9df36d34fe417624b0828b68a819ed74377 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Mon, 5 Aug 2024 10:03:49 +0200 Subject: [PATCH 147/371] Integration tests: fix flaky test_dictionaries_update_and_reload::test_reload_after_fail_by_timer --- .../test_dictionaries_update_and_reload/test.py | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_dictionaries_update_and_reload/test.py b/tests/integration/test_dictionaries_update_and_reload/test.py index db1c8e47467..32a9e1a033e 100644 --- a/tests/integration/test_dictionaries_update_and_reload/test.py +++ b/tests/integration/test_dictionaries_update_and_reload/test.py @@ -37,7 +37,7 @@ def get_status(dictionary_name): ).rstrip("\n") -def get_status_retry(dictionary_name, expect, retry_count=10, sleep_time=0.5): +def get_status_retry(dictionary_name, expect, retry_count=50, sleep_time=0.5): for _ in range(retry_count): res = get_status(dictionary_name) if res == expect: @@ -284,6 +284,11 @@ def test_reload_after_fail_by_timer(started_cluster): ) instance.query("SYSTEM RELOAD DICTIONARY no_file_2") instance.query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n" + if ( + instance.is_built_with_sanitizer() + and get_status("no_file_2") == "LOADED_AND_RELOADING" + ): + get_status_retry("no_file_2", expect="LOADED") assert get_status("no_file_2") == "LOADED" # Removing the file source should not spoil the loaded dictionary. @@ -292,6 +297,11 @@ def test_reload_after_fail_by_timer(started_cluster): ) time.sleep(6) instance.query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n" + if ( + instance.is_built_with_sanitizer() + and get_status("no_file_2") == "LOADED_AND_RELOADING" + ): + get_status_retry("no_file_2", expect="LOADED") assert get_status("no_file_2") == "LOADED" From bd6c7a504d214db606b7ba41bd3f9e172df8ae68 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Mon, 5 Aug 2024 11:45:49 +0200 Subject: [PATCH 148/371] Integration tests: fix flaky test_dictionaries_update_and_reload::test_reload_after_fail_by_timer --- .../test.py | 32 ++----------------- 1 file changed, 3 insertions(+), 29 deletions(-) diff --git a/tests/integration/test_dictionaries_update_and_reload/test.py b/tests/integration/test_dictionaries_update_and_reload/test.py index 32a9e1a033e..3ed854d2c9f 100644 --- a/tests/integration/test_dictionaries_update_and_reload/test.py +++ b/tests/integration/test_dictionaries_update_and_reload/test.py @@ -37,16 +37,6 @@ def get_status(dictionary_name): ).rstrip("\n") -def get_status_retry(dictionary_name, expect, retry_count=50, sleep_time=0.5): - for _ in range(retry_count): - res = get_status(dictionary_name) - if res == expect: - return res - time.sleep(sleep_time) - - raise Exception(f'Expected result "{expect}" did not occur') - - def get_last_exception(dictionary_name): return ( instance.query( @@ -263,13 +253,7 @@ def test_reload_after_fail_by_timer(started_cluster): # on sanitizers builds it can return 'FAILED_AND_RELOADING' which is not quite right # add retry for these builds - if ( - instance.is_built_with_sanitizer() - and get_status("no_file_2") == "FAILED_AND_RELOADING" - ): - get_status_retry("no_file_2", expect="FAILED") - - assert get_status("no_file_2") == "FAILED" + assert get_status("no_file_2") in ["FAILED", "FAILED_AND_RELOADING"] # Creating the file source makes the dictionary able to load. instance.copy_file_to_container( @@ -284,12 +268,7 @@ def test_reload_after_fail_by_timer(started_cluster): ) instance.query("SYSTEM RELOAD DICTIONARY no_file_2") instance.query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n" - if ( - instance.is_built_with_sanitizer() - and get_status("no_file_2") == "LOADED_AND_RELOADING" - ): - get_status_retry("no_file_2", expect="LOADED") - assert get_status("no_file_2") == "LOADED" + assert get_status("no_file_2") in ["LOADED", "LOADED_AND_RELOADING"] # Removing the file source should not spoil the loaded dictionary. instance.exec_in_container( @@ -297,12 +276,7 @@ def test_reload_after_fail_by_timer(started_cluster): ) time.sleep(6) instance.query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n" - if ( - instance.is_built_with_sanitizer() - and get_status("no_file_2") == "LOADED_AND_RELOADING" - ): - get_status_retry("no_file_2", expect="LOADED") - assert get_status("no_file_2") == "LOADED" + assert get_status("no_file_2") in ["LOADED", "LOADED_AND_RELOADING"] def test_reload_after_fail_in_cache_dictionary(started_cluster): 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 149/371] 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 150/371] 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 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 151/371] 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 152/371] 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 153/371] 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 154/371] 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 3e633ad0d0c579ca65467e35dd80504a92e3b7a4 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 5 Aug 2024 14:43:28 +0000 Subject: [PATCH 155/371] more changes --- .../test_access_control_on_cluster/test.py | 2 ++ .../integration/test_access_for_functions/test.py | 11 ++++++++--- .../test_alter_comment_on_cluster/test.py | 1 + .../integration/test_alter_moving_garbage/test.py | 14 ++++++++++++++ .../test_alter_on_mixed_type_cluster/test.py | 6 ++++++ 5 files changed, 31 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_access_control_on_cluster/test.py b/tests/integration/test_access_control_on_cluster/test.py index b12add7ad3f..c292d0cc3a4 100644 --- a/tests/integration/test_access_control_on_cluster/test.py +++ b/tests/integration/test_access_control_on_cluster/test.py @@ -89,3 +89,5 @@ def test_grant_current_database_on_cluster(): assert ch1.query("SHOW DATABASES", user="test_user") == "user_db\n" ch1.query("GRANT SELECT ON * TO test_user ON CLUSTER 'cluster'", user="test_user") assert ch1.query("SHOW DATABASES", user="test_user") == "user_db\n" + ch1.query("DROP DATABASE user_db ON CLUSTER 'cluster'") + ch1.query("DROP USER test_user ON CLUSTER 'cluster'") diff --git a/tests/integration/test_access_for_functions/test.py b/tests/integration/test_access_for_functions/test.py index 004d39e1dea..52777c60729 100644 --- a/tests/integration/test_access_for_functions/test.py +++ b/tests/integration/test_access_for_functions/test.py @@ -65,8 +65,8 @@ def test_ignore_obsolete_grant_on_database(): "-c", f""" cat > /var/lib/clickhouse/access/{user_id}.sql << EOF -ATTACH USER X; -ATTACH GRANT CREATE FUNCTION, SELECT ON mydb.* TO X; +ATTACH USER \`{user_id}\`; +ATTACH GRANT CREATE FUNCTION, SELECT ON mydb.* TO \`{user_id}\`; EOF""", ] ) @@ -76,4 +76,9 @@ EOF""", ) instance.start_clickhouse() - assert instance.query("SHOW GRANTS FOR X") == "GRANT SELECT ON mydb.* TO X\n" + assert ( + instance.query(f"SHOW GRANTS FOR `{user_id}`") + == f"GRANT SELECT ON mydb.* TO `{user_id}`\n" + ) + instance.stop_clickhouse() + instance.start_clickhouse() diff --git a/tests/integration/test_alter_comment_on_cluster/test.py b/tests/integration/test_alter_comment_on_cluster/test.py index e6767e35c1b..4cb10bbc751 100644 --- a/tests/integration/test_alter_comment_on_cluster/test.py +++ b/tests/integration/test_alter_comment_on_cluster/test.py @@ -59,3 +59,4 @@ def test_comment(started_cluster): expected = "CREATE TABLE default.test_table (`id` Int64 COMMENT \\'column_comment_2\\') ENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\') ORDER BY id SETTINGS index_granularity = 8192 COMMENT \\'table_comment_2\\'" assert_create_query([node_1, node_2], "default", "test_table", expected) + node_1.query("DROP TABLE test_table ON CLUSTER 'cluster'") diff --git a/tests/integration/test_alter_moving_garbage/test.py b/tests/integration/test_alter_moving_garbage/test.py index 21be46a7e1b..76b40b0df8d 100644 --- a/tests/integration/test_alter_moving_garbage/test.py +++ b/tests/integration/test_alter_moving_garbage/test.py @@ -36,6 +36,16 @@ def cluster(): cluster.shutdown() +def drop_table(node, table_name, replicated): + + create_table_statement = f"DROP TABLE {table_name} SYNC" + + if replicated: + node.query_with_retry(create_table_statement) + else: + node.query(create_table_statement) + + def create_table(node, table_name, replicated, additional_settings): settings = { "storage_policy": "two_disks", @@ -158,6 +168,9 @@ def test_alter_moving( assert data_digest == "1000\n" + for node in nodes: + drop_table(node, table_name, replicated_engine) + def test_delete_race_leftovers(cluster): """ @@ -248,3 +261,4 @@ def test_delete_race_leftovers(cluster): # Check that we have all data assert table_digest == node.query(table_digest_query) + drop_table(node, table_name, replicated=True) diff --git a/tests/integration/test_alter_on_mixed_type_cluster/test.py b/tests/integration/test_alter_on_mixed_type_cluster/test.py index f21a97d40e1..da126e307a0 100644 --- a/tests/integration/test_alter_on_mixed_type_cluster/test.py +++ b/tests/integration/test_alter_on_mixed_type_cluster/test.py @@ -88,6 +88,9 @@ def test_alter_on_cluter_non_replicated(started_cluster): assert node3.query("SELECT COUNT() FROM test_table") == "2\n" assert node4.query("SELECT COUNT() FROM test_table") == "2\n" + for node in [node1, node2, node3, node4]: + node.query("TRUNCATE TABLE test_table") + def test_alter_replicated_on_cluster(started_cluster): for node in [node1, node3]: @@ -133,3 +136,6 @@ def test_alter_replicated_on_cluster(started_cluster): assert node2.query("SELECT COUNT() FROM test_table_replicated") == "2\n" assert node3.query("SELECT COUNT() FROM test_table_replicated") == "2\n" assert node4.query("SELECT COUNT() FROM test_table_replicated") == "2\n" + + for node in [node1, node2, node3, node4]: + node.query("TRUNCATE TABLE test_table_replicated") From 65f5f628a9c47eb4dfa729e26fde131ad7bdc5e6 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 5 Aug 2024 14:46:41 +0000 Subject: [PATCH 156/371] Revert "Update docker/test/integration/runner/Dockerfile" This reverts commit d296e62bf363d7dfab9a5bf6925b67b5e4188151. --- docker/test/integration/runner/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 71cf3a16967..ceb8a1b2b58 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -47,7 +47,7 @@ RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - \ && add-apt-repository "deb https://download.docker.com/linux/ubuntu $(lsb_release -c -s) ${DOCKER_CHANNEL}" \ && apt-get update \ && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ - docker-ce="5:27.0.3*" \ + docker-ce \ && rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ From 6573482f9f030e93773c6f52cac15f9e79cd0dbe Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 5 Aug 2024 14:52:08 +0000 Subject: [PATCH 157/371] Revert "Unpin docker-ce in integration-tests-runner" This reverts commit 340214a246cd1c35d96cfb21be0576d87e05fea0. --- docker/test/integration/runner/Dockerfile | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index ceb8a1b2b58..d250b746e7d 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -43,11 +43,13 @@ ENV TZ=Etc/UTC RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone ENV DOCKER_CHANNEL stable +# Unpin the docker version after the release 24.0.3 is released +# https://github.com/moby/moby/issues/45770#issuecomment-1618255130 RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - \ && add-apt-repository "deb https://download.docker.com/linux/ubuntu $(lsb_release -c -s) ${DOCKER_CHANNEL}" \ && apt-get update \ && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ - docker-ce \ + docker-ce='5:23.*' \ && rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ From cb6baefa948ef1270ce9454f72075ac10bf6e729 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 5 Aug 2024 21:49:36 +0100 Subject: [PATCH 158/371] 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 24e1bfdb686bf4846c032eab49bc86c5389865ae Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 6 Aug 2024 07:37:00 +0000 Subject: [PATCH 159/371] Join engine support OPTIMIZE query Signed-off-by: Duc Canh Le --- src/Interpreters/HashJoin/HashJoin.cpp | 28 +++++++------ src/Interpreters/HashJoin/HashJoin.h | 2 +- src/Storages/StorageJoin.cpp | 42 +++++++++++++++++++ src/Storages/StorageJoin.h | 12 ++++++ .../03204_storage_join_optimize.reference | 10 +++++ .../03204_storage_join_optimize.sql | 5 +++ 6 files changed, 86 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/03204_storage_join_optimize.reference create mode 100644 tests/queries/0_stateless/03204_storage_join_optimize.sql diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 0c7cad4360d..a621ce16fb1 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -648,10 +648,8 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); } -void HashJoin::shrinkStoredBlocksToFit(size_t & total_bytes_in_join) +void HashJoin::shrinkStoredBlocksToFit(size_t & total_bytes_in_join, bool force_optimize) { - if (shrink_blocks) - return; /// Already shrunk Int64 current_memory_usage = getCurrentQueryMemoryUsage(); Int64 query_memory_usage_delta = current_memory_usage - memory_usage_before_adding_blocks; @@ -659,15 +657,21 @@ void HashJoin::shrinkStoredBlocksToFit(size_t & total_bytes_in_join) auto max_total_bytes_in_join = table_join->sizeLimits().max_bytes; - /** If accounted data size is more than half of `max_bytes_in_join` - * or query memory consumption growth from the beginning of adding blocks (estimation of memory consumed by join using memory tracker) - * is bigger than half of all memory available for query, - * then shrink stored blocks to fit. - */ - shrink_blocks = (max_total_bytes_in_join && total_bytes_in_join > max_total_bytes_in_join / 2) || - (max_total_bytes_for_query && query_memory_usage_delta > max_total_bytes_for_query / 2); - if (!shrink_blocks) - return; + if (!force_optimize) + { + if (shrink_blocks) + return; /// Already shrunk + + /** If accounted data size is more than half of `max_bytes_in_join` + * or query memory consumption growth from the beginning of adding blocks (estimation of memory consumed by join using memory tracker) + * is bigger than half of all memory available for query, + * then shrink stored blocks to fit. + */ + shrink_blocks = (max_total_bytes_in_join && total_bytes_in_join > max_total_bytes_in_join / 2) || + (max_total_bytes_for_query && query_memory_usage_delta > max_total_bytes_for_query / 2); + if (!shrink_blocks) + return; + } LOG_DEBUG(log, "Shrinking stored blocks, memory consumption is {} {} calculated by join, {} {} by memory tracker", ReadableSize(total_bytes_in_join), max_total_bytes_in_join ? fmt::format("/ {}", ReadableSize(max_total_bytes_in_join)) : "", diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 0b115b9fdbb..00f5ef6d214 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -372,7 +372,7 @@ public: void debugKeys() const; - void shrinkStoredBlocksToFit(size_t & total_bytes_in_join); + void shrinkStoredBlocksToFit(size_t & total_bytes_in_join, bool force_optimize = false); void setMaxJoinedBlockRows(size_t value) { max_joined_block_rows = value; } diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index a0d6cf11b64..695c41d3a62 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -75,6 +75,7 @@ StorageJoin::StorageJoin( table_join = std::make_shared(limits, use_nulls, kind, strictness, key_names); join = std::make_shared(table_join, getRightSampleBlock(), overwrite); restore(); + optimizeUnlocked(); } RWLockImpl::LockHolder StorageJoin::tryLockTimedWithContext(const RWLock & lock, RWLockImpl::Type type, ContextPtr context) const @@ -99,6 +100,47 @@ SinkToStoragePtr StorageJoin::write(const ASTPtr & query, const StorageMetadataP return StorageSetOrJoinBase::write(query, metadata_snapshot, context, /*async_insert=*/false); } +bool StorageJoin::optimize( + const ASTPtr & /*query*/, + const StorageMetadataPtr & /*metadata_snapshot*/, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Names & /* deduplicate_by_columns */, + bool cleanup, + ContextPtr context) +{ + + if (partition) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Partition cannot be specified when optimizing table of type EmbeddedRocksDB"); + + if (final) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "FINAL cannot be specified when optimizing table of type EmbeddedRocksDB"); + + if (deduplicate) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DEDUPLICATE cannot be specified when optimizing table of type EmbeddedRocksDB"); + + if (cleanup) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "CLEANUP cannot be specified when optimizing table of type EmbeddedRocksDB"); + + std::lock_guard mutate_lock(mutate_mutex); + TableLockHolder lock_holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, context); + + optimizeUnlocked(); + return true; +} + +void StorageJoin::optimizeUnlocked() +{ + size_t current_bytes = join->getTotalByteCount(); + size_t dummy = current_bytes; + join->shrinkStoredBlocksToFit(dummy, true); + + size_t optimized_bytes = join->getTotalByteCount(); + if (current_bytes > optimized_bytes) + LOG_INFO(getLogger("StorageJoin"), "Optimized Join storage from {} to {} bytes", current_bytes, optimized_bytes); +} + void StorageJoin::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr context, TableExclusiveLockHolder &) { std::lock_guard mutate_lock(mutate_mutex); diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index c76df0cb452..10a551b4063 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -61,6 +61,18 @@ public: 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 /*context*/) override; + + void optimizeUnlocked(); + Pipe read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, diff --git a/tests/queries/0_stateless/03204_storage_join_optimize.reference b/tests/queries/0_stateless/03204_storage_join_optimize.reference new file mode 100644 index 00000000000..af98bcd6397 --- /dev/null +++ b/tests/queries/0_stateless/03204_storage_join_optimize.reference @@ -0,0 +1,10 @@ +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 diff --git a/tests/queries/0_stateless/03204_storage_join_optimize.sql b/tests/queries/0_stateless/03204_storage_join_optimize.sql new file mode 100644 index 00000000000..03a4658ba6c --- /dev/null +++ b/tests/queries/0_stateless/03204_storage_join_optimize.sql @@ -0,0 +1,5 @@ +CREATE TABLE dict_03204 (k UInt64, v UInt64) ENGINE = Join(ANY, LEFT, k); +INSERT INTO dict_03204 SELECT number, number FROM numbers(10); +OPTIMIZE TABLE dict_03204; +SELECT * FROM dict_03204 ORDER BY k; +DROP TABLE dict_03204; From 11fd263be6e24ee4cdc3a51ac497510c82837fa5 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 1 Aug 2024 08:35:05 +0000 Subject: [PATCH 160/371] 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 161/371] 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 d7803ca621ca511f0bf2abe0fbc332658b5c2919 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 6 Aug 2024 10:48:10 +0000 Subject: [PATCH 162/371] small fix in log Signed-off-by: Duc Canh Le --- src/Storages/StorageJoin.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 695c41d3a62..9dace45d2ac 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -112,16 +112,16 @@ bool StorageJoin::optimize( { if (partition) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Partition cannot be specified when optimizing table of type EmbeddedRocksDB"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Partition cannot be specified when optimizing table of type Join"); if (final) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "FINAL cannot be specified when optimizing table of type EmbeddedRocksDB"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "FINAL cannot be specified when optimizing table of type Join"); if (deduplicate) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DEDUPLICATE cannot be specified when optimizing table of type EmbeddedRocksDB"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DEDUPLICATE cannot be specified when optimizing table of type Join"); if (cleanup) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "CLEANUP cannot be specified when optimizing table of type EmbeddedRocksDB"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "CLEANUP cannot be specified when optimizing table of type Join"); std::lock_guard mutate_lock(mutate_mutex); TableLockHolder lock_holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, context); From 995187006a8c2500ddb7fa234f3443c75d900be4 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 5 Aug 2024 20:23:41 +0200 Subject: [PATCH 163/371] 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 164/371] 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 97f1cfc232c331504754db4ba4221a282c690bb6 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Wed, 24 Jul 2024 14:32:35 +0000 Subject: [PATCH 165/371] 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 166/371] 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 167/371] 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 168/371] 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 169/371] 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 170/371] 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 171/371] 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 172/371] 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 173/371] 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 174/371] 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 175/371] 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 176/371] 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 177/371] 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 178/371] 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 179/371] 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 180/371] 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 181/371] 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 182/371] 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 183/371] 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 184/371] 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 185/371] 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 186/371] 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 187/371] 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 188/371] 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 df2675fad0d1bcb79c8a2d7edd0c08b1da49a945 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Thu, 18 Jul 2024 12:32:16 +0200 Subject: [PATCH 189/371] [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 190/371] 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 191/371] 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 192/371] 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 193/371] 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 194/371] 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 195/371] 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 196/371] 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 197/371] 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 198/371] 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 199/371] 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 200/371] 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 201/371] 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 202/371] 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 203/371] 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 8cfcf341aeaedd4defb5ff1b2dbb05578aceb1d6 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 6 Aug 2024 18:26:52 +0000 Subject: [PATCH 204/371] fix --- tests/integration/test_storage_hdfs/test.py | 23 ++++++++++++++------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 4aac0142026..7597fdcd229 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -610,44 +610,48 @@ def test_format_detection(started_cluster): def test_schema_inference_with_globs(started_cluster): + fs = HdfsClient(hosts=started_cluster.hdfs_ip) + dir = "/test_schema_inference_with_globs" + fs.mkdirs(dir) 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{dir}/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" + f"insert into table function hdfs('hdfs://hdfs1:9000{dir}/data2.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select 0" ) result = node1.query( - f"desc hdfs('hdfs://hdfs1:9000/data*.jsoncompacteachrow') settings input_format_json_infer_incomplete_types_as_strings=0" + f"desc hdfs('hdfs://hdfs1:9000{dir}/data*.jsoncompacteachrow') settings input_format_json_infer_incomplete_types_as_strings=0" ) assert result.strip() == "c1\tNullable(Int64)" result = node1.query( - f"select * from hdfs('hdfs://hdfs1:9000/data*.jsoncompacteachrow') settings input_format_json_infer_incomplete_types_as_strings=0" + f"select * from hdfs('hdfs://hdfs1:9000{dir}/data*.jsoncompacteachrow') settings input_format_json_infer_incomplete_types_as_strings=0" ) 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{dir}/data3.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL" ) filename = "data{1,3}.jsoncompacteachrow" result = node1.query_and_get_error( - f"desc hdfs('hdfs://hdfs1:9000/{filename}') settings schema_inference_use_cache_for_hdfs=0, input_format_json_infer_incomplete_types_as_strings=0" + f"desc hdfs('hdfs://hdfs1:9000{dir}/{filename}') settings schema_inference_use_cache_for_hdfs=0, input_format_json_infer_incomplete_types_as_strings=0" ) 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{dir}/data0.jsoncompacteachrow', 'TSV', 'x String') select '[123;]'" ) result = node1.query_and_get_error( - f"desc hdfs('hdfs://hdfs1:9000/data*.jsoncompacteachrow') settings schema_inference_use_cache_for_hdfs=0, input_format_json_infer_incomplete_types_as_strings=0" + f"desc hdfs('hdfs://hdfs1:9000{dir}/data*.jsoncompacteachrow') settings schema_inference_use_cache_for_hdfs=0, input_format_json_infer_incomplete_types_as_strings=0" ) assert "CANNOT_EXTRACT_TABLE_STRUCTURE" in result + fs.delete(dir, recursive=True) def test_insert_select_schema_inference(started_cluster): @@ -694,6 +698,7 @@ def test_cluster_macro(started_cluster): def test_virtual_columns_2(started_cluster): hdfs_api = started_cluster.hdfs_api + fs = HdfsClient(hosts=started_cluster.hdfs_ip) table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_2', 'Parquet', 'a Int32, b String')" @@ -710,6 +715,8 @@ def test_virtual_columns_2(started_cluster): result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "kek" + fs.delete("/parquet_2") + fs.delete("/parquet_3") def check_profile_event_for_query(node, file, profile_event, amount=1): 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 205/371] 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 206/371] 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 207/371] 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 9dec9be1b5254b17f5146e3aaabe3c66f763900d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 6 Aug 2024 21:58:55 +0000 Subject: [PATCH 208/371] Fixed --- .../queries/0_stateless/02558_system_processes_elapsed.sh | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02558_system_processes_elapsed.sh b/tests/queries/0_stateless/02558_system_processes_elapsed.sh index 891ac3cf7bc..8d2615541fd 100755 --- a/tests/queries/0_stateless/02558_system_processes_elapsed.sh +++ b/tests/queries/0_stateless/02558_system_processes_elapsed.sh @@ -9,7 +9,12 @@ while :; do pid=$! sleep 1.5 duration="$($CLICKHOUSE_CLIENT -q "select floor(elapsed) from system.processes where current_database = currentDatabase() and query not like '%system.processes%'")" - kill -INT $pid + # The process might not exist at this point in some exception situations + # maybe it was killed by OOM? + # It safe to skip this iteration. + if ! kill -INT $pid > /dev/null 2>&1; then + continue + fi wait $CLICKHOUSE_CLIENT -q "kill query where current_database = currentDatabase() sync format Null" if [[ $duration -eq 1 ]]; then From a43ed76ae84c7fe68bc82f8a726acf1299fcaec3 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 6 Aug 2024 22:20:12 +0000 Subject: [PATCH 209/371] Fixed session log parallel/sequenced test work within a single fixture --- tests/integration/parallel_skip.json | 5 - .../test_session_log/configs/users.xml | 12 -- tests/integration/test_session_log/test.py | 135 ++++++++---------- 3 files changed, 61 insertions(+), 91 deletions(-) diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 9b8109f3f17..99fa626bd1e 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -94,11 +94,6 @@ "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_setting_in_query", "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_client_suggestions_load", - "test_session_log/test.py::test_grpc_session", - "test_session_log/test.py::test_mysql_session", - "test_session_log/test.py::test_postgres_session", - "test_session_log/test.py::test_parallel_sessions", - "test_ttl_move/test.py::TestCancelBackgroundMoving::test_cancel_background_moving_on_stop_moves_query", "test_ttl_move/test.py::TestCancelBackgroundMoving::test_cancel_background_moving_on_table_detach", "test_ttl_move/test.py::TestCancelBackgroundMoving::test_cancel_background_moving_on_zookeeper_disconnect", diff --git a/tests/integration/test_session_log/configs/users.xml b/tests/integration/test_session_log/configs/users.xml index 0416dfadc8a..766fdbcf00f 100644 --- a/tests/integration/test_session_log/configs/users.xml +++ b/tests/integration/test_session_log/configs/users.xml @@ -7,17 +7,5 @@ - - pass - - - pass - - - pass - - - pass -
\ No newline at end of file diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py index 5e424610ba2..0eb614f7aa9 100644 --- a/tests/integration/test_session_log/test.py +++ b/tests/integration/test_session_log/test.py @@ -60,6 +60,19 @@ def next_session_id(): return str(session_id) +user_counter = 0 + + +def create_unique_user(prefix): + global user_counter + user_counter += 1 + user_name = f"{prefix}_{os.getppid()}_{user_counter}" + instance.query( + f"CREATE USER {user_name} IDENTIFIED WITH plaintext_password BY 'pass'" + ) + return user_name + + def grpc_query(query, user_, pass_, raise_exception): try: query_info = clickhouse_grpc_pb2.QueryInfo( @@ -131,6 +144,37 @@ def wait_for_corresponding_login_success_and_logout(user, expected_login_count): logins_and_logouts = instance.query(sql) +def check_session_log(user): + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + f"SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='{user}' AND type = 'LoginSuccess'" + ) + assert login_success_records == f"{user}\t1\t1\n" + logout_records = instance.query( + f"SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='{user}' AND type = 'Logout'" + ) + assert logout_records == f"{user}\t1\t1\n" + login_failure_records = instance.query( + f"SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='{user}' AND type = 'LoginFailure'" + ) + assert login_failure_records == f"{user}\t1\t1\n" + + wait_for_corresponding_login_success_and_logout(user, 1) + + +def session_log_test(prefix, query_function): + user = create_unique_user(prefix) + wrong_user = "wrong_" + user + + query_function("SELECT 1", user, "pass", False) + query_function("SELECT 2", user, "wrong_pass", True) + query_function("SELECT 3", wrong_user, "pass", True) + + check_session_log(user) + + instance.query(f"DROP USER {user}") + + @pytest.fixture(scope="module") def started_cluster(): try: @@ -145,78 +189,21 @@ def started_cluster(): def test_grpc_session(started_cluster): - grpc_query("SELECT 1", "grpc_user", "pass", False) - grpc_query("SELECT 2", "grpc_user", "wrong_pass", True) - grpc_query("SELECT 3", "wrong_grpc_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "grpc_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'Logout'" - ) - assert logout_records == "grpc_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "grpc_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" + session_log_test("grpc", grpc_query) def test_mysql_session(started_cluster): - mysql_query("SELECT 1", "mysql_user", "pass", False) - mysql_query("SELECT 2", "mysql_user", "wrong_pass", True) - mysql_query("SELECT 3", "wrong_mysql_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "mysql_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'Logout'" - ) - assert logout_records == "mysql_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "mysql_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" + session_log_test("mysql", mysql_query) def test_postgres_session(started_cluster): - postgres_query("SELECT 1", "postgres_user", "pass", False) - postgres_query("SELECT 2", "postgres_user", "wrong_pass", True) - postgres_query("SELECT 3", "wrong_postgres_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "postgres_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'Logout'" - ) - assert logout_records == "postgres_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "postgres_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" + session_log_test("postgres", postgres_query) def test_parallel_sessions(started_cluster): + user = create_unique_user("parallel") + wrong_user = "wrong_" + user + thread_list = [] for _ in range(10): # Sleep time does not significantly matter here, @@ -226,7 +213,7 @@ def test_parallel_sessions(started_cluster): target=function, args=( f"SELECT sleep({random.uniform(0.03, 0.04)})", - "parallel_user", + user, "pass", False, ), @@ -237,7 +224,7 @@ def test_parallel_sessions(started_cluster): target=function, args=( f"SELECT sleep({random.uniform(0.03, 0.04)})", - "parallel_user", + user, "wrong_pass", True, ), @@ -248,7 +235,7 @@ def test_parallel_sessions(started_cluster): target=function, args=( f"SELECT sleep({random.uniform(0.03, 0.04)})", - "wrong_parallel_user", + wrong_user, "pass", True, ), @@ -261,38 +248,38 @@ def test_parallel_sessions(started_cluster): instance.query("SYSTEM FLUSH LOGS") port_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user'" + f"SELECT COUNT(*) FROM system.session_log WHERE user = '{user}'" ) assert port_0_sessions == "90\n" port_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_port = 0" + f"SELECT COUNT(*) FROM system.session_log WHERE user = '{user}' AND client_port = 0" ) assert port_0_sessions == "0\n" address_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_address = toIPv6('::')" + f"SELECT COUNT(*) FROM system.session_log WHERE user = '{user}' AND client_address = toIPv6('::')" ) assert address_0_sessions == "0\n" grpc_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'gRPC'" + f"SELECT COUNT(*) FROM system.session_log WHERE user = '{user}' AND interface = 'gRPC'" ) assert grpc_sessions == "30\n" mysql_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'MySQL'" + f"SELECT COUNT(*) FROM system.session_log WHERE user = '{user}' AND interface = 'MySQL'" ) assert mysql_sessions == "30\n" postgres_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'PostgreSQL'" + f"SELECT COUNT(*) FROM system.session_log WHERE user = '{user}' AND interface = 'PostgreSQL'" ) assert postgres_sessions == "30\n" - wait_for_corresponding_login_success_and_logout("parallel_user", 30) + wait_for_corresponding_login_success_and_logout(user, 30) logout_failure_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'" + f"SELECT COUNT(*) FROM system.session_log WHERE user = '{user}' AND type = 'LoginFailure'" ) assert logout_failure_sessions == "30\n" From 5b3692b4f02421d56692107365ad4cc7a3297418 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 7 Aug 2024 00:29:19 +0200 Subject: [PATCH 210/371] 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 211/371] 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 212/371] 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 213/371] 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 214/371] 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 215/371] 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 216/371] 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 217/371] 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 218/371] 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 219/371] 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 220/371] 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 221/371] 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 222/371] 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 223/371] 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 224/371] 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 225/371] 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 226/371] 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 227/371] 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 228/371] 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 229/371] 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 230/371] 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 231/371] 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 232/371] 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 233/371] 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 234/371] 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 235/371] 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 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 236/371] 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 237/371] 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 238/371] 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 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 239/371] 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 016d1fea6d72c25179633f72f8ca8338dc59338f Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 7 Aug 2024 13:58:03 +0200 Subject: [PATCH 240/371] 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 241/371] 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 242/371] 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 243/371] 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 244/371] 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 245/371] 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 246/371] 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 247/371] 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 248/371] 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 249/371] 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 250/371] 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 251/371] 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 364e973ef7aaa561780c50cd795b9edbcae51a41 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 7 Aug 2024 16:23:47 +0200 Subject: [PATCH 252/371] Ping CI 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 253/371] 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 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 254/371] 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 255/371] 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 256/371] 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 257/371] 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 258/371] 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 259/371] 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 260/371] 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 261/371] 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 262/371] 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 263/371] 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 4fb1febe4859368de7ff3c8b73f2cc16d398c089 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Wed, 7 Aug 2024 18:51:24 +0200 Subject: [PATCH 264/371] Update table.md --- docs/en/sql-reference/statements/create/table.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 9c8984d698f..7428e6cd6ca 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -241,12 +241,12 @@ CREATE OR REPLACE TABLE test ( id UInt64, size_bytes Int64, - size String Alias formatReadableSize(size_bytes) + size String ALIAS formatReadableSize(size_bytes) ) ENGINE = MergeTree ORDER BY id; -INSERT INTO test Values (1, 4678899); +INSERT INTO test VALUES (1, 4678899); SELECT id, size_bytes, size FROM test; ┌─id─┬─size_bytes─┬─size─────┐ @@ -497,7 +497,7 @@ If you perform a SELECT query mentioning a specific value in an encrypted column ```sql CREATE TABLE mytable ( - x String Codec(AES_128_GCM_SIV) + x String CODEC(AES_128_GCM_SIV) ) ENGINE = MergeTree ORDER BY x; ``` From 3b48a1a92c1294cae71d6287a8adbc49e0b9890d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 7 Aug 2024 14:01:40 -0300 Subject: [PATCH 265/371] Update http.md --- docs/en/interfaces/http.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index f5b6326fa96..03fdfa048c8 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -379,7 +379,7 @@ You can mitigate this problem by enabling `wait_end_of_query=1` ([Response Buffe However, this does not completely solve the problem because the result must still fit within the `http_response_buffer_size`, and other settings like `send_progress_in_http_headers` can interfere with the delay of the header. The only way to catch all errors is to analyze the HTTP body before parsing it using the required format. -### Queries with Parameters {#cli-queries-with-parameters} +## Queries with Parameters {#cli-queries-with-parameters} You can create a query with parameters and pass values for them from the corresponding HTTP request parameters. For more information, see [Queries with Parameters for CLI](../interfaces/cli.md#cli-queries-with-parameters). From 06d154055f9e233180f13585e43e2992ae5ccfdf Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 7 Aug 2024 19:19:33 +0200 Subject: [PATCH 266/371] 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 267/371] 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 268/371] 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 269/371] 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 270/371] 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 271/371] 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 272/371] 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 273/371] 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 d81b5239debaf01b74521511db44d6cb4cd419c4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 7 Aug 2024 21:37:01 +0200 Subject: [PATCH 274/371] 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 275/371] 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 276/371] 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 277/371] 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 278/371] 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 279/371] 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 280/371] 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 55ad7d30946d609159fe5ae9156f02f5b160585a Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 8 Aug 2024 00:08:12 +0200 Subject: [PATCH 281/371] 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 ff8ce505d752eff1c867d73b47e39a03f0f13622 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 8 Aug 2024 00:20:16 +0200 Subject: [PATCH 282/371] Revert "Bump rocksdb from v8.10 to v9.4 + enable jemalloc and liburing" --- contrib/CMakeLists.txt | 2 +- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 44 ++++++++++------------------ 3 files changed, 18 insertions(+), 30 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index eb3afe0ccdf..977efda15ff 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -71,6 +71,7 @@ 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 @@ -147,7 +148,6 @@ 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 5f003e4a22d..49ce8a1064d 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 5f003e4a22d2e48e37c98d9620241237cd30dd24 +Subproject commit 49ce8a1064dd1ad89117899839bf136365e49e79 diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 7e5e9a28d0f..57c056532c6 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -5,38 +5,36 @@ 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 (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) +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() @@ -90,7 +88,6 @@ 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 @@ -107,7 +104,6 @@ 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 @@ -128,7 +124,6 @@ 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 @@ -186,7 +181,6 @@ 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 @@ -374,7 +368,6 @@ 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 @@ -395,7 +388,6 @@ 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 @@ -426,18 +418,14 @@ 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 8426e0d5e5d7f102fd57a45c82ae6acccda65369 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 7 Aug 2024 16:44:10 +0800 Subject: [PATCH 283/371] 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 284/371] 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 285/371] 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 286/371] 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 287/371] 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 288/371] 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 289/371] 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 290/371] 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 291/371] 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 292/371] 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 5c97205742ff12f28cab2b853a9473cbb59edfe3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 07:23:10 +0000 Subject: [PATCH 293/371] 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 294/371] 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 71a761232c36f5ec3c30df7f3c4c3294641e414f Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Thu, 8 Aug 2024 08:45:42 +0100 Subject: [PATCH 295/371] 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 296/371] 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 297/371] 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 298/371] 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 d0f35ce6a60e13b8aff9687a45e293ce89693241 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Aug 2024 10:29:01 +0200 Subject: [PATCH 299/371] 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 300/371] 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 301/371] 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 302/371] 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 303/371] 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 304/371] 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 305/371] 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 306/371] 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 307/371] 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 308/371] 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 309/371] 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 e90487fd54a5c87eb0875191e7547fc2b7f2e229 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 8 Aug 2024 12:57:50 +0200 Subject: [PATCH 310/371] 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 311/371] 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 312/371] 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 313/371] 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 314/371] 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 315/371] 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 316/371] 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 317/371] 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 318/371] 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 55b2000d38e0bc6282714fdb1204d450437433ec Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 13:58:43 +0000 Subject: [PATCH 319/371] 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 33ba78ee42bc85690dce69c82fd51d723a6d2eab Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 8 Aug 2024 17:47:12 +0200 Subject: [PATCH 320/371] 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 321/371] 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 322/371] 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 323/371] 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 324/371] 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 325/371] 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 326/371] 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 327/371] 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 328/371] 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 329/371] 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 330/371] 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 331/371] 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 332/371] 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 333/371] 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 334/371] 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 335/371] 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 336/371] [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 337/371] 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 338/371] 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 47f429a52435d84b3e1cee06202e743c11bcba0d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 20:54:29 +0000 Subject: [PATCH 339/371] 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 340/371] 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 341/371] 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 a497a2391455de21dec19b365cc939defdc56b1e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Aug 2024 08:33:11 +0000 Subject: [PATCH 342/371] 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 343/371] 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 344/371] 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 345/371] 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 346/371] 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 347/371] 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 2657e2b3ef2f98802fd2b8ebcd359fe756b709c6 Mon Sep 17 00:00:00 2001 From: Graham Campbell Date: Fri, 9 Aug 2024 11:08:41 +0100 Subject: [PATCH 348/371] 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 ade1228b9578d5c0d7124a9d5c40ac3207e48074 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 13:01:00 +0200 Subject: [PATCH 349/371] 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 350/371] 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 351/371] 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 352/371] 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 353/371] 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 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 354/371] 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 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 355/371] 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 65ebcd6f21b26144cb47e6b71c939517b1fb38a2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Aug 2024 13:55:47 +0000 Subject: [PATCH 356/371] 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 c13d348d1e8a467b9d16fc83214ef574752092e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 19:56:50 +0200 Subject: [PATCH 357/371] 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 358/371] 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 9b9fff4232d80e579b1d23ced8bfbb1b2c5e2147 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Sat, 10 Aug 2024 08:48:08 +0200 Subject: [PATCH 359/371] Push CI From e582118544f3c49c3c6600ac8fa252151714d25f Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 10 Aug 2024 13:09:40 +0200 Subject: [PATCH 360/371] 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 18d9bb2ade4e98051df007663a387eb74146c26f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 10 Aug 2024 16:25:32 +0200 Subject: [PATCH 361/371] 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 362/371] 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 d53513a81a10b8230a30fdbb386aca1d067cbcfa Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 10 Aug 2024 18:12:34 +0000 Subject: [PATCH 363/371] 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 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 364/371] 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 1142305b113e261d0c8910c0b622ba94727fe78d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 10 Aug 2024 09:53:43 +0200 Subject: [PATCH 365/371] 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 366/371] 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 367/371] 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 368/371] 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 4ef3fe416debecefcea4d7336aac7c679092cf0c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 11 Aug 2024 13:08:53 +0000 Subject: [PATCH 369/371] 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 d314e5aa45fb8ac91324721ab278185b09437a40 Mon Sep 17 00:00:00 2001 From: Vladimir Varankin Date: Sun, 11 Aug 2024 18:37:29 +0200 Subject: [PATCH 370/371] 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 371/371] 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 $?